diff --git a/.golangci.yml b/.golangci.yml index 48d011b..4be8758 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -76,3 +76,6 @@ linters: - "github.com/tarantool/go-iproto" - "github.com/vmihailenco/msgpack/v5" - "github.com/gojuno/minimock/v3" + - "go.etcd.io/etcd/client/v3" + - "go.etcd.io/etcd/tests/v3" + - "go.etcd.io/etcd/api/v3" diff --git a/driver/etcd/etcd.go b/driver/etcd/etcd.go index 152b556..00801eb 100644 --- a/driver/etcd/etcd.go +++ b/driver/etcd/etcd.go @@ -3,72 +3,262 @@ package etcd import ( + "bytes" "context" + "errors" "fmt" etcd "go.etcd.io/etcd/client/v3" "github.com/tarantool/go-storage/driver" + "github.com/tarantool/go-storage/kv" "github.com/tarantool/go-storage/operation" "github.com/tarantool/go-storage/predicate" "github.com/tarantool/go-storage/tx" "github.com/tarantool/go-storage/watch" ) +// Client defines the minimal interface needed for etcd operations. +// This allows for easier testing and mock implementations. +type Client interface { + // Txn creates a new transaction. + Txn(ctx context.Context) etcd.Txn +} + +// Watcher defines the interface for watching etcd changes. +// This extends the etcd.Watcher interface to match our usage pattern. +type Watcher interface { + // Watch watches for changes on a key (using etcd's signature). + Watch(ctx context.Context, key string, opts ...etcd.OpOption) etcd.WatchChan + // Close closes the watcher. + Close() error +} + +// WatcherFactory creates new watchers from a client. +type WatcherFactory interface { + // NewWatcher creates a new watcher. + NewWatcher(client Client) Watcher +} + // Driver is an etcd implementation of the storage driver interface. // It uses etcd as the underlying key-value storage backend. type Driver struct { - client *etcd.Client // etcd client instance.. + client Client // etcd client interface. + watcherFactory WatcherFactory // factory for creating watchers. } var ( _ driver.Driver = &Driver{} //nolint:exhaustruct + + // Static error definitions to avoid dynamic errors. + errUnsupportedPredicateTarget = errors.New("unsupported predicate target") + errValuePredicateRequiresBytes = errors.New("value predicate requires []byte value") + errUnsupportedValueOperation = errors.New("unsupported operation for value predicate") + errVersionPredicateRequiresInt = errors.New("version predicate requires int64 value") + errUnsupportedVersionOperation = errors.New("unsupported operation for version predicate") + errUnsupportedOperationType = errors.New("unsupported operation type") ) -// New creates a new etcd driver instance. -// It establishes a connection to the etcd cluster using the provided endpoints. -func New(ctx context.Context, endpoints []string) (*Driver, error) { - client, err := etcd.New(etcd.Config{ - Context: ctx, - Endpoints: endpoints, - AutoSyncInterval: 0, - DialTimeout: 0, - DialKeepAliveTime: 0, - DialKeepAliveTimeout: 0, - MaxCallSendMsgSize: 0, - MaxCallRecvMsgSize: 0, - TLS: nil, - Username: "", - Password: "", - RejectOldCluster: false, - DialOptions: nil, - Logger: nil, - LogConfig: nil, - PermitWithoutStream: false, - MaxUnaryRetries: 0, - BackoffWaitBetween: 0, - BackoffJitterFraction: 0, - }) - if err != nil { - return nil, fmt.Errorf("failed to create etcd client: %w", err) +// etcdClientAdapter wraps etcd.Client to implement our Client interface. +type etcdClientAdapter struct { + client *etcd.Client +} + +func (a *etcdClientAdapter) Txn(ctx context.Context) etcd.Txn { + return a.client.Txn(ctx) +} + +// etcdWatcherAdapter wraps etcd.Watcher to implement our Watcher interface. +type etcdWatcherAdapter struct { + watcher etcd.Watcher +} + +func (a *etcdWatcherAdapter) Watch(ctx context.Context, key string, opts ...etcd.OpOption) etcd.WatchChan { + return a.watcher.Watch(ctx, key, opts...) +} + +func (a *etcdWatcherAdapter) Close() error { + return fmt.Errorf("failed to close: %w", a.watcher.Close()) +} + +// etcdWatcherFactory implements WatcherFactory for etcd clients. +type etcdWatcherFactory struct{} + +func (f *etcdWatcherFactory) NewWatcher(client Client) Watcher { + // For etcd clients, we need access to the underlying client. + if adapter, ok := client.(*etcdClientAdapter); ok { + return &etcdWatcherAdapter{ + watcher: etcd.NewWatcher(adapter.client), + } + } + // For other implementations, return a no-op watcher. + return &noopWatcher{} +} + +// noopWatcher is a no-op implementation of Watcher for non-etcd clients. +type noopWatcher struct{} + +func (w *noopWatcher) Watch(_ context.Context, _ string, _ ...etcd.OpOption) etcd.WatchChan { + ch := make(chan etcd.WatchResponse) + close(ch) + + return ch +} + +func (w *noopWatcher) Close() error { + return nil +} + +// New creates a new etcd driver instance using an existing etcd client. +// The client should be properly configured and connected to an etcd cluster. +func New(client *etcd.Client) *Driver { + return &Driver{ + client: &etcdClientAdapter{client: client}, + watcherFactory: &etcdWatcherFactory{}, + } +} + +// NewWithInterfaces creates a new etcd driver instance using interface abstractions. +// This is useful for testing with mock clients. +func NewWithInterfaces(client Client, watcherFactory WatcherFactory) *Driver { + if watcherFactory == nil { + watcherFactory = &etcdWatcherFactory{} } - return &Driver{client: client}, nil + return &Driver{ + client: client, + watcherFactory: watcherFactory, + } } // Execute executes a transactional operation with conditional logic. // It processes predicates to determine whether to execute thenOps or elseOps. func (d Driver) Execute( - _ context.Context, - _ []predicate.Predicate, - _ []operation.Operation, - _ []operation.Operation, + ctx context.Context, + predicates []predicate.Predicate, + thenOps []operation.Operation, + elseOps []operation.Operation, ) (tx.Response, error) { - panic("implement me") + txn := d.client.Txn(ctx) + + convertedPredicates, err := predicatesToCmps(predicates) + if err != nil { + return tx.Response{}, fmt.Errorf("failed to convert predicates: %w", err) + } + + txn.If(convertedPredicates...) + + thenEtcdOps, err := operationsToEtcdOps(thenOps) + if err != nil { + return tx.Response{}, fmt.Errorf("failed to convert then operations: %w", err) + } + + txn = txn.Then(thenEtcdOps...) + + elseEtcdOps, err := operationsToEtcdOps(elseOps) + if err != nil { + return tx.Response{}, fmt.Errorf("failed to convert else operations: %w", err) + } + + txn = txn.Else(elseEtcdOps...) + + resp, err := txn.Commit() + if err != nil { + return tx.Response{}, fmt.Errorf("transaction failed: %w", err) + } + + return etcdResponseToTxResponse(resp), nil } +const ( + eventChannelSize = 100 +) + // Watch monitors changes to a specific key and returns a stream of events. // It supports optional watch configuration through the opts parameter. -func (d Driver) Watch(_ context.Context, _ []byte, _ ...watch.Option) (<-chan watch.Event, func(), error) { - panic("implement me") +func (d Driver) Watch(ctx context.Context, key []byte, _ ...watch.Option) (<-chan watch.Event, func(), error) { + eventCh := make(chan watch.Event, eventChannelSize) + + parentWatcher := d.watcherFactory.NewWatcher(d.client) + + go func() { + defer close(eventCh) + + var opts []etcd.OpOption + if bytes.HasSuffix(key, []byte("/")) { + opts = append(opts, etcd.WithPrefix()) + } + + watchChan := parentWatcher.Watch(ctx, string(key), opts...) + + for { + select { + case <-ctx.Done(): + return + case watchResp, ok := <-watchChan: + if !ok { + return + } + + if watchResp.Err() != nil { + continue + } + + for range watchResp.Events { + select { + case eventCh <- watch.Event{ + Prefix: key, + }: + case <-ctx.Done(): + return + } + } + } + } + }() + + return eventCh, func() { + _ = parentWatcher.Close() + }, nil +} + +// etcdResponseToTxResponse converts an etcd transaction response to tx.Response. +func etcdResponseToTxResponse(resp *etcd.TxnResponse) tx.Response { + results := make([]tx.RequestResponse, 0, len(resp.Responses)) + + for _, etcdResp := range resp.Responses { + var values []kv.KeyValue + + switch { + case etcdResp.GetResponseRange() != nil: + getResp := etcdResp.GetResponseRange() + for _, etcdKv := range getResp.Kvs { + values = append(values, kv.KeyValue{ + Key: etcdKv.Key, + Value: etcdKv.Value, + ModRevision: etcdKv.ModRevision, + }) + } + case etcdResp.GetResponsePut() != nil: + // Put operations don't return data. + case etcdResp.GetResponseDeleteRange() != nil: + deleteResp := etcdResp.GetResponseDeleteRange() + for _, etcdKv := range deleteResp.PrevKvs { + values = append(values, kv.KeyValue{ + Key: etcdKv.Key, + Value: etcdKv.Value, + ModRevision: etcdKv.ModRevision, + }) + } + } + + results = append(results, tx.RequestResponse{ + Values: values, + }) + } + + return tx.Response{ + Succeeded: resp.Succeeded, + Results: results, + } } diff --git a/driver/etcd/examples_test.go b/driver/etcd/examples_test.go new file mode 100644 index 0000000..e19eb74 --- /dev/null +++ b/driver/etcd/examples_test.go @@ -0,0 +1,421 @@ +package etcd_test + +// This file provides example tests for the etcd driver using LazyCluster. +// These examples demonstrate real usage with a running etcd instance. +// +// Due to inability to start multiple LazyClusters - we're using one LazyCluster +// and won't start tests in parallel here. + +import ( + "context" + "fmt" + "log" + "testing" + "time" + + etcdclient "go.etcd.io/etcd/client/v3" + etcdfintegration "go.etcd.io/etcd/tests/v3/framework/integration" + etcdintegration "go.etcd.io/etcd/tests/v3/integration" + + "github.com/tarantool/go-storage/driver/etcd" + testingUtils "github.com/tarantool/go-storage/internal/testing" + "github.com/tarantool/go-storage/operation" + "github.com/tarantool/go-storage/predicate" +) + +const ( + exampleTestDialTimeout = 5 * time.Second +) + +// createEtcdDriver creates an etcd driver for examples using the integration framework. +// Returns driver and cleanup function. +func createEtcdDriver(tb testing.TB) (*etcd.Driver, func()) { + tb.Helper() + + etcdfintegration.BeforeTest(tb, etcdfintegration.WithoutGoLeakDetection()) + + cluster := etcdintegration.NewLazyCluster() + + tb.Cleanup(func() { cluster.Terminate() }) + + endpoints := cluster.EndpointsGRPC() + + client, err := etcdclient.New(etcdclient.Config{ + Endpoints: endpoints, + DialTimeout: exampleTestDialTimeout, + + AutoSyncInterval: 0, + DialKeepAliveTime: 0, + DialKeepAliveTimeout: 0, + MaxCallSendMsgSize: 0, + MaxCallRecvMsgSize: 0, + TLS: nil, + Username: "", + Password: "", + RejectOldCluster: false, + DialOptions: nil, + Context: nil, + Logger: nil, + LogConfig: nil, + PermitWithoutStream: false, + MaxUnaryRetries: 0, + BackoffWaitBetween: 0, + BackoffJitterFraction: 0, + }) + if err != nil { + tb.Fatalf("Failed to create etcd client: %v", err) + } + + tb.Cleanup(func() { _ = client.Close() }) + + driver := etcd.New(client) + + return driver, func() {} +} + +// ExampleDriver_Execute_simple demonstrates basic Execute operations with the etcd driver. +// This example shows Put, Get, and Delete operations without predicates. +func ExampleDriver_Execute_simple() { + // Create a testing context for the example. + t := testingUtils.NewT() + defer t.Cleanups() + + ctx := context.Background() + + driver, cleanup := createEtcdDriver(t) + defer cleanup() + + // Example 1: Simple Put operation. + { + key := []byte("/config/app/version") + value := []byte("1.0.0") + + _, err := driver.Execute(ctx, nil, []operation.Operation{ + operation.Put(key, value), + }, nil) + if err != nil { + log.Printf("Put operation failed: %v", err) + return + } + + fmt.Println("Key", string(key), "stored with value:", string(value)) + } + + // Example 2: Simple Get operation. + { + key := []byte("/config/app/version") + + response, err := driver.Execute(ctx, nil, []operation.Operation{ + operation.Get(key), + }, nil) + if err != nil { + log.Printf("Get operation failed: %v", err) + return + } + + if response.Succeeded && len(response.Results) > 0 { + if len(response.Results[0].Values) > 0 { + kv := response.Results[0].Values[0] + fmt.Printf("Retrieved key: %s, value: %s, version: %d\n", + string(kv.Key), string(kv.Value), kv.ModRevision) + } + } + } + + // Example 3: Simple Delete operation. + { + key := []byte("/config/app/version") + + _, err := driver.Execute(ctx, nil, []operation.Operation{ + operation.Delete(key), + }, nil) + if err != nil { + log.Printf("Delete operation failed: %v", err) + return + } + + fmt.Println("Successfully deleted key:", string(key)) + } + + // Example 4: Multiple operations in single transaction. + { + response, err := driver.Execute(ctx, nil, []operation.Operation{ + operation.Put([]byte("/config/app/name"), []byte("MyApp")), + operation.Put([]byte("/config/app/environment"), []byte("production")), + }, nil) + if err != nil { + log.Printf("Multi-put operation failed: %v", err) + return + } + + fmt.Println("Successfully stored", len(response.Results), "configuration items") + } + + // Output: + // Key /config/app/version stored with value: 1.0.0 + // Retrieved key: /config/app/version, value: 1.0.0, version: 2 + // Successfully deleted key: /config/app/version + // Successfully stored 2 configuration items +} + +// ExampleDriver_Execute_with_predicates demonstrates conditional Execute operations using predicates. +// This example shows how to use value and version predicates for conditional execution. +func ExampleDriver_Execute_with_predicates() { + // Create a testing context for the example. + t := testingUtils.NewT() + defer t.Cleanups() + + ctx := context.Background() + + driver, cleanup := createEtcdDriver(t) + defer cleanup() + + // Example 1: Value-based conditional update. + { + key := []byte("/config/app/settings") + currentValue := []byte("old-settings") + newValue := []byte("new-settings") + + _, _ = driver.Execute(ctx, nil, []operation.Operation{ + operation.Put(key, currentValue), + }, nil) + + response, err := driver.Execute(ctx, []predicate.Predicate{ + predicate.ValueEqual(key, []byte("old-settings")), + }, []operation.Operation{ + operation.Put(key, newValue), + }, nil) + if err != nil { + log.Printf("Conditional update failed: %v", err) + return + } + + if response.Succeeded { + fmt.Println("Conditional update succeeded - value was updated") + } else { + fmt.Println("Conditional update failed - value did not match") + } + } + + // Example 2: Version-based conditional update. + { + key := []byte("/config/app/feature") + value := []byte("enabled") + + _, err := driver.Execute(ctx, nil, []operation.Operation{ + operation.Put(key, value), + }, nil) + if err != nil { + log.Printf("Initial update failed: %v", err) + return + } + + getResponse, err := driver.Execute(ctx, nil, []operation.Operation{ + operation.Get(key), + }, nil) + if err != nil { + log.Printf("Get operation failed: %v", err) + return + } + + var currentVersion int64 + if len(getResponse.Results) > 0 && len(getResponse.Results[0].Values) > 0 { + currentVersion = getResponse.Results[0].Values[0].ModRevision + } + + response, err := driver.Execute(ctx, []predicate.Predicate{ + predicate.VersionEqual(key, currentVersion), + }, []operation.Operation{ + operation.Put(key, []byte("disabled")), + }, nil) + if err != nil { + log.Printf("Version-based update failed: %v", err) + return + } + + if response.Succeeded { + fmt.Println("Version-based update succeeded - no concurrent modification") + } else { + fmt.Println("Version-based update failed - version conflict detected") + } + } + + // Example 3: Multiple predicates with Else operations. + { + key1 := []byte("/config/database/host") + key2 := []byte("/config/database/port") + + _, _ = driver.Execute(ctx, nil, []operation.Operation{ + operation.Put(key1, []byte("localhost")), + operation.Put(key2, []byte("5432")), + }, nil) + + response, err := driver.Execute(ctx, []predicate.Predicate{ + predicate.ValueEqual(key1, []byte("localhost")), + predicate.ValueEqual(key2, []byte("5432")), + }, []operation.Operation{ + operation.Put(key1, []byte("new-host")), + operation.Put(key2, []byte("6432")), + }, []operation.Operation{ + operation.Delete(key1), + operation.Delete(key2), + }) + if err != nil { + log.Printf("Multi-predicate transaction failed: %v", err) + return + } + + if response.Succeeded { + fmt.Println("Multi-predicate transaction succeeded - values were updated") + } else { + fmt.Println("Multi-predicate transaction failed - cleanup operations executed") + } + } + + // Output: + // Conditional update succeeded - value was updated + // Version-based update succeeded - no concurrent modification + // Multi-predicate transaction succeeded - values were updated +} + +// ExampleDriver_Watch demonstrates how to use Watch for real-time change notifications. +// This example shows watching individual keys and handling watch events. +func ExampleDriver_Watch() { + // Create a testing context for the example. + t := testingUtils.NewT() + defer t.Cleanups() + + ctx := context.Background() + + driver, cleanup := createEtcdDriver(t) + defer cleanup() + + // Example 1: Basic watch on a single key. + { + key := []byte("/config/app/status") + + watchCtx, cancel := context.WithTimeout(ctx, 30*time.Second) + defer cancel() + + eventCh, stopWatch, err := driver.Watch(watchCtx, key) + if err != nil { + log.Printf("Failed to start watch: %v", err) + return + } + defer stopWatch() + + fmt.Println("Watching for changes on:", string(key)) + + go func() { + time.Sleep(100 * time.Millisecond) + + _, _ = driver.Execute(ctx, nil, []operation.Operation{ + operation.Put(key, []byte("running")), + }, nil) + }() + + select { + case event := <-eventCh: + fmt.Printf("Received watch event for key: %s\n", string(event.Prefix)) + case <-watchCtx.Done(): + fmt.Println("Watch context expired") + } + } + + // Example 2: Watch with multiple operations. + { + key := []byte("/config/database/") + + watchCtx, cancel := context.WithTimeout(ctx, 10*time.Second) + defer cancel() + + eventCh, stopWatch, err := driver.Watch(watchCtx, key) + if err != nil { + log.Printf("Failed to start watch: %v", err) + return + } + defer stopWatch() + + fmt.Println("Watching for changes on prefix:", string(key)) + + go func() { + time.Sleep(100 * time.Millisecond) + + _, _ = driver.Execute(ctx, nil, []operation.Operation{ + operation.Put([]byte("/config/database/host"), []byte("db1")), + }, nil) + + time.Sleep(200 * time.Millisecond) + + _, _ = driver.Execute(ctx, nil, []operation.Operation{ + operation.Put([]byte("/config/database/port"), []byte("5432")), + }, nil) + + time.Sleep(300 * time.Millisecond) + + _, _ = driver.Execute(ctx, nil, []operation.Operation{ + operation.Delete([]byte("/config/database/host")), + }, nil) + }() + + eventCount := 0 + for eventCount < 3 { + select { + case event := <-eventCh: + fmt.Printf("Event %d: change detected on %s\n", eventCount+1, string(event.Prefix)) + + eventCount++ + case <-watchCtx.Done(): + fmt.Println("Watch context expired") + return + } + } + } + + // Example 3: Graceful watch termination. + { + key := []byte("/config/monitoring/metrics") + + watchCtx, cancel := context.WithCancel(ctx) + + eventCh, stopWatch, err := driver.Watch(watchCtx, key) + if err != nil { + log.Printf("Failed to start watch: %v", err) + cancel() + + return + } + + fmt.Println("Started watch with manual control") + + go func() { + time.Sleep(100 * time.Millisecond) + fmt.Println("Stopping watch gracefully...") + stopWatch() + cancel() + }() + + for { + select { + case event, ok := <-eventCh: + if !ok { + return + } + + fmt.Printf("Received event: %s\n", string(event.Prefix)) + case <-watchCtx.Done(): + return + } + } + } + + // Output: + // Watching for changes on: /config/app/status + // Received watch event for key: /config/app/status + // Watching for changes on prefix: /config/database/ + // Event 1: change detected on /config/database/ + // Event 2: change detected on /config/database/ + // Event 3: change detected on /config/database/ + // Started watch with manual control + // Stopping watch gracefully... +} diff --git a/driver/etcd/integration_test.go b/driver/etcd/integration_test.go new file mode 100644 index 0000000..1f863a7 --- /dev/null +++ b/driver/etcd/integration_test.go @@ -0,0 +1,301 @@ +// Package etcd_test provides integration tests for the etcd driver. +// These tests require a running etcd instance and test full functionality. +// +// Due to inability to start multiple LazyClusters - we're using one LazyCluster +// and won't start tests in parallel here. +// +//nolint:paralleltest +package etcd_test + +import ( + "context" + "errors" + "os" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + etcdclient "go.etcd.io/etcd/client/v3" + etcdfintegration "go.etcd.io/etcd/tests/v3/framework/integration" + etcdintegration "go.etcd.io/etcd/tests/v3/integration" + + etcddriver "github.com/tarantool/go-storage/driver/etcd" + "github.com/tarantool/go-storage/kv" + "github.com/tarantool/go-storage/operation" + "github.com/tarantool/go-storage/predicate" +) + +const ( + defaultWaitTimeout = 5 * time.Second + testDialTimeout = 5 * time.Second +) + +// createTestDriver creates an etcd driver for testing using the integration framework. +// Returns driver and cleanup function for simple test scenarios. +func createTestDriver(t *testing.T) (*etcddriver.Driver, func()) { + t.Helper() + + etcdfintegration.BeforeTest(t, etcdfintegration.WithoutGoLeakDetection()) + + cluster := etcdintegration.NewLazyCluster() + + t.Cleanup(func() { cluster.Terminate() }) + + endpoints := cluster.EndpointsGRPC() + + client, err := etcdclient.New(etcdclient.Config{ + Endpoints: endpoints, + DialTimeout: testDialTimeout, + + AutoSyncInterval: 0, + DialKeepAliveTime: 0, + DialKeepAliveTimeout: 0, + MaxCallSendMsgSize: 0, + MaxCallRecvMsgSize: 0, + TLS: nil, + Username: "", + Password: "", + RejectOldCluster: false, + DialOptions: nil, + Context: nil, + Logger: nil, + LogConfig: nil, + PermitWithoutStream: false, + MaxUnaryRetries: 0, + BackoffWaitBetween: 0, + BackoffJitterFraction: 0, + }) + require.NoError(t, err, "Failed to create etcd client") + t.Cleanup(func() { _ = client.Close() }) + + driver := etcddriver.New(client) + + return driver, func() {} +} + +// testKey generates a unique test key to avoid conflicts between tests. +func testKey(t *testing.T, prefix string) []byte { + t.Helper() + + return []byte("/test/" + prefix + "/" + t.Name()) +} + +// putValue is a helper that puts a key-value pair and fails the test on error. +func putValue(ctx context.Context, t *testing.T, driver *etcddriver.Driver, key, value []byte) { + t.Helper() + + response, err := driver.Execute(ctx, nil, []operation.Operation{ + operation.Put(key, value), + }, nil) + require.NoError(t, err, "Put operation failed") + assert.True(t, response.Succeeded, "Put operation should succeed") +} + +// getValue is a helper that gets a value and returns the key-value pair. +func getValue(ctx context.Context, t *testing.T, driver *etcddriver.Driver, key []byte) kv.KeyValue { + t.Helper() + + response, err := driver.Execute(ctx, nil, []operation.Operation{ + operation.Get(key), + }, nil) + require.NoError(t, err, "Get operation failed") + assert.True(t, response.Succeeded, "Get operation should succeed") + require.Len(t, response.Results, 1, "Get operation should return one result") + require.Len(t, response.Results[0].Values, 1, "Get operation should return one value") + + return response.Results[0].Values[0] +} + +// deleteValue is a helper that deletes a key and returns the deleted key-value pair. +func deleteValue(ctx context.Context, t *testing.T, driver *etcddriver.Driver, key []byte) kv.KeyValue { + t.Helper() + + response, err := driver.Execute(ctx, nil, []operation.Operation{ + operation.Delete(key), + }, nil) + + require.NoError(t, err, "Delete operation failed") + assert.True(t, response.Succeeded, "Delete operation should succeed") + require.Len(t, response.Results, 1, "Delete operation should return one result") + require.Len(t, response.Results[0].Values, 1, "Delete operation should return one value") + + return response.Results[0].Values[0] +} + +func TestEtcdDriverWithIntegrationCluster(t *testing.T) { + ctx := context.Background() + + driver, cleanup := createTestDriver(t) + defer cleanup() + + t.Run("Basic Operations", func(t *testing.T) { + key := testKey(t, "basic-operations") + value := []byte("test-value") + + putValue(ctx, t, driver, key, value) + + retrievedGetKv := getValue(ctx, t, driver, key) + + assert.Equal(t, key, retrievedGetKv.Key, "Returned key should match requested key") + assert.Equal(t, value, retrievedGetKv.Value, "Returned value should match stored value") + assert.Positive(t, retrievedGetKv.ModRevision, "ModRevision should be greater than 0") + }) + + t.Run("Conditional Transaction", func(t *testing.T) { + key := testKey(t, "conditional-txn") + initialValue := []byte("initial-value") + updatedValue := []byte("new-value") + + putValue(ctx, t, driver, key, initialValue) + + response, err := driver.Execute(ctx, []predicate.Predicate{}, []operation.Operation{ + operation.Put(key, updatedValue), + }, []operation.Operation{ + operation.Get(key), + }) + require.NoError(t, err, "Transaction should succeed") + assert.True(t, response.Succeeded, "Transaction should succeed") + + retrievedKv := getValue(ctx, t, driver, key) + assert.Equal(t, updatedValue, retrievedKv.Value, "Value should be updated to new value") + }) + + t.Run("Watch Functionality", func(t *testing.T) { + key := testKey(t, "watch-functionality") + value := []byte("watch-value") + + eventCh, cancel, err := driver.Watch(ctx, key) + require.NoError(t, err, "Watch setup failed") + + defer cancel() + + putValue(ctx, t, driver, key, value) + + select { + case event := <-eventCh: + assert.NotNil(t, event, "Should receive watch event") + assert.Equal(t, key, event.Prefix, "Event prefix should match key") + case <-time.After(defaultWaitTimeout): + t.Fatal("Did not receive watch event within timeout") + } + }) +} + +func TestEtcdDriver_Put(t *testing.T) { + ctx := context.Background() + + driver, cleanup := createTestDriver(t) + defer cleanup() + + key := testKey(t, "put") + value := []byte("put-test-value") + + putValue(ctx, t, driver, key, value) +} + +func TestEtcdDriver_Get(t *testing.T) { + ctx := context.Background() + + driver, cleanup := createTestDriver(t) + defer cleanup() + + key := testKey(t, "get") + value := []byte("get-test-value") + + putValue(ctx, t, driver, key, value) + + retrievedKv := getValue(ctx, t, driver, key) + + assert.Equal(t, key, retrievedKv.Key, "Returned key should match requested key") + assert.Equal(t, value, retrievedKv.Value, "Returned value should match stored value") + assert.Positive(t, retrievedKv.ModRevision, "ModRevision should be greater than 0") +} + +func TestEtcdDriver_Delete(t *testing.T) { + ctx := context.Background() + + driver, cleanup := createTestDriver(t) + defer cleanup() + + key := testKey(t, "delete") + value := []byte("delete-test-value") + + putValue(ctx, t, driver, key, value) + + deletedKv := deleteValue(ctx, t, driver, key) + + assert.Equal(t, key, deletedKv.Key, "Returned key should match deleted key") + assert.Equal(t, value, deletedKv.Value, "Returned value should match deleted value") + assert.Positive(t, deletedKv.ModRevision, "ModRevision should be greater than 0") +} + +func TestEtcdDriver_ValueEqualPredicate(t *testing.T) { + ctx := context.Background() + + driver, cleanup := createTestDriver(t) + defer cleanup() + + key := testKey(t, "value-equal") + initialValue := []byte("initial") + updatedValue := []byte("updated") + + putValue(ctx, t, driver, key, initialValue) + + response, err := driver.Execute(ctx, []predicate.Predicate{ + predicate.ValueEqual(key, initialValue), + }, []operation.Operation{ + operation.Put(key, updatedValue), + }, nil) + require.NoError(t, err, "Predicate transaction should succeed") + assert.True(t, response.Succeeded, "Should succeed when value matches") + + require.Len(t, response.Results, 1, "TX should return one result") + assert.Empty(t, response.Results[0].Values, "Put operation should not return any values in response") +} + +func TestEtcdDriver_VersionEqualPredicate(t *testing.T) { + ctx := context.Background() + + driver, cleanup := createTestDriver(t) + defer cleanup() + + key := testKey(t, "version-equal") + value := []byte("version-test") + updatedValue := []byte("updated") + + putValue(ctx, t, driver, key, value) + + kv := getValue(ctx, t, driver, key) + initialRevision := kv.ModRevision + + response, err := driver.Execute(ctx, []predicate.Predicate{ + predicate.VersionEqual(key, initialRevision), + }, []operation.Operation{ + operation.Put(key, updatedValue), + }, nil) + require.NoError(t, err, "Predicate transaction should succeed") + assert.True(t, response.Succeeded, "Should succeed when version matches") +} + +func TestEtcdDriver_ContextCancellation(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + cancel() // Cancel immediately. + + driver, cleanup := createTestDriver(t) + defer cleanup() + + key := testKey(t, "context-cancellation") + value := []byte("value") + + _, err := driver.Execute(ctx, nil, []operation.Operation{ + operation.Put(key, value), + }, nil) + require.Error(t, err, "Should return error when context is cancelled") + assert.True(t, errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded), + "Error should be context cancellation related") +} + +func TestMain(m *testing.M) { + os.Exit(m.Run()) +} diff --git a/driver/etcd/operation.go b/driver/etcd/operation.go new file mode 100644 index 0000000..27e36e2 --- /dev/null +++ b/driver/etcd/operation.go @@ -0,0 +1,46 @@ +package etcd + +import ( + "fmt" + + etcd "go.etcd.io/etcd/client/v3" + + "github.com/tarantool/go-storage/operation" +) + +// operationsToEtcdOps converts operations to etcd operations. +func operationsToEtcdOps(ops []operation.Operation) ([]etcd.Op, error) { + etcdOps := make([]etcd.Op, 0, len(ops)) + for _, op := range ops { + etcdOp, err := operationToEtcdOp(op) + if err != nil { + return nil, err + } + + etcdOps = append(etcdOps, etcdOp) + } + + return etcdOps, nil +} + +// operationToEtcdOp converts an operation to an etcd operation. +func operationToEtcdOp(storageOperation operation.Operation) (etcd.Op, error) { + key := string(storageOperation.Key()) + + var ops []etcd.OpOption + if storageOperation.IsPrefix() { + ops = append(ops, etcd.WithPrefix()) + } + + switch storageOperation.Type() { + case operation.TypeGet: + return etcd.OpGet(key, ops...), nil + case operation.TypePut: + return etcd.OpPut(key, string(storageOperation.Value()), ops...), nil + case operation.TypeDelete: + ops = append(ops, etcd.WithPrevKV()) + return etcd.OpDelete(key, ops...), nil + default: + return etcd.Op{}, fmt.Errorf("%w: %v", errUnsupportedOperationType, storageOperation.Type()) + } +} diff --git a/driver/etcd/operation_test.go b/driver/etcd/operation_test.go new file mode 100644 index 0000000..8f815ec --- /dev/null +++ b/driver/etcd/operation_test.go @@ -0,0 +1,116 @@ +package etcd //nolint:testpackage + +import ( + "testing" + + etcd "go.etcd.io/etcd/client/v3" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/tarantool/go-storage/operation" +) + +func TestOperationsToEtcdOps(t *testing.T) { + t.Parallel() + + tests := []struct { + name string + operations []operation.Operation + expectedOps int + }{ + { + name: "empty operations slice", + operations: []operation.Operation{}, + expectedOps: 0, + }, + { + name: "single get operation", + operations: []operation.Operation{ + operation.Get([]byte("test-key")), + }, + expectedOps: 1, + }, + { + name: "multiple operations", + operations: []operation.Operation{ + operation.Get([]byte("key1")), + operation.Put([]byte("key2"), []byte("value2")), + operation.Delete([]byte("key3")), + }, + expectedOps: 3, + }, + { + name: "single put operation", + operations: []operation.Operation{ + operation.Put([]byte("test-key"), []byte("test-value")), + }, + expectedOps: 1, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + + etcdOps, err := operationsToEtcdOps(tt.operations) + + require.NoError(t, err) + + if tt.expectedOps > 0 { + assert.Len(t, etcdOps, tt.expectedOps) + } else { + assert.Empty(t, etcdOps) + } + }) + } +} + +func TestOperationToEtcdOp(t *testing.T) { + t.Parallel() + + tests := []struct { + name string + operation operation.Operation + checkerFunc func(op etcd.Op) + }{ + { + name: "get operation", + operation: operation.Get([]byte("test-key")), + checkerFunc: func(op etcd.Op) { + assert.True(t, op.IsGet()) + assert.Equal(t, op.KeyBytes(), []byte("test-key")) + }, + }, + { + name: "put operation", + operation: operation.Put([]byte("test-key"), []byte("test-value")), + checkerFunc: func(op etcd.Op) { + assert.True(t, op.IsPut()) + assert.Equal(t, op.KeyBytes(), []byte("test-key")) + assert.Equal(t, op.ValueBytes(), []byte("test-value")) + }, + }, + { + name: "delete operation", + operation: operation.Delete([]byte("test-key")), + checkerFunc: func(op etcd.Op) { + assert.True(t, op.IsDelete()) + assert.Equal(t, op.KeyBytes(), []byte("test-key")) + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + + etcdOp, err := operationToEtcdOp(tt.operation) + + require.NoError(t, err) + assert.NotNil(t, etcdOp) + + tt.checkerFunc(etcdOp) + }) + } +} diff --git a/driver/etcd/predicate.go b/driver/etcd/predicate.go new file mode 100644 index 0000000..4ab599e --- /dev/null +++ b/driver/etcd/predicate.go @@ -0,0 +1,80 @@ +package etcd + +import ( + "fmt" + + etcd "go.etcd.io/etcd/client/v3" + + "github.com/tarantool/go-storage/predicate" +) + +// predicatesToCmps converts a predicate list to an etcd comparison list. +func predicatesToCmps(predicates []predicate.Predicate) ([]etcd.Cmp, error) { + convertedPredicates := make([]etcd.Cmp, 0, len(predicates)) + for _, pred := range predicates { + convertedPredicate, err := predicateToCmp(pred) + if err != nil { + return nil, err + } + + convertedPredicates = append(convertedPredicates, convertedPredicate) + } + + return convertedPredicates, nil +} + +// predicateToCmp converts a predicate to an etcd comparison. +func predicateToCmp(pred predicate.Predicate) (etcd.Cmp, error) { + switch pred.Target() { + case predicate.TargetValue: + return valuePredicateToCmp(pred) + case predicate.TargetVersion: + return versionPredicateToCmp(pred) + default: + return etcd.Cmp{}, fmt.Errorf("%w: %v", errUnsupportedPredicateTarget, pred.Target()) + } +} + +// valuePredicateToCmp converts a value predicate to an etcd comparison. +func valuePredicateToCmp(pred predicate.Predicate) (etcd.Cmp, error) { + key := string(pred.Key()) + value, ok := pred.Value().([]byte) + + if !ok { + return etcd.Cmp{}, errValuePredicateRequiresBytes + } + + switch pred.Operation() { + case predicate.OpEqual: + return etcd.Compare(etcd.Value(key), "=", string(value)), nil + case predicate.OpNotEqual: + return etcd.Compare(etcd.Value(key), "!=", string(value)), nil + case predicate.OpGreater, predicate.OpLess: + return etcd.Cmp{}, fmt.Errorf("%w: %v", errUnsupportedValueOperation, pred.Operation()) + default: + return etcd.Cmp{}, fmt.Errorf("%w: %v", errUnsupportedValueOperation, pred.Operation()) + } +} + +// versionPredicateToCmp converts a version predicate to an etcd comparison. +func versionPredicateToCmp(pred predicate.Predicate) (etcd.Cmp, error) { + key := string(pred.Key()) + version, ok := pred.Value().(int64) + + if !ok { + return etcd.Cmp{}, errVersionPredicateRequiresInt + } + + switch pred.Operation() { + case predicate.OpEqual: + return etcd.Compare(etcd.ModRevision(key), "=", version), nil + case predicate.OpNotEqual: + return etcd.Compare(etcd.ModRevision(key), "!=", version), nil + case predicate.OpGreater: + return etcd.Compare(etcd.ModRevision(key), ">", version), nil + case predicate.OpLess: + return etcd.Compare(etcd.ModRevision(key), "<", version), nil + default: + return etcd.Cmp{}, fmt.Errorf("%w: %v", errUnsupportedVersionOperation, pred.Operation()) + } +} diff --git a/driver/etcd/predicate_test.go b/driver/etcd/predicate_test.go new file mode 100644 index 0000000..b4e5ff8 --- /dev/null +++ b/driver/etcd/predicate_test.go @@ -0,0 +1,253 @@ +package etcd //nolint:testpackage + +import ( + "testing" + + "go.etcd.io/etcd/api/v3/etcdserverpb" + etcd "go.etcd.io/etcd/client/v3" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/tarantool/go-storage/predicate" +) + +func TestPredicateToCmp(t *testing.T) { + t.Parallel() + + tests := []struct { + name string + predicate predicate.Predicate + checkerFunc func(cmp etcd.Cmp) + }{ + { + name: "value equal predicate", + predicate: predicate.ValueEqual([]byte("test-key"), []byte("test-value")), + checkerFunc: func(cmp etcd.Cmp) { + assert.Equal(t, etcdserverpb.Compare_VALUE, cmp.Target) + assert.Equal(t, etcdserverpb.Compare_EQUAL, cmp.Result) + assert.Equal(t, []byte("test-key"), cmp.KeyBytes()) + assert.Equal(t, []byte("test-value"), cmp.ValueBytes()) + }, + }, + { + name: "value not equal predicate", + predicate: predicate.ValueNotEqual([]byte("test-key"), []byte("test-value")), + checkerFunc: func(cmp etcd.Cmp) { + assert.Equal(t, etcdserverpb.Compare_VALUE, cmp.Target) + assert.Equal(t, etcdserverpb.Compare_NOT_EQUAL, cmp.Result) + assert.Equal(t, []byte("test-key"), cmp.KeyBytes()) + require.IsType(t, &etcdserverpb.Compare_Value{}, cmp.TargetUnion) //nolint:exhaustruct + assert.Equal(t, []byte("test-value"), + cmp.TargetUnion.(*etcdserverpb.Compare_Value).Value) //nolint:forcetypeassert + }, + }, + { + name: "version equal predicate", + predicate: predicate.VersionEqual([]byte("test-key"), int64(123)), + checkerFunc: func(cmp etcd.Cmp) { + assert.Equal(t, etcdserverpb.Compare_MOD, cmp.Target) + assert.Equal(t, etcdserverpb.Compare_EQUAL, cmp.Result) + assert.Equal(t, []byte("test-key"), cmp.KeyBytes()) + require.IsType(t, &etcdserverpb.Compare_ModRevision{}, cmp.TargetUnion) //nolint:exhaustruct + assert.Equal(t, int64(123), + cmp.TargetUnion.(*etcdserverpb.Compare_ModRevision).ModRevision) //nolint:forcetypeassert + }, + }, + { + name: "version not equal predicate", + predicate: predicate.VersionNotEqual([]byte("test-key"), int64(123)), + checkerFunc: func(cmp etcd.Cmp) { + assert.Equal(t, etcdserverpb.Compare_MOD, cmp.Target) + assert.Equal(t, etcdserverpb.Compare_NOT_EQUAL, cmp.Result) + assert.Equal(t, []byte("test-key"), cmp.KeyBytes()) + require.IsType(t, &etcdserverpb.Compare_ModRevision{}, cmp.TargetUnion) //nolint:exhaustruct + assert.Equal(t, int64(123), + cmp.TargetUnion.(*etcdserverpb.Compare_ModRevision).ModRevision) //nolint:forcetypeassert + }, + }, + { + name: "version greater predicate", + predicate: predicate.VersionGreater([]byte("test-key"), int64(123)), + checkerFunc: func(cmp etcd.Cmp) { + assert.Equal(t, etcdserverpb.Compare_MOD, cmp.Target) + assert.Equal(t, etcdserverpb.Compare_GREATER, cmp.Result) + assert.Equal(t, []byte("test-key"), cmp.KeyBytes()) + require.IsType(t, &etcdserverpb.Compare_ModRevision{}, cmp.TargetUnion) //nolint:exhaustruct + assert.Equal(t, int64(123), + cmp.TargetUnion.(*etcdserverpb.Compare_ModRevision).ModRevision) //nolint:forcetypeassert + }, + }, + { + name: "version less predicate", + predicate: predicate.VersionLess([]byte("test-key"), int64(123)), + checkerFunc: func(cmp etcd.Cmp) { + assert.Equal(t, etcdserverpb.Compare_MOD, cmp.Target) + assert.Equal(t, etcdserverpb.Compare_LESS, cmp.Result) + assert.Equal(t, []byte("test-key"), cmp.KeyBytes()) + require.IsType(t, &etcdserverpb.Compare_ModRevision{}, cmp.TargetUnion) //nolint:exhaustruct + assert.Equal(t, int64(123), + cmp.TargetUnion.(*etcdserverpb.Compare_ModRevision).ModRevision) //nolint:forcetypeassert + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + + cmp, err := predicateToCmp(tt.predicate) + + require.NoError(t, err) + assert.NotNil(t, cmp) + + if tt.checkerFunc != nil { + tt.checkerFunc(cmp) + } + }) + } +} + +func TestValuePredicateToCmp(t *testing.T) { + t.Parallel() + + tests := []struct { + name string + predicate predicate.Predicate + checkerFunc func(cmp etcd.Cmp) + }{ + { + name: "value equal operation", + predicate: predicate.ValueEqual([]byte("test-key"), []byte("test-value")), + checkerFunc: func(cmp etcd.Cmp) { + assert.Equal(t, etcdserverpb.Compare_VALUE, cmp.Target) + assert.Equal(t, etcdserverpb.Compare_EQUAL, cmp.Result) + assert.Equal(t, []byte("test-key"), cmp.KeyBytes()) + require.IsType(t, &etcdserverpb.Compare_Value{}, cmp.TargetUnion) //nolint:exhaustruct + assert.Equal(t, []byte("test-value"), + cmp.TargetUnion.(*etcdserverpb.Compare_Value).Value) //nolint:forcetypeassert + }, + }, + { + name: "value not equal operation", + predicate: predicate.ValueNotEqual([]byte("test-key"), []byte("test-value")), + checkerFunc: func(cmp etcd.Cmp) { + assert.Equal(t, etcdserverpb.Compare_VALUE, cmp.Target) + assert.Equal(t, etcdserverpb.Compare_NOT_EQUAL, cmp.Result) + assert.Equal(t, []byte("test-key"), cmp.KeyBytes()) + require.IsType(t, &etcdserverpb.Compare_Value{}, cmp.TargetUnion) //nolint:exhaustruct + assert.Equal(t, []byte("test-value"), + cmp.TargetUnion.(*etcdserverpb.Compare_Value).Value) //nolint:forcetypeassert + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + + cmp, err := valuePredicateToCmp(tt.predicate) + + require.NoError(t, err) + assert.NotNil(t, cmp) + + if tt.checkerFunc != nil { + tt.checkerFunc(cmp) + } + }) + } +} + +func TestVersionPredicateToCmp(t *testing.T) { + t.Parallel() + + tests := []struct { + name string + predicate predicate.Predicate + checkerFunc func(cmp etcd.Cmp) + }{ + { + name: "version equal operation", + predicate: predicate.VersionEqual([]byte("test-key"), int64(123)), + checkerFunc: func(cmp etcd.Cmp) { + assert.Equal(t, etcdserverpb.Compare_MOD, cmp.Target) + assert.Equal(t, etcdserverpb.Compare_EQUAL, cmp.Result) + assert.Equal(t, []byte("test-key"), cmp.KeyBytes()) + require.IsType(t, &etcdserverpb.Compare_ModRevision{}, cmp.TargetUnion) //nolint:exhaustruct + assert.Equal(t, int64(123), + cmp.TargetUnion.(*etcdserverpb.Compare_ModRevision).ModRevision) //nolint:forcetypeassert + }, + }, + { + name: "version not equal operation", + predicate: predicate.VersionNotEqual([]byte("test-key"), int64(123)), + checkerFunc: func(cmp etcd.Cmp) { + assert.Equal(t, etcdserverpb.Compare_MOD, cmp.Target) + assert.Equal(t, etcdserverpb.Compare_NOT_EQUAL, cmp.Result) + assert.Equal(t, []byte("test-key"), cmp.KeyBytes()) + require.IsType(t, &etcdserverpb.Compare_ModRevision{}, cmp.TargetUnion) //nolint:exhaustruct + assert.Equal(t, int64(123), + cmp.TargetUnion.(*etcdserverpb.Compare_ModRevision).ModRevision) //nolint:forcetypeassert + }, + }, + { + name: "version greater operation", + predicate: predicate.VersionGreater([]byte("test-key"), int64(123)), + checkerFunc: func(cmp etcd.Cmp) { + assert.Equal(t, etcdserverpb.Compare_MOD, cmp.Target) + assert.Equal(t, etcdserverpb.Compare_GREATER, cmp.Result) + assert.Equal(t, []byte("test-key"), cmp.KeyBytes()) + require.IsType(t, &etcdserverpb.Compare_ModRevision{}, cmp.TargetUnion) //nolint:exhaustruct + assert.Equal(t, int64(123), + cmp.TargetUnion.(*etcdserverpb.Compare_ModRevision).ModRevision) //nolint:forcetypeassert + }, + }, + { + name: "version less operation", + predicate: predicate.VersionLess([]byte("test-key"), int64(123)), + checkerFunc: func(cmp etcd.Cmp) { + assert.Equal(t, etcdserverpb.Compare_MOD, cmp.Target) + assert.Equal(t, etcdserverpb.Compare_LESS, cmp.Result) + assert.Equal(t, []byte("test-key"), cmp.KeyBytes()) + require.IsType(t, &etcdserverpb.Compare_ModRevision{}, cmp.TargetUnion) //nolint:exhaustruct + assert.Equal(t, int64(123), + cmp.TargetUnion.(*etcdserverpb.Compare_ModRevision).ModRevision) //nolint:forcetypeassert + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + + cmp, err := versionPredicateToCmp(tt.predicate) + + require.NoError(t, err) + assert.NotNil(t, cmp) + + if tt.checkerFunc != nil { + tt.checkerFunc(cmp) + } + }) + } +} + +func TestPredicateToCmp_EmptyKey(t *testing.T) { + t.Parallel() + + pred := predicate.ValueEqual([]byte(""), []byte("test-value")) + + cmp, err := predicateToCmp(pred) + require.NoError(t, err) + assert.NotNil(t, cmp) +} + +func TestPredicateToCmp_EmptyValue(t *testing.T) { + t.Parallel() + + pred := predicate.ValueEqual([]byte("test-key"), []byte("")) + + cmp, err := predicateToCmp(pred) + require.NoError(t, err) + assert.NotNil(t, cmp) +} diff --git a/go.mod b/go.mod index 490d788..7fa18da 100644 --- a/go.mod +++ b/go.mod @@ -5,39 +5,98 @@ go 1.24.0 require ( github.com/gojuno/minimock/v3 v3.4.7 github.com/stretchr/testify v1.11.1 + github.com/tarantool/go-iproto v1.1.0 github.com/tarantool/go-option v1.0.0 github.com/tarantool/go-tarantool/v2 v2.4.0 github.com/vmihailenco/msgpack/v5 v5.4.1 go.etcd.io/etcd/client/v3 v3.6.5 + go.etcd.io/etcd/tests/v3 v3.6.5 ) require ( + github.com/VividCortex/ewma v1.2.0 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/bgentry/speakeasy v0.2.0 // indirect + github.com/cenkalti/backoff/v4 v4.3.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect + github.com/cheggaaa/pb/v3 v3.1.6 // indirect github.com/coreos/go-semver v0.3.1 // indirect github.com/coreos/go-systemd/v22 v22.6.0 // indirect github.com/davecgh/go-spew v1.1.1 // indirect + github.com/dustin/go-humanize v1.0.1 // indirect + github.com/fatih/color v1.18.0 // indirect + github.com/go-logr/logr v1.4.3 // indirect + github.com/go-logr/stdr v1.2.2 // indirect github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang-jwt/jwt/v5 v5.2.2 // indirect + github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect github.com/golang/protobuf v1.5.4 // indirect + github.com/google/btree v1.1.3 // indirect + github.com/google/go-cmp v0.7.0 // indirect + github.com/google/uuid v1.6.0 // indirect + github.com/gorilla/websocket v1.4.2 // indirect + github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus v1.0.1 // indirect + github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.1.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.27.2 // indirect github.com/hexdigest/gowrap v1.4.3 // indirect + github.com/inconshreveable/mousetrap v1.1.0 // indirect + github.com/jonboulle/clockwork v0.5.0 // indirect + github.com/klauspost/compress v1.17.9 // indirect + github.com/mattn/go-colorable v0.1.14 // indirect + github.com/mattn/go-isatty v0.0.20 // indirect + github.com/mattn/go-runewidth v0.0.16 // indirect + github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect + github.com/olekukonko/tablewriter v0.0.5 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect - github.com/tarantool/go-iproto v1.1.0 // indirect + github.com/prometheus/client_golang v1.20.5 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/common v0.62.0 // indirect + github.com/prometheus/procfs v0.15.1 // indirect + github.com/rivo/uniseg v0.4.7 // indirect + github.com/sirupsen/logrus v1.9.3 // indirect + github.com/soheilhy/cmux v0.1.5 // indirect + github.com/spf13/cobra v1.9.1 // indirect + github.com/spf13/pflag v1.0.6 // indirect + github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 // indirect github.com/vmihailenco/tagparser/v2 v2.0.0 // indirect + github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 // indirect + go.etcd.io/bbolt v1.4.3 // indirect go.etcd.io/etcd/api/v3 v3.6.5 // indirect go.etcd.io/etcd/client/pkg/v3 v3.6.5 // indirect + go.etcd.io/etcd/etcdctl/v3 v3.6.5 // indirect + go.etcd.io/etcd/pkg/v3 v3.6.5 // indirect + go.etcd.io/etcd/server/v3 v3.6.5 // indirect + go.etcd.io/gofail v0.2.0 // indirect + go.etcd.io/raft/v3 v3.6.0 // indirect + go.opentelemetry.io/auto/sdk v1.1.0 // indirect + go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.59.0 // indirect + go.opentelemetry.io/otel v1.37.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.34.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.34.0 // indirect + go.opentelemetry.io/otel/metric v1.37.0 // indirect + go.opentelemetry.io/otel/sdk v1.37.0 // indirect + go.opentelemetry.io/otel/trace v1.37.0 // indirect + go.opentelemetry.io/proto/otlp v1.5.0 // indirect go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.27.0 // indirect + golang.org/x/crypto v0.42.0 // indirect golang.org/x/mod v0.27.0 // indirect golang.org/x/net v0.44.0 // indirect golang.org/x/sync v0.17.0 // indirect golang.org/x/sys v0.36.0 // indirect golang.org/x/text v0.29.0 // indirect + golang.org/x/time v0.9.0 // indirect golang.org/x/tools v0.36.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20250922171735-9219d122eba9 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20250922171735-9219d122eba9 // indirect google.golang.org/grpc v1.75.1 // indirect google.golang.org/protobuf v1.36.9 // indirect + gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect + sigs.k8s.io/json v0.0.0-20211020170558-c049b76a60c6 // indirect + sigs.k8s.io/yaml v1.4.0 // indirect ) tool github.com/gojuno/minimock/v3/cmd/minimock diff --git a/go.sum b/go.sum index 13fa7a5..8626d6a 100644 --- a/go.sum +++ b/go.sum @@ -1,39 +1,143 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/VividCortex/ewma v1.2.0 h1:f58SaIzcDXrSy3kWaHNvuJgJ3Nmz59Zji6XoJR/q1ow= +github.com/VividCortex/ewma v1.2.0/go.mod h1:nz4BbCtbLyFDeC9SUHbtcT5644juEuWfUAUnGx7j5l4= +github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= +github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/bgentry/speakeasy v0.2.0 h1:tgObeVOf8WAvtuAX6DhJ4xks4CFNwPDZiqzGqIHE51E= +github.com/bgentry/speakeasy v0.2.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= +github.com/cenkalti/backoff/v4 v4.3.0 h1:MyRJ/UdXutAwSAT+s3wNd7MfTIcy71VQueUuFK343L8= +github.com/cenkalti/backoff/v4 v4.3.0/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cheggaaa/pb/v3 v3.1.6 h1:h0x+vd7EiUohAJ29DJtJy+SNAc55t/elW3jCD086EXk= +github.com/cheggaaa/pb/v3 v3.1.6/go.mod h1:urxmfVtaxT+9aWk92DbsvXFZtNSWQSO5TRAp+MJ3l1s= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/cockroachdb/datadriven v1.0.2 h1:H9MtNqVoVhvd9nCBwOyDjUEdZCREqbIdCJD93PBm/jA= +github.com/cockroachdb/datadriven v1.0.2/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU= github.com/coreos/go-semver v0.3.1 h1:yi21YpKnrx1gt5R+la8n5WgS0kCrsPp33dmEyHReZr4= github.com/coreos/go-semver v0.3.1/go.mod h1:irMmmIw/7yzSRPWryHsK7EYSg09caPQL03VsM8rvUec= github.com/coreos/go-systemd/v22 v22.6.0 h1:aGVa/v8B7hpb0TKl0MWoAavPDmHvobFe5R5zn0bCJWo= github.com/coreos/go-systemd/v22 v22.6.0/go.mod h1:iG+pp635Fo7ZmV/j14KUcmEyWF+0X7Lua8rrTWzYgWU= +github.com/cpuguy83/go-md2man/v2 v2.0.6/go.mod h1:oOW0eioCTA6cOiMLiUPZOpcVxMig6NIQQ7OS05n1F4g= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= +github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/fatih/color v1.18.0 h1:S8gINlzdQ840/4pfAwic/ZE0djQEH3wM94VfqLTZcOM= +github.com/fatih/color v1.18.0/go.mod h1:4FelSpRwEGDpQ12mAdzqdOukCy4u8WUtOY6lkT/6HfU= +github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.4.3 h1:CjnDlHq8ikf6E492q6eKboGOC0T8CDaOvkHCIg8idEI= github.com/go-logr/logr v1.4.3/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= +github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/gojuno/minimock/v3 v3.4.7 h1:vhE5zpniyPDRT0DXd5s3DbtZJVlcbmC5k80izYtj9lY= github.com/gojuno/minimock/v3 v3.4.7/go.mod h1:QxJk4mdPrVyYUmEZGc2yD2NONpqM/j4dWhsy9twjFHg= +github.com/golang-jwt/jwt/v5 v5.2.2 h1:Rl4B7itRWVtYIHFrSNd7vhTiz9UpLdi6gZhZ3wEeDy8= +github.com/golang-jwt/jwt/v5 v5.2.2/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= +github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= +github.com/google/btree v1.1.3 h1:CVpQJjYgC4VbzxeGVHfvZrv1ctoYCAI8vbl07Fcxlyg= +github.com/google/btree v1.1.3/go.mod h1:qOPhT0dTNdNzV6Z/lhRX0YXUafgPLFUh+gZMl761Gm4= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8= github.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc= +github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= +github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw= +github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= +github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus v1.0.1 h1:qnpSQwGEnkcRpTqNOIR6bJbR0gAorgP9CSALpRcKoAA= +github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus v1.0.1/go.mod h1:lXGCsh6c22WGtjr+qGHj1otzZpV/1kwTMAqkwZsnWRU= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.1.0 h1:pRhl55Yx1eC7BZ1N+BBWwnKaMyD8uC+34TLdndZMAKk= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.1.0/go.mod h1:XKMd7iuf/RGPSMJ/U4HP0zS2Z9Fh8Ps9a+6X26m/tmI= github.com/grpc-ecosystem/grpc-gateway/v2 v2.27.2 h1:8Tjv8EJ+pM1xP8mK6egEbD1OgnVTyacbefKhmbLhIhU= github.com/grpc-ecosystem/grpc-gateway/v2 v2.27.2/go.mod h1:pkJQ2tZHJ0aFOVEEot6oZmaVEZcRme73eIFmhiVuRWs= github.com/hexdigest/gowrap v1.4.3 h1:m+t8aj1pUiFQbEiE8QJg2xdYVH5DAMluLgZ9P/qEF0k= github.com/hexdigest/gowrap v1.4.3/go.mod h1:XWL8oQW2H3fX5ll8oT3Fduh4mt2H3cUAGQHQLMUbmG4= +github.com/inconshreveable/mousetrap v1.1.0 h1:wN+x4NVGpMsO7ErUn/mUI3vEoE6Jt13X2s0bqwp9tc8= +github.com/inconshreveable/mousetrap v1.1.0/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= +github.com/jonboulle/clockwork v0.5.0 h1:Hyh9A8u51kptdkR+cqRpT1EebBwTn1oK9YfGYbdFz6I= +github.com/jonboulle/clockwork v0.5.0/go.mod h1:3mZlmanh0g2NDKO5TWZVJAfofYk64M7XN3SzBPjZF60= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/klauspost/compress v1.17.9 h1:6KIumPrER1LHsvBVuDa0r5xaG0Es51mhhB9BQB2qeMA= +github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= +github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= +github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= +github.com/mattn/go-colorable v0.1.14 h1:9A9LHSqF/7dyVVX6g0U9cwm9pG3kP9gSzcuIPHPsaIE= +github.com/mattn/go-colorable v0.1.14/go.mod h1:6LmQG8QLFO4G5z1gPvYEzlUgJ2wF+stgPZH1UqBm1s8= +github.com/mattn/go-isatty v0.0.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY= +github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= +github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= +github.com/mattn/go-runewidth v0.0.16 h1:E5ScNMtiwvlvB5paMFdw9p4kSQzbXFikJ5SQO6TULQc= +github.com/mattn/go-runewidth v0.0.16/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= +github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec= +github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= +github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prometheus/client_golang v1.20.5 h1:cxppBPuYhUnsO6yo/aoRol4L7q7UFfdm+bR9r+8l63Y= +github.com/prometheus/client_golang v1.20.5/go.mod h1:PIEt8X02hGcP8JWbeHyeZ53Y/jReSnHgO035n//V5WE= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.62.0 h1:xasJaQlnWAeyHdUBeGjXmutelfJHWMRr+Fg4QszZ2Io= +github.com/prometheus/common v0.62.0/go.mod h1:vyBcEuLSvWos9B1+CyL7JZ2up+uFzXhkqml0W5zIY1I= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= +github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= +github.com/rivo/uniseg v0.4.7 h1:WUdvkW8uEhrYfLC4ZzdpI2ztxP1I582+49Oc5Mq64VQ= +github.com/rivo/uniseg v0.4.7/go.mod h1:FN3SvrM+Zdj16jyLfmOkMNblXMcoc8DfTHruCPUcx88= github.com/rogpeppe/go-internal v1.14.1 h1:UQB4HGPB6osV0SQTLymcB4TgvyWu6ZyliaW0tI/otEQ= github.com/rogpeppe/go-internal v1.14.1/go.mod h1:MaRKkUm5W0goXpeCfT7UZI6fk/L7L7so1lCWt35ZSgc= +github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= +github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= +github.com/soheilhy/cmux v0.1.5 h1:jjzc5WVemNEDTLwv9tlmemhC73tI08BNOIGwBOo10Js= +github.com/soheilhy/cmux v0.1.5/go.mod h1:T7TcVDs9LWfQgPlPsdngu6I6QIoyIFZDDC6sNE1GqG0= +github.com/spf13/cobra v1.9.1 h1:CXSaggrXdbHK9CF+8ywj8Amf7PBRmPCOJugH954Nnlo= +github.com/spf13/cobra v1.9.1/go.mod h1:nDyEzZ8ogv936Cinf6g1RU9MRY64Ir93oCnqb9wxYW0= +github.com/spf13/pflag v1.0.6 h1:jFzHGLGAlb3ruxLB8MhbI6A8+AQX/2eW4qeyNZXNp2o= +github.com/spf13/pflag v1.0.6/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.11.1 h1:7s2iGBzp5EwR7/aIZr8ao5+dra3wiQyKjjFuvgVKu7U= github.com/stretchr/testify v1.11.1/go.mod h1:wZwfW3scLgRK+23gO65QZefKpKQRnfz6sD981Nm4B6U= github.com/tarantool/go-iproto v1.1.0 h1:HULVOIHsiehI+FnHfM7wMDntuzUddO09DKqu2WnFQ5A= @@ -42,22 +146,46 @@ github.com/tarantool/go-option v1.0.0 h1:+Etw0i3TjsXvADTo5rfZNCfsXe3BfHOs+iVfIrl github.com/tarantool/go-option v1.0.0/go.mod h1:lXzzeZtL+rPUtLOCDP6ny3FemFBjruG9aHKzNN2bS08= github.com/tarantool/go-tarantool/v2 v2.4.0 h1:cfGngxdknpVVbd/vF2LvaoWsKjsLV9i3xC859XgsJlI= github.com/tarantool/go-tarantool/v2 v2.4.0/go.mod h1:MTbhdjFc3Jl63Lgi/UJr5D+QbT+QegqOzsNJGmaw7VM= +github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 h1:uruHq4dN7GR16kFc5fp3d1RIYzJW5onx8Ybykw2YQFA= +github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/vmihailenco/msgpack/v5 v5.4.1 h1:cQriyiUvjTwOHg8QZaPihLWeRAAVoCpE00IUPn0Bjt8= github.com/vmihailenco/msgpack/v5 v5.4.1/go.mod h1:GaZTsDaehaPpQVyxrf5mtQlH+pc21PIudVV/E3rRQok= github.com/vmihailenco/tagparser/v2 v2.0.0 h1:y09buUbR+b5aycVFQs/g70pqKVZNBmxwAhO7/IwNM9g= github.com/vmihailenco/tagparser/v2 v2.0.0/go.mod h1:Wri+At7QHww0WTrCBeu4J6bNtoV6mEfg5OIWRZA9qds= +github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8= +github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +go.etcd.io/bbolt v1.4.3 h1:dEadXpI6G79deX5prL3QRNP6JB8UxVkqo4UPnHaNXJo= +go.etcd.io/bbolt v1.4.3/go.mod h1:tKQlpPaYCVFctUIgFKFnAlvbmB3tpy1vkTnDWohtc0E= go.etcd.io/etcd/api/v3 v3.6.5 h1:pMMc42276sgR1j1raO/Qv3QI9Af/AuyQUW6CBAWuntA= go.etcd.io/etcd/api/v3 v3.6.5/go.mod h1:ob0/oWA/UQQlT1BmaEkWQzI0sJ1M0Et0mMpaABxguOQ= go.etcd.io/etcd/client/pkg/v3 v3.6.5 h1:Duz9fAzIZFhYWgRjp/FgNq2gO1jId9Yae/rLn3RrBP8= go.etcd.io/etcd/client/pkg/v3 v3.6.5/go.mod h1:8Wx3eGRPiy0qOFMZT/hfvdos+DjEaPxdIDiCDUv/FQk= go.etcd.io/etcd/client/v3 v3.6.5 h1:yRwZNFBx/35VKHTcLDeO7XVLbCBFbPi+XV4OC3QJf2U= go.etcd.io/etcd/client/v3 v3.6.5/go.mod h1:ZqwG/7TAFZ0BJ0jXRPoJjKQJtbFo/9NIY8uoFFKcCyo= +go.etcd.io/etcd/etcdctl/v3 v3.6.5 h1:p1uG0/xYNpOogy6cR1rqvZoQhcQseh552DVAAbICVR8= +go.etcd.io/etcd/etcdctl/v3 v3.6.5/go.mod h1:h3m9lUyEsevVvIyphoDBS3z0VL+c6zVJXwsL4vZ9g+E= +go.etcd.io/etcd/pkg/v3 v3.6.5 h1:byxWB4AqIKI4SBmquZUG1WGtvMfMaorXFoCcFbVeoxM= +go.etcd.io/etcd/pkg/v3 v3.6.5/go.mod h1:uqrXrzmMIJDEy5j00bCqhVLzR5jEJIwDp5wTlLwPGOU= +go.etcd.io/etcd/server/v3 v3.6.5 h1:4RbUb1Bd4y1WkBHmuF+cZII83JNQMuNXzyjwigQ06y0= +go.etcd.io/etcd/server/v3 v3.6.5/go.mod h1:PLuhyVXz8WWRhzXDsl3A3zv/+aK9e4A9lpQkqawIaH0= +go.etcd.io/etcd/tests/v3 v3.6.5 h1:4GcDq8+3FJfEEml52jqevuBbhZgkybbfT/JZtBFMHmA= +go.etcd.io/etcd/tests/v3 v3.6.5/go.mod h1:AMKnnWNuEYZWUNvHv8e+JlcQTDNtg7TTG2I8d15CT5I= +go.etcd.io/gofail v0.2.0 h1:p19drv16FKK345a09a1iubchlw/vmRuksmRzgBIGjcA= +go.etcd.io/gofail v0.2.0/go.mod h1:nL3ILMGfkXTekKI3clMBNazKnjUZjYLKmBHzsVAnC1o= +go.etcd.io/raft/v3 v3.6.0 h1:5NtvbDVYpnfZWcIHgGRk9DyzkBIXOi8j+DDp1IcnUWQ= +go.etcd.io/raft/v3 v3.6.0/go.mod h1:nLvLevg6+xrVtHUmVaTcTz603gQPHfh7kUAwV6YpfGo= go.opentelemetry.io/auto/sdk v1.1.0 h1:cH53jehLUN6UFLY71z+NDOiNJqDdPRaXzTel0sJySYA= go.opentelemetry.io/auto/sdk v1.1.0/go.mod h1:3wSPjt5PWp2RhlCcmmOial7AvC4DQqZb7a7wCow3W8A= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.59.0 h1:rgMkmiGfix9vFJDcDi1PK8WEQP4FLQwLDfhp5ZLpFeE= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.59.0/go.mod h1:ijPqXp5P6IRRByFVVg9DY8P5HkxkHE5ARIa+86aXPf4= go.opentelemetry.io/otel v1.37.0 h1:9zhNfelUvx0KBfu/gb+ZgeAfAgtWrfHJZcAqFC228wQ= go.opentelemetry.io/otel v1.37.0/go.mod h1:ehE/umFRLnuLa/vSccNq9oS1ErUlkkK71gMcN34UG8I= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.34.0 h1:OeNbIYk/2C15ckl7glBlOBp5+WlYsOElzTNmiPW/x60= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.34.0/go.mod h1:7Bept48yIeqxP2OZ9/AqIpYS94h2or0aB4FypJTc8ZM= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.34.0 h1:tgJ0uaNS4c98WRNUEx5U3aDlrDOI5Rs+1Vifcw4DJ8U= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.34.0/go.mod h1:U7HYyW0zt/a9x5J1Kjs+r1f/d4ZHnYFclhYY2+YbeoE= go.opentelemetry.io/otel/metric v1.37.0 h1:mvwbQS5m0tbmqML4NqK+e3aDiO02vsf/WgbsdpcPoZE= go.opentelemetry.io/otel/metric v1.37.0/go.mod h1:04wGrZurHYKOc+RKeye86GwKiTb9FKm1WHtO+4EVr2E= go.opentelemetry.io/otel/sdk v1.37.0 h1:ItB0QUqnjesGRvNcmAcU0LyvkVyGJ2xftD29bWdDvKI= @@ -66,40 +194,69 @@ go.opentelemetry.io/otel/sdk/metric v1.37.0 h1:90lI228XrB9jCMuSdA0673aubgRobVZFh go.opentelemetry.io/otel/sdk/metric v1.37.0/go.mod h1:cNen4ZWfiD37l5NhS+Keb5RXVWZWpRE+9WyVCpbo5ps= go.opentelemetry.io/otel/trace v1.37.0 h1:HLdcFNbRQBE2imdSEgm/kwqmQj1Or1l/7bW6mxVK7z4= go.opentelemetry.io/otel/trace v1.37.0/go.mod h1:TlgrlQ+PtQO5XFerSPUYG0JSgGyryXewPGyayAWSBS0= +go.opentelemetry.io/proto/otlp v1.5.0 h1:xJvq7gMzB31/d406fB8U5CBdyQGw4P399D1aQWU/3i4= +go.opentelemetry.io/proto/otlp v1.5.0/go.mod h1:keN8WnHxOy8PG0rQZjJJ5A2ebUoafqWp0eVQ4yIXvJ4= +go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= +go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= +go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.42.0 h1:chiH31gIWm57EkTXpwnqf8qeuMUi0yekh6mT2AvFlqI= +golang.org/x/crypto v0.42.0/go.mod h1:4+rDnOTJhQCx2q7/j6rAN5XDw8kPjeaXEUR2eL94ix8= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.27.0 h1:kb+q2PyFnEADO2IEF935ehFUXlWiNjJWtRNgBLSfbxQ= golang.org/x/mod v0.27.0/go.mod h1:rWI627Fq0DEoudcK+MBkNkCe0EetEaDSwJJkCcjpazc= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20201202161906-c7110b5ffcbb/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.44.0 h1:evd8IRDyfNBMBTTY5XRF1vaZlD+EmWx6x8PkhR04H/I= golang.org/x/net v0.44.0/go.mod h1:ECOoLqd5U3Lhyeyo/QDCEVQ4sNgYsqvCZ722XogGieY= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= golang.org/x/sync v0.17.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.36.0 h1:KVRy2GtZBrk1cBYA7MKu5bEZFxQk4NIDV6RLVcC8o0k= golang.org/x/sys v0.36.0/go.mod h1:OgkHotnGiDImocRcuBABYBEXf8A9a87e/uXjp9XT3ks= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.29.0 h1:1neNs90w9YzJ9BocxfsQNHKuAT4pkghyXc4nhZ6sJvk= golang.org/x/text v0.29.0/go.mod h1:7MhJOA9CD2qZyOKYazxdYMF85OwPdEr9jTtBpO7ydH4= +golang.org/x/time v0.9.0 h1:EsRrnYcQiGH+5FfbgvV4AP7qEZstoyrHB0DzarOQ4ZY= +golang.org/x/time v0.9.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= @@ -111,10 +268,20 @@ golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= gonum.org/v1/gonum v0.16.0 h1:5+ul4Swaf3ESvrOnidPp4GZbzf0mxVQpDCYUQE7OJfk= gonum.org/v1/gonum v0.16.0/go.mod h1:fef3am4MQ93R2HHpKnLk4/Tbh/s0+wqD5nfa6Pnwy4E= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto/googleapis/api v0.0.0-20250922171735-9219d122eba9 h1:jm6v6kMRpTYKxBRrDkYAitNJegUeO1Mf3Kt80obv0gg= google.golang.org/genproto/googleapis/api v0.0.0-20250922171735-9219d122eba9/go.mod h1:LmwNphe5Afor5V3R5BppOULHOnt2mCIf+NxMd4XiygE= google.golang.org/genproto/googleapis/rpc v0.0.0-20250922171735-9219d122eba9 h1:V1jCN2HBa8sySkR5vLcCSqJSTMv093Rw9EJefhQGP7M= google.golang.org/genproto/googleapis/rpc v0.0.0-20250922171735-9219d122eba9/go.mod h1:HSkG/KdJWusxU1F6CNrwNDjBMgisKxGnc5dAZfT0mjQ= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.75.1 h1:/ODCNEuf9VghjgO3rqLcfg8fiOP0nSluljWFlDxELLI= google.golang.org/grpc v1.75.1/go.mod h1:JtPAzKiq4v1xcAB2hydNlWI2RnF85XXcV0mhKXr2ecQ= google.golang.org/protobuf v1.36.9 h1:w2gp2mA27hUeUzj9Ex9FBjsBm40zfaDtEWow293U7Iw= @@ -122,5 +289,15 @@ google.golang.org/protobuf v1.36.9/go.mod h1:fuxRtAxBytpl4zzqUh6/eyUujkJdNiuEkXn gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= +gopkg.in/natefinch/lumberjack.v2 v2.2.1 h1:bBRl1b0OH9s/DuPhuXpNl+VtCaJXFZ5/uEFST95x9zc= +gopkg.in/natefinch/lumberjack.v2 v2.2.1/go.mod h1:YD8tP3GAjkrDg1eZH7EGmyESg/lsYskCTPBJVb9jqSc= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +sigs.k8s.io/json v0.0.0-20211020170558-c049b76a60c6 h1:fD1pz4yfdADVNfFmcP2aBEtudwUQ1AlLnRBALr33v3s= +sigs.k8s.io/json v0.0.0-20211020170558-c049b76a60c6/go.mod h1:p4QtZmO4uMYipTQNzagwnNoseA6OxSUutVw05NhYDRs= +sigs.k8s.io/yaml v1.4.0 h1:Mk1wCc2gy/F0THH0TAp1QYyJNzRm2KCLy3o5ASXVI5E= +sigs.k8s.io/yaml v1.4.0/go.mod h1:Ejl7/uTz7PSA4eKMyQCUTnhZYNmLIl+5c2lQPGR2BPY= diff --git a/internal/mocks/etcd_client.go b/internal/mocks/etcd_client.go new file mode 100644 index 0000000..4a4c6ec --- /dev/null +++ b/internal/mocks/etcd_client.go @@ -0,0 +1,3 @@ +package mocks + +//go:generate go tool minimock -g -i github.com/tarantool/go-storage/driver/etcd.Client -pr etcd_ -s _mock.go -n EtcdClientMock diff --git a/internal/mocks/etcd_client_mock.go b/internal/mocks/etcd_client_mock.go new file mode 100644 index 0000000..0bdde47 --- /dev/null +++ b/internal/mocks/etcd_client_mock.go @@ -0,0 +1,384 @@ +// Code generated by http://github.com/gojuno/minimock (v3.4.7). DO NOT EDIT. + +package mocks + +import ( + "context" + "sync" + mm_atomic "sync/atomic" + mm_time "time" + + "github.com/gojuno/minimock/v3" + etcd "go.etcd.io/etcd/client/v3" +) + +// EtcdClientMock implements mm_etcd.Client +type EtcdClientMock struct { + t minimock.Tester + finishOnce sync.Once + + funcTxn func(ctx context.Context) (t1 etcd.Txn) + funcTxnOrigin string + inspectFuncTxn func(ctx context.Context) + afterTxnCounter uint64 + beforeTxnCounter uint64 + TxnMock mEtcdClientMockTxn +} + +// NewEtcdClientMock returns a mock for mm_etcd.Client +func NewEtcdClientMock(t minimock.Tester) *EtcdClientMock { + m := &EtcdClientMock{t: t} + + if controller, ok := t.(minimock.MockController); ok { + controller.RegisterMocker(m) + } + + m.TxnMock = mEtcdClientMockTxn{mock: m} + m.TxnMock.callArgs = []*EtcdClientMockTxnParams{} + + t.Cleanup(m.MinimockFinish) + + return m +} + +type mEtcdClientMockTxn struct { + optional bool + mock *EtcdClientMock + defaultExpectation *EtcdClientMockTxnExpectation + expectations []*EtcdClientMockTxnExpectation + + callArgs []*EtcdClientMockTxnParams + mutex sync.RWMutex + + expectedInvocations uint64 + expectedInvocationsOrigin string +} + +// EtcdClientMockTxnExpectation specifies expectation struct of the Client.Txn +type EtcdClientMockTxnExpectation struct { + mock *EtcdClientMock + params *EtcdClientMockTxnParams + paramPtrs *EtcdClientMockTxnParamPtrs + expectationOrigins EtcdClientMockTxnExpectationOrigins + results *EtcdClientMockTxnResults + returnOrigin string + Counter uint64 +} + +// EtcdClientMockTxnParams contains parameters of the Client.Txn +type EtcdClientMockTxnParams struct { + ctx context.Context +} + +// EtcdClientMockTxnParamPtrs contains pointers to parameters of the Client.Txn +type EtcdClientMockTxnParamPtrs struct { + ctx *context.Context +} + +// EtcdClientMockTxnResults contains results of the Client.Txn +type EtcdClientMockTxnResults struct { + t1 etcd.Txn +} + +// EtcdClientMockTxnOrigins contains origins of expectations of the Client.Txn +type EtcdClientMockTxnExpectationOrigins struct { + origin string + originCtx string +} + +// Marks this method to be optional. The default behavior of any method with Return() is '1 or more', meaning +// the test will fail minimock's automatic final call check if the mocked method was not called at least once. +// Optional() makes method check to work in '0 or more' mode. +// It is NOT RECOMMENDED to use this option unless you really need it, as default behaviour helps to +// catch the problems when the expected method call is totally skipped during test run. +func (mmTxn *mEtcdClientMockTxn) Optional() *mEtcdClientMockTxn { + mmTxn.optional = true + return mmTxn +} + +// Expect sets up expected params for Client.Txn +func (mmTxn *mEtcdClientMockTxn) Expect(ctx context.Context) *mEtcdClientMockTxn { + if mmTxn.mock.funcTxn != nil { + mmTxn.mock.t.Fatalf("EtcdClientMock.Txn mock is already set by Set") + } + + if mmTxn.defaultExpectation == nil { + mmTxn.defaultExpectation = &EtcdClientMockTxnExpectation{} + } + + if mmTxn.defaultExpectation.paramPtrs != nil { + mmTxn.mock.t.Fatalf("EtcdClientMock.Txn mock is already set by ExpectParams functions") + } + + mmTxn.defaultExpectation.params = &EtcdClientMockTxnParams{ctx} + mmTxn.defaultExpectation.expectationOrigins.origin = minimock.CallerInfo(1) + for _, e := range mmTxn.expectations { + if minimock.Equal(e.params, mmTxn.defaultExpectation.params) { + mmTxn.mock.t.Fatalf("Expectation set by When has same params: %#v", *mmTxn.defaultExpectation.params) + } + } + + return mmTxn +} + +// ExpectCtxParam1 sets up expected param ctx for Client.Txn +func (mmTxn *mEtcdClientMockTxn) ExpectCtxParam1(ctx context.Context) *mEtcdClientMockTxn { + if mmTxn.mock.funcTxn != nil { + mmTxn.mock.t.Fatalf("EtcdClientMock.Txn mock is already set by Set") + } + + if mmTxn.defaultExpectation == nil { + mmTxn.defaultExpectation = &EtcdClientMockTxnExpectation{} + } + + if mmTxn.defaultExpectation.params != nil { + mmTxn.mock.t.Fatalf("EtcdClientMock.Txn mock is already set by Expect") + } + + if mmTxn.defaultExpectation.paramPtrs == nil { + mmTxn.defaultExpectation.paramPtrs = &EtcdClientMockTxnParamPtrs{} + } + mmTxn.defaultExpectation.paramPtrs.ctx = &ctx + mmTxn.defaultExpectation.expectationOrigins.originCtx = minimock.CallerInfo(1) + + return mmTxn +} + +// Inspect accepts an inspector function that has same arguments as the Client.Txn +func (mmTxn *mEtcdClientMockTxn) Inspect(f func(ctx context.Context)) *mEtcdClientMockTxn { + if mmTxn.mock.inspectFuncTxn != nil { + mmTxn.mock.t.Fatalf("Inspect function is already set for EtcdClientMock.Txn") + } + + mmTxn.mock.inspectFuncTxn = f + + return mmTxn +} + +// Return sets up results that will be returned by Client.Txn +func (mmTxn *mEtcdClientMockTxn) Return(t1 etcd.Txn) *EtcdClientMock { + if mmTxn.mock.funcTxn != nil { + mmTxn.mock.t.Fatalf("EtcdClientMock.Txn mock is already set by Set") + } + + if mmTxn.defaultExpectation == nil { + mmTxn.defaultExpectation = &EtcdClientMockTxnExpectation{mock: mmTxn.mock} + } + mmTxn.defaultExpectation.results = &EtcdClientMockTxnResults{t1} + mmTxn.defaultExpectation.returnOrigin = minimock.CallerInfo(1) + return mmTxn.mock +} + +// Set uses given function f to mock the Client.Txn method +func (mmTxn *mEtcdClientMockTxn) Set(f func(ctx context.Context) (t1 etcd.Txn)) *EtcdClientMock { + if mmTxn.defaultExpectation != nil { + mmTxn.mock.t.Fatalf("Default expectation is already set for the Client.Txn method") + } + + if len(mmTxn.expectations) > 0 { + mmTxn.mock.t.Fatalf("Some expectations are already set for the Client.Txn method") + } + + mmTxn.mock.funcTxn = f + mmTxn.mock.funcTxnOrigin = minimock.CallerInfo(1) + return mmTxn.mock +} + +// When sets expectation for the Client.Txn which will trigger the result defined by the following +// Then helper +func (mmTxn *mEtcdClientMockTxn) When(ctx context.Context) *EtcdClientMockTxnExpectation { + if mmTxn.mock.funcTxn != nil { + mmTxn.mock.t.Fatalf("EtcdClientMock.Txn mock is already set by Set") + } + + expectation := &EtcdClientMockTxnExpectation{ + mock: mmTxn.mock, + params: &EtcdClientMockTxnParams{ctx}, + expectationOrigins: EtcdClientMockTxnExpectationOrigins{origin: minimock.CallerInfo(1)}, + } + mmTxn.expectations = append(mmTxn.expectations, expectation) + return expectation +} + +// Then sets up Client.Txn return parameters for the expectation previously defined by the When method +func (e *EtcdClientMockTxnExpectation) Then(t1 etcd.Txn) *EtcdClientMock { + e.results = &EtcdClientMockTxnResults{t1} + return e.mock +} + +// Times sets number of times Client.Txn should be invoked +func (mmTxn *mEtcdClientMockTxn) Times(n uint64) *mEtcdClientMockTxn { + if n == 0 { + mmTxn.mock.t.Fatalf("Times of EtcdClientMock.Txn mock can not be zero") + } + mm_atomic.StoreUint64(&mmTxn.expectedInvocations, n) + mmTxn.expectedInvocationsOrigin = minimock.CallerInfo(1) + return mmTxn +} + +func (mmTxn *mEtcdClientMockTxn) invocationsDone() bool { + if len(mmTxn.expectations) == 0 && mmTxn.defaultExpectation == nil && mmTxn.mock.funcTxn == nil { + return true + } + + totalInvocations := mm_atomic.LoadUint64(&mmTxn.mock.afterTxnCounter) + expectedInvocations := mm_atomic.LoadUint64(&mmTxn.expectedInvocations) + + return totalInvocations > 0 && (expectedInvocations == 0 || expectedInvocations == totalInvocations) +} + +// Txn implements mm_etcd.Client +func (mmTxn *EtcdClientMock) Txn(ctx context.Context) (t1 etcd.Txn) { + mm_atomic.AddUint64(&mmTxn.beforeTxnCounter, 1) + defer mm_atomic.AddUint64(&mmTxn.afterTxnCounter, 1) + + mmTxn.t.Helper() + + if mmTxn.inspectFuncTxn != nil { + mmTxn.inspectFuncTxn(ctx) + } + + mm_params := EtcdClientMockTxnParams{ctx} + + // Record call args + mmTxn.TxnMock.mutex.Lock() + mmTxn.TxnMock.callArgs = append(mmTxn.TxnMock.callArgs, &mm_params) + mmTxn.TxnMock.mutex.Unlock() + + for _, e := range mmTxn.TxnMock.expectations { + if minimock.Equal(*e.params, mm_params) { + mm_atomic.AddUint64(&e.Counter, 1) + return e.results.t1 + } + } + + if mmTxn.TxnMock.defaultExpectation != nil { + mm_atomic.AddUint64(&mmTxn.TxnMock.defaultExpectation.Counter, 1) + mm_want := mmTxn.TxnMock.defaultExpectation.params + mm_want_ptrs := mmTxn.TxnMock.defaultExpectation.paramPtrs + + mm_got := EtcdClientMockTxnParams{ctx} + + if mm_want_ptrs != nil { + + if mm_want_ptrs.ctx != nil && !minimock.Equal(*mm_want_ptrs.ctx, mm_got.ctx) { + mmTxn.t.Errorf("EtcdClientMock.Txn got unexpected parameter ctx, expected at\n%s:\nwant: %#v\n got: %#v%s\n", + mmTxn.TxnMock.defaultExpectation.expectationOrigins.originCtx, *mm_want_ptrs.ctx, mm_got.ctx, minimock.Diff(*mm_want_ptrs.ctx, mm_got.ctx)) + } + + } else if mm_want != nil && !minimock.Equal(*mm_want, mm_got) { + mmTxn.t.Errorf("EtcdClientMock.Txn got unexpected parameters, expected at\n%s:\nwant: %#v\n got: %#v%s\n", + mmTxn.TxnMock.defaultExpectation.expectationOrigins.origin, *mm_want, mm_got, minimock.Diff(*mm_want, mm_got)) + } + + mm_results := mmTxn.TxnMock.defaultExpectation.results + if mm_results == nil { + mmTxn.t.Fatal("No results are set for the EtcdClientMock.Txn") + } + return (*mm_results).t1 + } + if mmTxn.funcTxn != nil { + return mmTxn.funcTxn(ctx) + } + mmTxn.t.Fatalf("Unexpected call to EtcdClientMock.Txn. %v", ctx) + return +} + +// TxnAfterCounter returns a count of finished EtcdClientMock.Txn invocations +func (mmTxn *EtcdClientMock) TxnAfterCounter() uint64 { + return mm_atomic.LoadUint64(&mmTxn.afterTxnCounter) +} + +// TxnBeforeCounter returns a count of EtcdClientMock.Txn invocations +func (mmTxn *EtcdClientMock) TxnBeforeCounter() uint64 { + return mm_atomic.LoadUint64(&mmTxn.beforeTxnCounter) +} + +// Calls returns a list of arguments used in each call to EtcdClientMock.Txn. +// The list is in the same order as the calls were made (i.e. recent calls have a higher index) +func (mmTxn *mEtcdClientMockTxn) Calls() []*EtcdClientMockTxnParams { + mmTxn.mutex.RLock() + + argCopy := make([]*EtcdClientMockTxnParams, len(mmTxn.callArgs)) + copy(argCopy, mmTxn.callArgs) + + mmTxn.mutex.RUnlock() + + return argCopy +} + +// MinimockTxnDone returns true if the count of the Txn invocations corresponds +// the number of defined expectations +func (m *EtcdClientMock) MinimockTxnDone() bool { + if m.TxnMock.optional { + // Optional methods provide '0 or more' call count restriction. + return true + } + + for _, e := range m.TxnMock.expectations { + if mm_atomic.LoadUint64(&e.Counter) < 1 { + return false + } + } + + return m.TxnMock.invocationsDone() +} + +// MinimockTxnInspect logs each unmet expectation +func (m *EtcdClientMock) MinimockTxnInspect() { + for _, e := range m.TxnMock.expectations { + if mm_atomic.LoadUint64(&e.Counter) < 1 { + m.t.Errorf("Expected call to EtcdClientMock.Txn at\n%s with params: %#v", e.expectationOrigins.origin, *e.params) + } + } + + afterTxnCounter := mm_atomic.LoadUint64(&m.afterTxnCounter) + // if default expectation was set then invocations count should be greater than zero + if m.TxnMock.defaultExpectation != nil && afterTxnCounter < 1 { + if m.TxnMock.defaultExpectation.params == nil { + m.t.Errorf("Expected call to EtcdClientMock.Txn at\n%s", m.TxnMock.defaultExpectation.returnOrigin) + } else { + m.t.Errorf("Expected call to EtcdClientMock.Txn at\n%s with params: %#v", m.TxnMock.defaultExpectation.expectationOrigins.origin, *m.TxnMock.defaultExpectation.params) + } + } + // if func was set then invocations count should be greater than zero + if m.funcTxn != nil && afterTxnCounter < 1 { + m.t.Errorf("Expected call to EtcdClientMock.Txn at\n%s", m.funcTxnOrigin) + } + + if !m.TxnMock.invocationsDone() && afterTxnCounter > 0 { + m.t.Errorf("Expected %d calls to EtcdClientMock.Txn at\n%s but found %d calls", + mm_atomic.LoadUint64(&m.TxnMock.expectedInvocations), m.TxnMock.expectedInvocationsOrigin, afterTxnCounter) + } +} + +// MinimockFinish checks that all mocked methods have been called the expected number of times +func (m *EtcdClientMock) MinimockFinish() { + m.finishOnce.Do(func() { + if !m.minimockDone() { + m.MinimockTxnInspect() + } + }) +} + +// MinimockWait waits for all mocked methods to be called the expected number of times +func (m *EtcdClientMock) MinimockWait(timeout mm_time.Duration) { + timeoutCh := mm_time.After(timeout) + for { + if m.minimockDone() { + return + } + select { + case <-timeoutCh: + m.MinimockFinish() + return + case <-mm_time.After(10 * mm_time.Millisecond): + } + } +} + +func (m *EtcdClientMock) minimockDone() bool { + done := true + return done && + m.MinimockTxnDone() +} diff --git a/internal/mocks/etcd_txn.go b/internal/mocks/etcd_txn.go new file mode 100644 index 0000000..23b4757 --- /dev/null +++ b/internal/mocks/etcd_txn.go @@ -0,0 +1,3 @@ +package mocks + +//go:generate go tool minimock -g -i go.etcd.io/etcd/client/v3.Txn -pr etcd_ -s _mock.go -n ETCDTxnMock diff --git a/internal/mocks/etcd_txn_mock.go b/internal/mocks/etcd_txn_mock.go new file mode 100644 index 0000000..b1f7aa7 --- /dev/null +++ b/internal/mocks/etcd_txn_mock.go @@ -0,0 +1,1230 @@ +// Code generated by http://github.com/gojuno/minimock (v3.4.7). DO NOT EDIT. + +package mocks + +import ( + "sync" + mm_atomic "sync/atomic" + mm_time "time" + + "github.com/gojuno/minimock/v3" + mm_clientv3 "go.etcd.io/etcd/client/v3" +) + +// ETCDTxnMock implements mm_clientv3.Txn +type ETCDTxnMock struct { + t minimock.Tester + finishOnce sync.Once + + funcCommit func() (tp1 *mm_clientv3.TxnResponse, err error) + funcCommitOrigin string + inspectFuncCommit func() + afterCommitCounter uint64 + beforeCommitCounter uint64 + CommitMock mETCDTxnMockCommit + + funcElse func(ops ...mm_clientv3.Op) (t1 mm_clientv3.Txn) + funcElseOrigin string + inspectFuncElse func(ops ...mm_clientv3.Op) + afterElseCounter uint64 + beforeElseCounter uint64 + ElseMock mETCDTxnMockElse + + funcIf func(cs ...mm_clientv3.Cmp) (t1 mm_clientv3.Txn) + funcIfOrigin string + inspectFuncIf func(cs ...mm_clientv3.Cmp) + afterIfCounter uint64 + beforeIfCounter uint64 + IfMock mETCDTxnMockIf + + funcThen func(ops ...mm_clientv3.Op) (t1 mm_clientv3.Txn) + funcThenOrigin string + inspectFuncThen func(ops ...mm_clientv3.Op) + afterThenCounter uint64 + beforeThenCounter uint64 + ThenMock mETCDTxnMockThen +} + +// NewETCDTxnMock returns a mock for mm_clientv3.Txn +func NewETCDTxnMock(t minimock.Tester) *ETCDTxnMock { + m := &ETCDTxnMock{t: t} + + if controller, ok := t.(minimock.MockController); ok { + controller.RegisterMocker(m) + } + + m.CommitMock = mETCDTxnMockCommit{mock: m} + + m.ElseMock = mETCDTxnMockElse{mock: m} + m.ElseMock.callArgs = []*ETCDTxnMockElseParams{} + + m.IfMock = mETCDTxnMockIf{mock: m} + m.IfMock.callArgs = []*ETCDTxnMockIfParams{} + + m.ThenMock = mETCDTxnMockThen{mock: m} + m.ThenMock.callArgs = []*ETCDTxnMockThenParams{} + + t.Cleanup(m.MinimockFinish) + + return m +} + +type mETCDTxnMockCommit struct { + optional bool + mock *ETCDTxnMock + defaultExpectation *ETCDTxnMockCommitExpectation + expectations []*ETCDTxnMockCommitExpectation + + expectedInvocations uint64 + expectedInvocationsOrigin string +} + +// ETCDTxnMockCommitExpectation specifies expectation struct of the Txn.Commit +type ETCDTxnMockCommitExpectation struct { + mock *ETCDTxnMock + + results *ETCDTxnMockCommitResults + returnOrigin string + Counter uint64 +} + +// ETCDTxnMockCommitResults contains results of the Txn.Commit +type ETCDTxnMockCommitResults struct { + tp1 *mm_clientv3.TxnResponse + err error +} + +// Marks this method to be optional. The default behavior of any method with Return() is '1 or more', meaning +// the test will fail minimock's automatic final call check if the mocked method was not called at least once. +// Optional() makes method check to work in '0 or more' mode. +// It is NOT RECOMMENDED to use this option unless you really need it, as default behaviour helps to +// catch the problems when the expected method call is totally skipped during test run. +func (mmCommit *mETCDTxnMockCommit) Optional() *mETCDTxnMockCommit { + mmCommit.optional = true + return mmCommit +} + +// Expect sets up expected params for Txn.Commit +func (mmCommit *mETCDTxnMockCommit) Expect() *mETCDTxnMockCommit { + if mmCommit.mock.funcCommit != nil { + mmCommit.mock.t.Fatalf("ETCDTxnMock.Commit mock is already set by Set") + } + + if mmCommit.defaultExpectation == nil { + mmCommit.defaultExpectation = &ETCDTxnMockCommitExpectation{} + } + + return mmCommit +} + +// Inspect accepts an inspector function that has same arguments as the Txn.Commit +func (mmCommit *mETCDTxnMockCommit) Inspect(f func()) *mETCDTxnMockCommit { + if mmCommit.mock.inspectFuncCommit != nil { + mmCommit.mock.t.Fatalf("Inspect function is already set for ETCDTxnMock.Commit") + } + + mmCommit.mock.inspectFuncCommit = f + + return mmCommit +} + +// Return sets up results that will be returned by Txn.Commit +func (mmCommit *mETCDTxnMockCommit) Return(tp1 *mm_clientv3.TxnResponse, err error) *ETCDTxnMock { + if mmCommit.mock.funcCommit != nil { + mmCommit.mock.t.Fatalf("ETCDTxnMock.Commit mock is already set by Set") + } + + if mmCommit.defaultExpectation == nil { + mmCommit.defaultExpectation = &ETCDTxnMockCommitExpectation{mock: mmCommit.mock} + } + mmCommit.defaultExpectation.results = &ETCDTxnMockCommitResults{tp1, err} + mmCommit.defaultExpectation.returnOrigin = minimock.CallerInfo(1) + return mmCommit.mock +} + +// Set uses given function f to mock the Txn.Commit method +func (mmCommit *mETCDTxnMockCommit) Set(f func() (tp1 *mm_clientv3.TxnResponse, err error)) *ETCDTxnMock { + if mmCommit.defaultExpectation != nil { + mmCommit.mock.t.Fatalf("Default expectation is already set for the Txn.Commit method") + } + + if len(mmCommit.expectations) > 0 { + mmCommit.mock.t.Fatalf("Some expectations are already set for the Txn.Commit method") + } + + mmCommit.mock.funcCommit = f + mmCommit.mock.funcCommitOrigin = minimock.CallerInfo(1) + return mmCommit.mock +} + +// Times sets number of times Txn.Commit should be invoked +func (mmCommit *mETCDTxnMockCommit) Times(n uint64) *mETCDTxnMockCommit { + if n == 0 { + mmCommit.mock.t.Fatalf("Times of ETCDTxnMock.Commit mock can not be zero") + } + mm_atomic.StoreUint64(&mmCommit.expectedInvocations, n) + mmCommit.expectedInvocationsOrigin = minimock.CallerInfo(1) + return mmCommit +} + +func (mmCommit *mETCDTxnMockCommit) invocationsDone() bool { + if len(mmCommit.expectations) == 0 && mmCommit.defaultExpectation == nil && mmCommit.mock.funcCommit == nil { + return true + } + + totalInvocations := mm_atomic.LoadUint64(&mmCommit.mock.afterCommitCounter) + expectedInvocations := mm_atomic.LoadUint64(&mmCommit.expectedInvocations) + + return totalInvocations > 0 && (expectedInvocations == 0 || expectedInvocations == totalInvocations) +} + +// Commit implements mm_clientv3.Txn +func (mmCommit *ETCDTxnMock) Commit() (tp1 *mm_clientv3.TxnResponse, err error) { + mm_atomic.AddUint64(&mmCommit.beforeCommitCounter, 1) + defer mm_atomic.AddUint64(&mmCommit.afterCommitCounter, 1) + + mmCommit.t.Helper() + + if mmCommit.inspectFuncCommit != nil { + mmCommit.inspectFuncCommit() + } + + if mmCommit.CommitMock.defaultExpectation != nil { + mm_atomic.AddUint64(&mmCommit.CommitMock.defaultExpectation.Counter, 1) + + mm_results := mmCommit.CommitMock.defaultExpectation.results + if mm_results == nil { + mmCommit.t.Fatal("No results are set for the ETCDTxnMock.Commit") + } + return (*mm_results).tp1, (*mm_results).err + } + if mmCommit.funcCommit != nil { + return mmCommit.funcCommit() + } + mmCommit.t.Fatalf("Unexpected call to ETCDTxnMock.Commit.") + return +} + +// CommitAfterCounter returns a count of finished ETCDTxnMock.Commit invocations +func (mmCommit *ETCDTxnMock) CommitAfterCounter() uint64 { + return mm_atomic.LoadUint64(&mmCommit.afterCommitCounter) +} + +// CommitBeforeCounter returns a count of ETCDTxnMock.Commit invocations +func (mmCommit *ETCDTxnMock) CommitBeforeCounter() uint64 { + return mm_atomic.LoadUint64(&mmCommit.beforeCommitCounter) +} + +// MinimockCommitDone returns true if the count of the Commit invocations corresponds +// the number of defined expectations +func (m *ETCDTxnMock) MinimockCommitDone() bool { + if m.CommitMock.optional { + // Optional methods provide '0 or more' call count restriction. + return true + } + + for _, e := range m.CommitMock.expectations { + if mm_atomic.LoadUint64(&e.Counter) < 1 { + return false + } + } + + return m.CommitMock.invocationsDone() +} + +// MinimockCommitInspect logs each unmet expectation +func (m *ETCDTxnMock) MinimockCommitInspect() { + for _, e := range m.CommitMock.expectations { + if mm_atomic.LoadUint64(&e.Counter) < 1 { + m.t.Error("Expected call to ETCDTxnMock.Commit") + } + } + + afterCommitCounter := mm_atomic.LoadUint64(&m.afterCommitCounter) + // if default expectation was set then invocations count should be greater than zero + if m.CommitMock.defaultExpectation != nil && afterCommitCounter < 1 { + m.t.Errorf("Expected call to ETCDTxnMock.Commit at\n%s", m.CommitMock.defaultExpectation.returnOrigin) + } + // if func was set then invocations count should be greater than zero + if m.funcCommit != nil && afterCommitCounter < 1 { + m.t.Errorf("Expected call to ETCDTxnMock.Commit at\n%s", m.funcCommitOrigin) + } + + if !m.CommitMock.invocationsDone() && afterCommitCounter > 0 { + m.t.Errorf("Expected %d calls to ETCDTxnMock.Commit at\n%s but found %d calls", + mm_atomic.LoadUint64(&m.CommitMock.expectedInvocations), m.CommitMock.expectedInvocationsOrigin, afterCommitCounter) + } +} + +type mETCDTxnMockElse struct { + optional bool + mock *ETCDTxnMock + defaultExpectation *ETCDTxnMockElseExpectation + expectations []*ETCDTxnMockElseExpectation + + callArgs []*ETCDTxnMockElseParams + mutex sync.RWMutex + + expectedInvocations uint64 + expectedInvocationsOrigin string +} + +// ETCDTxnMockElseExpectation specifies expectation struct of the Txn.Else +type ETCDTxnMockElseExpectation struct { + mock *ETCDTxnMock + params *ETCDTxnMockElseParams + paramPtrs *ETCDTxnMockElseParamPtrs + expectationOrigins ETCDTxnMockElseExpectationOrigins + results *ETCDTxnMockElseResults + returnOrigin string + Counter uint64 +} + +// ETCDTxnMockElseParams contains parameters of the Txn.Else +type ETCDTxnMockElseParams struct { + ops []mm_clientv3.Op +} + +// ETCDTxnMockElseParamPtrs contains pointers to parameters of the Txn.Else +type ETCDTxnMockElseParamPtrs struct { + ops *[]mm_clientv3.Op +} + +// ETCDTxnMockElseResults contains results of the Txn.Else +type ETCDTxnMockElseResults struct { + t1 mm_clientv3.Txn +} + +// ETCDTxnMockElseOrigins contains origins of expectations of the Txn.Else +type ETCDTxnMockElseExpectationOrigins struct { + origin string + originOps string +} + +// Marks this method to be optional. The default behavior of any method with Return() is '1 or more', meaning +// the test will fail minimock's automatic final call check if the mocked method was not called at least once. +// Optional() makes method check to work in '0 or more' mode. +// It is NOT RECOMMENDED to use this option unless you really need it, as default behaviour helps to +// catch the problems when the expected method call is totally skipped during test run. +func (mmElse *mETCDTxnMockElse) Optional() *mETCDTxnMockElse { + mmElse.optional = true + return mmElse +} + +// Expect sets up expected params for Txn.Else +func (mmElse *mETCDTxnMockElse) Expect(ops ...mm_clientv3.Op) *mETCDTxnMockElse { + if mmElse.mock.funcElse != nil { + mmElse.mock.t.Fatalf("ETCDTxnMock.Else mock is already set by Set") + } + + if mmElse.defaultExpectation == nil { + mmElse.defaultExpectation = &ETCDTxnMockElseExpectation{} + } + + if mmElse.defaultExpectation.paramPtrs != nil { + mmElse.mock.t.Fatalf("ETCDTxnMock.Else mock is already set by ExpectParams functions") + } + + mmElse.defaultExpectation.params = &ETCDTxnMockElseParams{ops} + mmElse.defaultExpectation.expectationOrigins.origin = minimock.CallerInfo(1) + for _, e := range mmElse.expectations { + if minimock.Equal(e.params, mmElse.defaultExpectation.params) { + mmElse.mock.t.Fatalf("Expectation set by When has same params: %#v", *mmElse.defaultExpectation.params) + } + } + + return mmElse +} + +// ExpectOpsParam1 sets up expected param ops for Txn.Else +func (mmElse *mETCDTxnMockElse) ExpectOpsParam1(ops ...mm_clientv3.Op) *mETCDTxnMockElse { + if mmElse.mock.funcElse != nil { + mmElse.mock.t.Fatalf("ETCDTxnMock.Else mock is already set by Set") + } + + if mmElse.defaultExpectation == nil { + mmElse.defaultExpectation = &ETCDTxnMockElseExpectation{} + } + + if mmElse.defaultExpectation.params != nil { + mmElse.mock.t.Fatalf("ETCDTxnMock.Else mock is already set by Expect") + } + + if mmElse.defaultExpectation.paramPtrs == nil { + mmElse.defaultExpectation.paramPtrs = &ETCDTxnMockElseParamPtrs{} + } + mmElse.defaultExpectation.paramPtrs.ops = &ops + mmElse.defaultExpectation.expectationOrigins.originOps = minimock.CallerInfo(1) + + return mmElse +} + +// Inspect accepts an inspector function that has same arguments as the Txn.Else +func (mmElse *mETCDTxnMockElse) Inspect(f func(ops ...mm_clientv3.Op)) *mETCDTxnMockElse { + if mmElse.mock.inspectFuncElse != nil { + mmElse.mock.t.Fatalf("Inspect function is already set for ETCDTxnMock.Else") + } + + mmElse.mock.inspectFuncElse = f + + return mmElse +} + +// Return sets up results that will be returned by Txn.Else +func (mmElse *mETCDTxnMockElse) Return(t1 mm_clientv3.Txn) *ETCDTxnMock { + if mmElse.mock.funcElse != nil { + mmElse.mock.t.Fatalf("ETCDTxnMock.Else mock is already set by Set") + } + + if mmElse.defaultExpectation == nil { + mmElse.defaultExpectation = &ETCDTxnMockElseExpectation{mock: mmElse.mock} + } + mmElse.defaultExpectation.results = &ETCDTxnMockElseResults{t1} + mmElse.defaultExpectation.returnOrigin = minimock.CallerInfo(1) + return mmElse.mock +} + +// Set uses given function f to mock the Txn.Else method +func (mmElse *mETCDTxnMockElse) Set(f func(ops ...mm_clientv3.Op) (t1 mm_clientv3.Txn)) *ETCDTxnMock { + if mmElse.defaultExpectation != nil { + mmElse.mock.t.Fatalf("Default expectation is already set for the Txn.Else method") + } + + if len(mmElse.expectations) > 0 { + mmElse.mock.t.Fatalf("Some expectations are already set for the Txn.Else method") + } + + mmElse.mock.funcElse = f + mmElse.mock.funcElseOrigin = minimock.CallerInfo(1) + return mmElse.mock +} + +// When sets expectation for the Txn.Else which will trigger the result defined by the following +// Then helper +func (mmElse *mETCDTxnMockElse) When(ops ...mm_clientv3.Op) *ETCDTxnMockElseExpectation { + if mmElse.mock.funcElse != nil { + mmElse.mock.t.Fatalf("ETCDTxnMock.Else mock is already set by Set") + } + + expectation := &ETCDTxnMockElseExpectation{ + mock: mmElse.mock, + params: &ETCDTxnMockElseParams{ops}, + expectationOrigins: ETCDTxnMockElseExpectationOrigins{origin: minimock.CallerInfo(1)}, + } + mmElse.expectations = append(mmElse.expectations, expectation) + return expectation +} + +// Then sets up Txn.Else return parameters for the expectation previously defined by the When method +func (e *ETCDTxnMockElseExpectation) Then(t1 mm_clientv3.Txn) *ETCDTxnMock { + e.results = &ETCDTxnMockElseResults{t1} + return e.mock +} + +// Times sets number of times Txn.Else should be invoked +func (mmElse *mETCDTxnMockElse) Times(n uint64) *mETCDTxnMockElse { + if n == 0 { + mmElse.mock.t.Fatalf("Times of ETCDTxnMock.Else mock can not be zero") + } + mm_atomic.StoreUint64(&mmElse.expectedInvocations, n) + mmElse.expectedInvocationsOrigin = minimock.CallerInfo(1) + return mmElse +} + +func (mmElse *mETCDTxnMockElse) invocationsDone() bool { + if len(mmElse.expectations) == 0 && mmElse.defaultExpectation == nil && mmElse.mock.funcElse == nil { + return true + } + + totalInvocations := mm_atomic.LoadUint64(&mmElse.mock.afterElseCounter) + expectedInvocations := mm_atomic.LoadUint64(&mmElse.expectedInvocations) + + return totalInvocations > 0 && (expectedInvocations == 0 || expectedInvocations == totalInvocations) +} + +// Else implements mm_clientv3.Txn +func (mmElse *ETCDTxnMock) Else(ops ...mm_clientv3.Op) (t1 mm_clientv3.Txn) { + mm_atomic.AddUint64(&mmElse.beforeElseCounter, 1) + defer mm_atomic.AddUint64(&mmElse.afterElseCounter, 1) + + mmElse.t.Helper() + + if mmElse.inspectFuncElse != nil { + mmElse.inspectFuncElse(ops...) + } + + mm_params := ETCDTxnMockElseParams{ops} + + // Record call args + mmElse.ElseMock.mutex.Lock() + mmElse.ElseMock.callArgs = append(mmElse.ElseMock.callArgs, &mm_params) + mmElse.ElseMock.mutex.Unlock() + + for _, e := range mmElse.ElseMock.expectations { + if minimock.Equal(*e.params, mm_params) { + mm_atomic.AddUint64(&e.Counter, 1) + return e.results.t1 + } + } + + if mmElse.ElseMock.defaultExpectation != nil { + mm_atomic.AddUint64(&mmElse.ElseMock.defaultExpectation.Counter, 1) + mm_want := mmElse.ElseMock.defaultExpectation.params + mm_want_ptrs := mmElse.ElseMock.defaultExpectation.paramPtrs + + mm_got := ETCDTxnMockElseParams{ops} + + if mm_want_ptrs != nil { + + if mm_want_ptrs.ops != nil && !minimock.Equal(*mm_want_ptrs.ops, mm_got.ops) { + mmElse.t.Errorf("ETCDTxnMock.Else got unexpected parameter ops, expected at\n%s:\nwant: %#v\n got: %#v%s\n", + mmElse.ElseMock.defaultExpectation.expectationOrigins.originOps, *mm_want_ptrs.ops, mm_got.ops, minimock.Diff(*mm_want_ptrs.ops, mm_got.ops)) + } + + } else if mm_want != nil && !minimock.Equal(*mm_want, mm_got) { + mmElse.t.Errorf("ETCDTxnMock.Else got unexpected parameters, expected at\n%s:\nwant: %#v\n got: %#v%s\n", + mmElse.ElseMock.defaultExpectation.expectationOrigins.origin, *mm_want, mm_got, minimock.Diff(*mm_want, mm_got)) + } + + mm_results := mmElse.ElseMock.defaultExpectation.results + if mm_results == nil { + mmElse.t.Fatal("No results are set for the ETCDTxnMock.Else") + } + return (*mm_results).t1 + } + if mmElse.funcElse != nil { + return mmElse.funcElse(ops...) + } + mmElse.t.Fatalf("Unexpected call to ETCDTxnMock.Else. %v", ops) + return +} + +// ElseAfterCounter returns a count of finished ETCDTxnMock.Else invocations +func (mmElse *ETCDTxnMock) ElseAfterCounter() uint64 { + return mm_atomic.LoadUint64(&mmElse.afterElseCounter) +} + +// ElseBeforeCounter returns a count of ETCDTxnMock.Else invocations +func (mmElse *ETCDTxnMock) ElseBeforeCounter() uint64 { + return mm_atomic.LoadUint64(&mmElse.beforeElseCounter) +} + +// Calls returns a list of arguments used in each call to ETCDTxnMock.Else. +// The list is in the same order as the calls were made (i.e. recent calls have a higher index) +func (mmElse *mETCDTxnMockElse) Calls() []*ETCDTxnMockElseParams { + mmElse.mutex.RLock() + + argCopy := make([]*ETCDTxnMockElseParams, len(mmElse.callArgs)) + copy(argCopy, mmElse.callArgs) + + mmElse.mutex.RUnlock() + + return argCopy +} + +// MinimockElseDone returns true if the count of the Else invocations corresponds +// the number of defined expectations +func (m *ETCDTxnMock) MinimockElseDone() bool { + if m.ElseMock.optional { + // Optional methods provide '0 or more' call count restriction. + return true + } + + for _, e := range m.ElseMock.expectations { + if mm_atomic.LoadUint64(&e.Counter) < 1 { + return false + } + } + + return m.ElseMock.invocationsDone() +} + +// MinimockElseInspect logs each unmet expectation +func (m *ETCDTxnMock) MinimockElseInspect() { + for _, e := range m.ElseMock.expectations { + if mm_atomic.LoadUint64(&e.Counter) < 1 { + m.t.Errorf("Expected call to ETCDTxnMock.Else at\n%s with params: %#v", e.expectationOrigins.origin, *e.params) + } + } + + afterElseCounter := mm_atomic.LoadUint64(&m.afterElseCounter) + // if default expectation was set then invocations count should be greater than zero + if m.ElseMock.defaultExpectation != nil && afterElseCounter < 1 { + if m.ElseMock.defaultExpectation.params == nil { + m.t.Errorf("Expected call to ETCDTxnMock.Else at\n%s", m.ElseMock.defaultExpectation.returnOrigin) + } else { + m.t.Errorf("Expected call to ETCDTxnMock.Else at\n%s with params: %#v", m.ElseMock.defaultExpectation.expectationOrigins.origin, *m.ElseMock.defaultExpectation.params) + } + } + // if func was set then invocations count should be greater than zero + if m.funcElse != nil && afterElseCounter < 1 { + m.t.Errorf("Expected call to ETCDTxnMock.Else at\n%s", m.funcElseOrigin) + } + + if !m.ElseMock.invocationsDone() && afterElseCounter > 0 { + m.t.Errorf("Expected %d calls to ETCDTxnMock.Else at\n%s but found %d calls", + mm_atomic.LoadUint64(&m.ElseMock.expectedInvocations), m.ElseMock.expectedInvocationsOrigin, afterElseCounter) + } +} + +type mETCDTxnMockIf struct { + optional bool + mock *ETCDTxnMock + defaultExpectation *ETCDTxnMockIfExpectation + expectations []*ETCDTxnMockIfExpectation + + callArgs []*ETCDTxnMockIfParams + mutex sync.RWMutex + + expectedInvocations uint64 + expectedInvocationsOrigin string +} + +// ETCDTxnMockIfExpectation specifies expectation struct of the Txn.If +type ETCDTxnMockIfExpectation struct { + mock *ETCDTxnMock + params *ETCDTxnMockIfParams + paramPtrs *ETCDTxnMockIfParamPtrs + expectationOrigins ETCDTxnMockIfExpectationOrigins + results *ETCDTxnMockIfResults + returnOrigin string + Counter uint64 +} + +// ETCDTxnMockIfParams contains parameters of the Txn.If +type ETCDTxnMockIfParams struct { + cs []mm_clientv3.Cmp +} + +// ETCDTxnMockIfParamPtrs contains pointers to parameters of the Txn.If +type ETCDTxnMockIfParamPtrs struct { + cs *[]mm_clientv3.Cmp +} + +// ETCDTxnMockIfResults contains results of the Txn.If +type ETCDTxnMockIfResults struct { + t1 mm_clientv3.Txn +} + +// ETCDTxnMockIfOrigins contains origins of expectations of the Txn.If +type ETCDTxnMockIfExpectationOrigins struct { + origin string + originCs string +} + +// Marks this method to be optional. The default behavior of any method with Return() is '1 or more', meaning +// the test will fail minimock's automatic final call check if the mocked method was not called at least once. +// Optional() makes method check to work in '0 or more' mode. +// It is NOT RECOMMENDED to use this option unless you really need it, as default behaviour helps to +// catch the problems when the expected method call is totally skipped during test run. +func (mmIf *mETCDTxnMockIf) Optional() *mETCDTxnMockIf { + mmIf.optional = true + return mmIf +} + +// Expect sets up expected params for Txn.If +func (mmIf *mETCDTxnMockIf) Expect(cs ...mm_clientv3.Cmp) *mETCDTxnMockIf { + if mmIf.mock.funcIf != nil { + mmIf.mock.t.Fatalf("ETCDTxnMock.If mock is already set by Set") + } + + if mmIf.defaultExpectation == nil { + mmIf.defaultExpectation = &ETCDTxnMockIfExpectation{} + } + + if mmIf.defaultExpectation.paramPtrs != nil { + mmIf.mock.t.Fatalf("ETCDTxnMock.If mock is already set by ExpectParams functions") + } + + mmIf.defaultExpectation.params = &ETCDTxnMockIfParams{cs} + mmIf.defaultExpectation.expectationOrigins.origin = minimock.CallerInfo(1) + for _, e := range mmIf.expectations { + if minimock.Equal(e.params, mmIf.defaultExpectation.params) { + mmIf.mock.t.Fatalf("Expectation set by When has same params: %#v", *mmIf.defaultExpectation.params) + } + } + + return mmIf +} + +// ExpectCsParam1 sets up expected param cs for Txn.If +func (mmIf *mETCDTxnMockIf) ExpectCsParam1(cs ...mm_clientv3.Cmp) *mETCDTxnMockIf { + if mmIf.mock.funcIf != nil { + mmIf.mock.t.Fatalf("ETCDTxnMock.If mock is already set by Set") + } + + if mmIf.defaultExpectation == nil { + mmIf.defaultExpectation = &ETCDTxnMockIfExpectation{} + } + + if mmIf.defaultExpectation.params != nil { + mmIf.mock.t.Fatalf("ETCDTxnMock.If mock is already set by Expect") + } + + if mmIf.defaultExpectation.paramPtrs == nil { + mmIf.defaultExpectation.paramPtrs = &ETCDTxnMockIfParamPtrs{} + } + mmIf.defaultExpectation.paramPtrs.cs = &cs + mmIf.defaultExpectation.expectationOrigins.originCs = minimock.CallerInfo(1) + + return mmIf +} + +// Inspect accepts an inspector function that has same arguments as the Txn.If +func (mmIf *mETCDTxnMockIf) Inspect(f func(cs ...mm_clientv3.Cmp)) *mETCDTxnMockIf { + if mmIf.mock.inspectFuncIf != nil { + mmIf.mock.t.Fatalf("Inspect function is already set for ETCDTxnMock.If") + } + + mmIf.mock.inspectFuncIf = f + + return mmIf +} + +// Return sets up results that will be returned by Txn.If +func (mmIf *mETCDTxnMockIf) Return(t1 mm_clientv3.Txn) *ETCDTxnMock { + if mmIf.mock.funcIf != nil { + mmIf.mock.t.Fatalf("ETCDTxnMock.If mock is already set by Set") + } + + if mmIf.defaultExpectation == nil { + mmIf.defaultExpectation = &ETCDTxnMockIfExpectation{mock: mmIf.mock} + } + mmIf.defaultExpectation.results = &ETCDTxnMockIfResults{t1} + mmIf.defaultExpectation.returnOrigin = minimock.CallerInfo(1) + return mmIf.mock +} + +// Set uses given function f to mock the Txn.If method +func (mmIf *mETCDTxnMockIf) Set(f func(cs ...mm_clientv3.Cmp) (t1 mm_clientv3.Txn)) *ETCDTxnMock { + if mmIf.defaultExpectation != nil { + mmIf.mock.t.Fatalf("Default expectation is already set for the Txn.If method") + } + + if len(mmIf.expectations) > 0 { + mmIf.mock.t.Fatalf("Some expectations are already set for the Txn.If method") + } + + mmIf.mock.funcIf = f + mmIf.mock.funcIfOrigin = minimock.CallerInfo(1) + return mmIf.mock +} + +// When sets expectation for the Txn.If which will trigger the result defined by the following +// Then helper +func (mmIf *mETCDTxnMockIf) When(cs ...mm_clientv3.Cmp) *ETCDTxnMockIfExpectation { + if mmIf.mock.funcIf != nil { + mmIf.mock.t.Fatalf("ETCDTxnMock.If mock is already set by Set") + } + + expectation := &ETCDTxnMockIfExpectation{ + mock: mmIf.mock, + params: &ETCDTxnMockIfParams{cs}, + expectationOrigins: ETCDTxnMockIfExpectationOrigins{origin: minimock.CallerInfo(1)}, + } + mmIf.expectations = append(mmIf.expectations, expectation) + return expectation +} + +// Then sets up Txn.If return parameters for the expectation previously defined by the When method +func (e *ETCDTxnMockIfExpectation) Then(t1 mm_clientv3.Txn) *ETCDTxnMock { + e.results = &ETCDTxnMockIfResults{t1} + return e.mock +} + +// Times sets number of times Txn.If should be invoked +func (mmIf *mETCDTxnMockIf) Times(n uint64) *mETCDTxnMockIf { + if n == 0 { + mmIf.mock.t.Fatalf("Times of ETCDTxnMock.If mock can not be zero") + } + mm_atomic.StoreUint64(&mmIf.expectedInvocations, n) + mmIf.expectedInvocationsOrigin = minimock.CallerInfo(1) + return mmIf +} + +func (mmIf *mETCDTxnMockIf) invocationsDone() bool { + if len(mmIf.expectations) == 0 && mmIf.defaultExpectation == nil && mmIf.mock.funcIf == nil { + return true + } + + totalInvocations := mm_atomic.LoadUint64(&mmIf.mock.afterIfCounter) + expectedInvocations := mm_atomic.LoadUint64(&mmIf.expectedInvocations) + + return totalInvocations > 0 && (expectedInvocations == 0 || expectedInvocations == totalInvocations) +} + +// If implements mm_clientv3.Txn +func (mmIf *ETCDTxnMock) If(cs ...mm_clientv3.Cmp) (t1 mm_clientv3.Txn) { + mm_atomic.AddUint64(&mmIf.beforeIfCounter, 1) + defer mm_atomic.AddUint64(&mmIf.afterIfCounter, 1) + + mmIf.t.Helper() + + if mmIf.inspectFuncIf != nil { + mmIf.inspectFuncIf(cs...) + } + + mm_params := ETCDTxnMockIfParams{cs} + + // Record call args + mmIf.IfMock.mutex.Lock() + mmIf.IfMock.callArgs = append(mmIf.IfMock.callArgs, &mm_params) + mmIf.IfMock.mutex.Unlock() + + for _, e := range mmIf.IfMock.expectations { + if minimock.Equal(*e.params, mm_params) { + mm_atomic.AddUint64(&e.Counter, 1) + return e.results.t1 + } + } + + if mmIf.IfMock.defaultExpectation != nil { + mm_atomic.AddUint64(&mmIf.IfMock.defaultExpectation.Counter, 1) + mm_want := mmIf.IfMock.defaultExpectation.params + mm_want_ptrs := mmIf.IfMock.defaultExpectation.paramPtrs + + mm_got := ETCDTxnMockIfParams{cs} + + if mm_want_ptrs != nil { + + if mm_want_ptrs.cs != nil && !minimock.Equal(*mm_want_ptrs.cs, mm_got.cs) { + mmIf.t.Errorf("ETCDTxnMock.If got unexpected parameter cs, expected at\n%s:\nwant: %#v\n got: %#v%s\n", + mmIf.IfMock.defaultExpectation.expectationOrigins.originCs, *mm_want_ptrs.cs, mm_got.cs, minimock.Diff(*mm_want_ptrs.cs, mm_got.cs)) + } + + } else if mm_want != nil && !minimock.Equal(*mm_want, mm_got) { + mmIf.t.Errorf("ETCDTxnMock.If got unexpected parameters, expected at\n%s:\nwant: %#v\n got: %#v%s\n", + mmIf.IfMock.defaultExpectation.expectationOrigins.origin, *mm_want, mm_got, minimock.Diff(*mm_want, mm_got)) + } + + mm_results := mmIf.IfMock.defaultExpectation.results + if mm_results == nil { + mmIf.t.Fatal("No results are set for the ETCDTxnMock.If") + } + return (*mm_results).t1 + } + if mmIf.funcIf != nil { + return mmIf.funcIf(cs...) + } + mmIf.t.Fatalf("Unexpected call to ETCDTxnMock.If. %v", cs) + return +} + +// IfAfterCounter returns a count of finished ETCDTxnMock.If invocations +func (mmIf *ETCDTxnMock) IfAfterCounter() uint64 { + return mm_atomic.LoadUint64(&mmIf.afterIfCounter) +} + +// IfBeforeCounter returns a count of ETCDTxnMock.If invocations +func (mmIf *ETCDTxnMock) IfBeforeCounter() uint64 { + return mm_atomic.LoadUint64(&mmIf.beforeIfCounter) +} + +// Calls returns a list of arguments used in each call to ETCDTxnMock.If. +// The list is in the same order as the calls were made (i.e. recent calls have a higher index) +func (mmIf *mETCDTxnMockIf) Calls() []*ETCDTxnMockIfParams { + mmIf.mutex.RLock() + + argCopy := make([]*ETCDTxnMockIfParams, len(mmIf.callArgs)) + copy(argCopy, mmIf.callArgs) + + mmIf.mutex.RUnlock() + + return argCopy +} + +// MinimockIfDone returns true if the count of the If invocations corresponds +// the number of defined expectations +func (m *ETCDTxnMock) MinimockIfDone() bool { + if m.IfMock.optional { + // Optional methods provide '0 or more' call count restriction. + return true + } + + for _, e := range m.IfMock.expectations { + if mm_atomic.LoadUint64(&e.Counter) < 1 { + return false + } + } + + return m.IfMock.invocationsDone() +} + +// MinimockIfInspect logs each unmet expectation +func (m *ETCDTxnMock) MinimockIfInspect() { + for _, e := range m.IfMock.expectations { + if mm_atomic.LoadUint64(&e.Counter) < 1 { + m.t.Errorf("Expected call to ETCDTxnMock.If at\n%s with params: %#v", e.expectationOrigins.origin, *e.params) + } + } + + afterIfCounter := mm_atomic.LoadUint64(&m.afterIfCounter) + // if default expectation was set then invocations count should be greater than zero + if m.IfMock.defaultExpectation != nil && afterIfCounter < 1 { + if m.IfMock.defaultExpectation.params == nil { + m.t.Errorf("Expected call to ETCDTxnMock.If at\n%s", m.IfMock.defaultExpectation.returnOrigin) + } else { + m.t.Errorf("Expected call to ETCDTxnMock.If at\n%s with params: %#v", m.IfMock.defaultExpectation.expectationOrigins.origin, *m.IfMock.defaultExpectation.params) + } + } + // if func was set then invocations count should be greater than zero + if m.funcIf != nil && afterIfCounter < 1 { + m.t.Errorf("Expected call to ETCDTxnMock.If at\n%s", m.funcIfOrigin) + } + + if !m.IfMock.invocationsDone() && afterIfCounter > 0 { + m.t.Errorf("Expected %d calls to ETCDTxnMock.If at\n%s but found %d calls", + mm_atomic.LoadUint64(&m.IfMock.expectedInvocations), m.IfMock.expectedInvocationsOrigin, afterIfCounter) + } +} + +type mETCDTxnMockThen struct { + optional bool + mock *ETCDTxnMock + defaultExpectation *ETCDTxnMockThenExpectation + expectations []*ETCDTxnMockThenExpectation + + callArgs []*ETCDTxnMockThenParams + mutex sync.RWMutex + + expectedInvocations uint64 + expectedInvocationsOrigin string +} + +// ETCDTxnMockThenExpectation specifies expectation struct of the Txn.Then +type ETCDTxnMockThenExpectation struct { + mock *ETCDTxnMock + params *ETCDTxnMockThenParams + paramPtrs *ETCDTxnMockThenParamPtrs + expectationOrigins ETCDTxnMockThenExpectationOrigins + results *ETCDTxnMockThenResults + returnOrigin string + Counter uint64 +} + +// ETCDTxnMockThenParams contains parameters of the Txn.Then +type ETCDTxnMockThenParams struct { + ops []mm_clientv3.Op +} + +// ETCDTxnMockThenParamPtrs contains pointers to parameters of the Txn.Then +type ETCDTxnMockThenParamPtrs struct { + ops *[]mm_clientv3.Op +} + +// ETCDTxnMockThenResults contains results of the Txn.Then +type ETCDTxnMockThenResults struct { + t1 mm_clientv3.Txn +} + +// ETCDTxnMockThenOrigins contains origins of expectations of the Txn.Then +type ETCDTxnMockThenExpectationOrigins struct { + origin string + originOps string +} + +// Marks this method to be optional. The default behavior of any method with Return() is '1 or more', meaning +// the test will fail minimock's automatic final call check if the mocked method was not called at least once. +// Optional() makes method check to work in '0 or more' mode. +// It is NOT RECOMMENDED to use this option unless you really need it, as default behaviour helps to +// catch the problems when the expected method call is totally skipped during test run. +func (mmThen *mETCDTxnMockThen) Optional() *mETCDTxnMockThen { + mmThen.optional = true + return mmThen +} + +// Expect sets up expected params for Txn.Then +func (mmThen *mETCDTxnMockThen) Expect(ops ...mm_clientv3.Op) *mETCDTxnMockThen { + if mmThen.mock.funcThen != nil { + mmThen.mock.t.Fatalf("ETCDTxnMock.Then mock is already set by Set") + } + + if mmThen.defaultExpectation == nil { + mmThen.defaultExpectation = &ETCDTxnMockThenExpectation{} + } + + if mmThen.defaultExpectation.paramPtrs != nil { + mmThen.mock.t.Fatalf("ETCDTxnMock.Then mock is already set by ExpectParams functions") + } + + mmThen.defaultExpectation.params = &ETCDTxnMockThenParams{ops} + mmThen.defaultExpectation.expectationOrigins.origin = minimock.CallerInfo(1) + for _, e := range mmThen.expectations { + if minimock.Equal(e.params, mmThen.defaultExpectation.params) { + mmThen.mock.t.Fatalf("Expectation set by When has same params: %#v", *mmThen.defaultExpectation.params) + } + } + + return mmThen +} + +// ExpectOpsParam1 sets up expected param ops for Txn.Then +func (mmThen *mETCDTxnMockThen) ExpectOpsParam1(ops ...mm_clientv3.Op) *mETCDTxnMockThen { + if mmThen.mock.funcThen != nil { + mmThen.mock.t.Fatalf("ETCDTxnMock.Then mock is already set by Set") + } + + if mmThen.defaultExpectation == nil { + mmThen.defaultExpectation = &ETCDTxnMockThenExpectation{} + } + + if mmThen.defaultExpectation.params != nil { + mmThen.mock.t.Fatalf("ETCDTxnMock.Then mock is already set by Expect") + } + + if mmThen.defaultExpectation.paramPtrs == nil { + mmThen.defaultExpectation.paramPtrs = &ETCDTxnMockThenParamPtrs{} + } + mmThen.defaultExpectation.paramPtrs.ops = &ops + mmThen.defaultExpectation.expectationOrigins.originOps = minimock.CallerInfo(1) + + return mmThen +} + +// Inspect accepts an inspector function that has same arguments as the Txn.Then +func (mmThen *mETCDTxnMockThen) Inspect(f func(ops ...mm_clientv3.Op)) *mETCDTxnMockThen { + if mmThen.mock.inspectFuncThen != nil { + mmThen.mock.t.Fatalf("Inspect function is already set for ETCDTxnMock.Then") + } + + mmThen.mock.inspectFuncThen = f + + return mmThen +} + +// Return sets up results that will be returned by Txn.Then +func (mmThen *mETCDTxnMockThen) Return(t1 mm_clientv3.Txn) *ETCDTxnMock { + if mmThen.mock.funcThen != nil { + mmThen.mock.t.Fatalf("ETCDTxnMock.Then mock is already set by Set") + } + + if mmThen.defaultExpectation == nil { + mmThen.defaultExpectation = &ETCDTxnMockThenExpectation{mock: mmThen.mock} + } + mmThen.defaultExpectation.results = &ETCDTxnMockThenResults{t1} + mmThen.defaultExpectation.returnOrigin = minimock.CallerInfo(1) + return mmThen.mock +} + +// Set uses given function f to mock the Txn.Then method +func (mmThen *mETCDTxnMockThen) Set(f func(ops ...mm_clientv3.Op) (t1 mm_clientv3.Txn)) *ETCDTxnMock { + if mmThen.defaultExpectation != nil { + mmThen.mock.t.Fatalf("Default expectation is already set for the Txn.Then method") + } + + if len(mmThen.expectations) > 0 { + mmThen.mock.t.Fatalf("Some expectations are already set for the Txn.Then method") + } + + mmThen.mock.funcThen = f + mmThen.mock.funcThenOrigin = minimock.CallerInfo(1) + return mmThen.mock +} + +// When sets expectation for the Txn.Then which will trigger the result defined by the following +// Then helper +func (mmThen *mETCDTxnMockThen) When(ops ...mm_clientv3.Op) *ETCDTxnMockThenExpectation { + if mmThen.mock.funcThen != nil { + mmThen.mock.t.Fatalf("ETCDTxnMock.Then mock is already set by Set") + } + + expectation := &ETCDTxnMockThenExpectation{ + mock: mmThen.mock, + params: &ETCDTxnMockThenParams{ops}, + expectationOrigins: ETCDTxnMockThenExpectationOrigins{origin: minimock.CallerInfo(1)}, + } + mmThen.expectations = append(mmThen.expectations, expectation) + return expectation +} + +// Then sets up Txn.Then return parameters for the expectation previously defined by the When method +func (e *ETCDTxnMockThenExpectation) Then(t1 mm_clientv3.Txn) *ETCDTxnMock { + e.results = &ETCDTxnMockThenResults{t1} + return e.mock +} + +// Times sets number of times Txn.Then should be invoked +func (mmThen *mETCDTxnMockThen) Times(n uint64) *mETCDTxnMockThen { + if n == 0 { + mmThen.mock.t.Fatalf("Times of ETCDTxnMock.Then mock can not be zero") + } + mm_atomic.StoreUint64(&mmThen.expectedInvocations, n) + mmThen.expectedInvocationsOrigin = minimock.CallerInfo(1) + return mmThen +} + +func (mmThen *mETCDTxnMockThen) invocationsDone() bool { + if len(mmThen.expectations) == 0 && mmThen.defaultExpectation == nil && mmThen.mock.funcThen == nil { + return true + } + + totalInvocations := mm_atomic.LoadUint64(&mmThen.mock.afterThenCounter) + expectedInvocations := mm_atomic.LoadUint64(&mmThen.expectedInvocations) + + return totalInvocations > 0 && (expectedInvocations == 0 || expectedInvocations == totalInvocations) +} + +// Then implements mm_clientv3.Txn +func (mmThen *ETCDTxnMock) Then(ops ...mm_clientv3.Op) (t1 mm_clientv3.Txn) { + mm_atomic.AddUint64(&mmThen.beforeThenCounter, 1) + defer mm_atomic.AddUint64(&mmThen.afterThenCounter, 1) + + mmThen.t.Helper() + + if mmThen.inspectFuncThen != nil { + mmThen.inspectFuncThen(ops...) + } + + mm_params := ETCDTxnMockThenParams{ops} + + // Record call args + mmThen.ThenMock.mutex.Lock() + mmThen.ThenMock.callArgs = append(mmThen.ThenMock.callArgs, &mm_params) + mmThen.ThenMock.mutex.Unlock() + + for _, e := range mmThen.ThenMock.expectations { + if minimock.Equal(*e.params, mm_params) { + mm_atomic.AddUint64(&e.Counter, 1) + return e.results.t1 + } + } + + if mmThen.ThenMock.defaultExpectation != nil { + mm_atomic.AddUint64(&mmThen.ThenMock.defaultExpectation.Counter, 1) + mm_want := mmThen.ThenMock.defaultExpectation.params + mm_want_ptrs := mmThen.ThenMock.defaultExpectation.paramPtrs + + mm_got := ETCDTxnMockThenParams{ops} + + if mm_want_ptrs != nil { + + if mm_want_ptrs.ops != nil && !minimock.Equal(*mm_want_ptrs.ops, mm_got.ops) { + mmThen.t.Errorf("ETCDTxnMock.Then got unexpected parameter ops, expected at\n%s:\nwant: %#v\n got: %#v%s\n", + mmThen.ThenMock.defaultExpectation.expectationOrigins.originOps, *mm_want_ptrs.ops, mm_got.ops, minimock.Diff(*mm_want_ptrs.ops, mm_got.ops)) + } + + } else if mm_want != nil && !minimock.Equal(*mm_want, mm_got) { + mmThen.t.Errorf("ETCDTxnMock.Then got unexpected parameters, expected at\n%s:\nwant: %#v\n got: %#v%s\n", + mmThen.ThenMock.defaultExpectation.expectationOrigins.origin, *mm_want, mm_got, minimock.Diff(*mm_want, mm_got)) + } + + mm_results := mmThen.ThenMock.defaultExpectation.results + if mm_results == nil { + mmThen.t.Fatal("No results are set for the ETCDTxnMock.Then") + } + return (*mm_results).t1 + } + if mmThen.funcThen != nil { + return mmThen.funcThen(ops...) + } + mmThen.t.Fatalf("Unexpected call to ETCDTxnMock.Then. %v", ops) + return +} + +// ThenAfterCounter returns a count of finished ETCDTxnMock.Then invocations +func (mmThen *ETCDTxnMock) ThenAfterCounter() uint64 { + return mm_atomic.LoadUint64(&mmThen.afterThenCounter) +} + +// ThenBeforeCounter returns a count of ETCDTxnMock.Then invocations +func (mmThen *ETCDTxnMock) ThenBeforeCounter() uint64 { + return mm_atomic.LoadUint64(&mmThen.beforeThenCounter) +} + +// Calls returns a list of arguments used in each call to ETCDTxnMock.Then. +// The list is in the same order as the calls were made (i.e. recent calls have a higher index) +func (mmThen *mETCDTxnMockThen) Calls() []*ETCDTxnMockThenParams { + mmThen.mutex.RLock() + + argCopy := make([]*ETCDTxnMockThenParams, len(mmThen.callArgs)) + copy(argCopy, mmThen.callArgs) + + mmThen.mutex.RUnlock() + + return argCopy +} + +// MinimockThenDone returns true if the count of the Then invocations corresponds +// the number of defined expectations +func (m *ETCDTxnMock) MinimockThenDone() bool { + if m.ThenMock.optional { + // Optional methods provide '0 or more' call count restriction. + return true + } + + for _, e := range m.ThenMock.expectations { + if mm_atomic.LoadUint64(&e.Counter) < 1 { + return false + } + } + + return m.ThenMock.invocationsDone() +} + +// MinimockThenInspect logs each unmet expectation +func (m *ETCDTxnMock) MinimockThenInspect() { + for _, e := range m.ThenMock.expectations { + if mm_atomic.LoadUint64(&e.Counter) < 1 { + m.t.Errorf("Expected call to ETCDTxnMock.Then at\n%s with params: %#v", e.expectationOrigins.origin, *e.params) + } + } + + afterThenCounter := mm_atomic.LoadUint64(&m.afterThenCounter) + // if default expectation was set then invocations count should be greater than zero + if m.ThenMock.defaultExpectation != nil && afterThenCounter < 1 { + if m.ThenMock.defaultExpectation.params == nil { + m.t.Errorf("Expected call to ETCDTxnMock.Then at\n%s", m.ThenMock.defaultExpectation.returnOrigin) + } else { + m.t.Errorf("Expected call to ETCDTxnMock.Then at\n%s with params: %#v", m.ThenMock.defaultExpectation.expectationOrigins.origin, *m.ThenMock.defaultExpectation.params) + } + } + // if func was set then invocations count should be greater than zero + if m.funcThen != nil && afterThenCounter < 1 { + m.t.Errorf("Expected call to ETCDTxnMock.Then at\n%s", m.funcThenOrigin) + } + + if !m.ThenMock.invocationsDone() && afterThenCounter > 0 { + m.t.Errorf("Expected %d calls to ETCDTxnMock.Then at\n%s but found %d calls", + mm_atomic.LoadUint64(&m.ThenMock.expectedInvocations), m.ThenMock.expectedInvocationsOrigin, afterThenCounter) + } +} + +// MinimockFinish checks that all mocked methods have been called the expected number of times +func (m *ETCDTxnMock) MinimockFinish() { + m.finishOnce.Do(func() { + if !m.minimockDone() { + m.MinimockCommitInspect() + + m.MinimockElseInspect() + + m.MinimockIfInspect() + + m.MinimockThenInspect() + } + }) +} + +// MinimockWait waits for all mocked methods to be called the expected number of times +func (m *ETCDTxnMock) MinimockWait(timeout mm_time.Duration) { + timeoutCh := mm_time.After(timeout) + for { + if m.minimockDone() { + return + } + select { + case <-timeoutCh: + m.MinimockFinish() + return + case <-mm_time.After(10 * mm_time.Millisecond): + } + } +} + +func (m *ETCDTxnMock) minimockDone() bool { + done := true + return done && + m.MinimockCommitDone() && + m.MinimockElseDone() && + m.MinimockIfDone() && + m.MinimockThenDone() +} diff --git a/internal/testing/t.go b/internal/testing/t.go index 0da99c3..311c01e 100644 --- a/internal/testing/t.go +++ b/internal/testing/t.go @@ -3,6 +3,7 @@ package testing import ( "fmt" "os" + "testing" ) // T is a dummy implementation of the testing.T interface to use in examples. @@ -14,27 +15,113 @@ type T interface { Errorf(format string, args ...any) } -type dummyT struct{} +// DummyT is a dummy implementation of the testing.T interface to use in examples. +type DummyT struct { + testing.T -func (t *dummyT) Helper() {} + cleanups []func() +} + +// NewT returns a new dummy T instance. +func NewT() *DummyT { + return &DummyT{T: testing.T{}, cleanups: nil} +} + +var ( + _ T = &DummyT{} //nolint:exhaustruct + _ testing.TB = &DummyT{} //nolint:exhaustruct +) + +// Attr is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Attr(_, _ string) {} + +// Error is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Error(_ ...any) { +} + +// Fail is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Fail() { + panic("fail") +} + +// FailNow is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) FailNow() { + panic("fail now") +} + +// Failed is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Failed() bool { + return false +} + +// Fatal is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Fatal(args ...any) { + args = append([]any{"fatal: "}, args...) + panic(fmt.Sprint(args...)) +} -func (t *dummyT) Log(args ...any) { +// Name is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Name() string { + return "DummyT" +} + +// Setenv is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Setenv(_, _ string) {} + +// Chdir is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Chdir(_ string) {} + +// Skip is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Skip(_ ...any) { + panic("skip") +} + +// SkipNow is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) SkipNow() { + panic("skip now") +} + +// Skipf is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Skipf(format string, args ...any) { + panic(fmt.Sprintf("skip: "+format, args...)) +} + +// Skipped is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Skipped() bool { + return false +} + +// Helper is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Helper() {} + +// Log is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Log(args ...any) { _, _ = fmt.Fprintln(os.Stderr, args...) } -func (t *dummyT) Logf(format string, args ...any) { +// Logf is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Logf(format string, args ...any) { _, _ = fmt.Fprintf(os.Stderr, format, args...) } -func (t *dummyT) Fatalf(format string, args ...any) { +// Fatalf is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Fatalf(format string, args ...any) { panic("fatal error: " + fmt.Sprintf(format, args...)) } -func (t *dummyT) Errorf(format string, args ...any) { +// Errorf is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Errorf(format string, args ...any) { panic("error: " + fmt.Sprintf(format, args...)) } -// NewT returns a new dummy T instance. -func NewT() T { - return &dummyT{} +// Cleanup is a dummy implementation of the testing.T interface to use in examples. +func (t *DummyT) Cleanup(f func()) { + t.cleanups = append(t.cleanups, f) +} + +// Cleanups should be called to call all cleanup functions on "test" exit. +func (t *DummyT) Cleanups() { + for _, cleanupFunc := range t.cleanups { + cleanupFunc() + } } diff --git a/operation/operation.go b/operation/operation.go index e246b32..3af3737 100644 --- a/operation/operation.go +++ b/operation/operation.go @@ -2,6 +2,15 @@ // It defines operation types and configurations used in transactional contexts. package operation +import ( + "bytes" +) + +var ( + // isPrefixSuffix is a byte sequence that indicates that key is a prefix (ends with /). + isPrefixSuffix = []byte("/") //nolint:gochecknoglobals +) + // Operation represents a storage operation to be executed. // This is used within transactions and other operation contexts. type Operation struct { @@ -35,6 +44,15 @@ func (o Operation) Options() []Option { return o.options } +// IsPrefix returns true if the operation is a prefix operation. +func (o Operation) IsPrefix() bool { + if o.tp == TypeGet || o.tp == TypeDelete { + return bytes.HasSuffix(o.key, isPrefixSuffix) + } + + return false +} + // Get creates a new read operation for the specified key. // Returns an Operation configured for reading data from storage. func Get(key []byte, options ...Option) Operation { diff --git a/operation/operation_test.go b/operation/operation_test.go index 39aa50e..3075edb 100644 --- a/operation/operation_test.go +++ b/operation/operation_test.go @@ -81,3 +81,62 @@ func TestDeleteWithOptions(t *testing.T) { assert.Nil(t, op.Value()) assert.Len(t, op.Options(), 2) } + +func TestOperation_IsPrefix(t *testing.T) { + t.Parallel() + + tests := []struct { + name string + operation operation.Operation + want bool + }{ + { + name: "get operation with key", + operation: operation.Get([]byte("/test-key")), + want: false, + }, + { + name: "put operation with key and value", + operation: operation.Put([]byte("/test-key"), []byte("test-value")), + want: false, + }, + { + name: "delete operation with key", + operation: operation.Delete([]byte("/test-key")), + want: false, + }, + { + name: "get prefix operation with key", + operation: operation.Get([]byte("/test-key/")), + want: true, + }, + { + name: "delete prefix operation with key", + operation: operation.Delete([]byte("/test-key/")), + want: true, + }, + { + name: "put prefix operation with key and value", + operation: operation.Put([]byte("/test-key/"), []byte("test-value")), + want: false, + }, + { + name: "get root", + operation: operation.Get([]byte("/")), + want: true, + }, + { + name: "delete root", + operation: operation.Delete([]byte("/")), + want: true, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + + assert.Equal(t, tt.want, tt.operation.IsPrefix()) + }) + } +}