From 87ec3f0198fb7902822962e8e39bee514501ba70 Mon Sep 17 00:00:00 2001 From: ronaldngounou Date: Sun, 5 Oct 2025 14:25:24 -0700 Subject: [PATCH] Add mirror test implementation - Implement mirror test structure - Add test/common functionality - Create ctl_v3_make_mirror_test Signed-off-by: Ronald Ngounou Signed-off-by: ronaldngounou --- tests/common/e2e_test.go | 16 +++ tests/common/integration_test.go | 14 ++ tests/common/make_mirror_test.go | 159 +++++++++++++++++++++ tests/common/unit_test.go | 6 + tests/e2e/ctl_v3_make_mirror_test.go | 115 --------------- tests/framework/config/client.go | 11 ++ tests/framework/e2e/config.go | 7 +- tests/framework/e2e/e2e.go | 3 + tests/framework/e2e/etcdctl.go | 45 ++++++ tests/framework/integration/integration.go | 5 + tests/framework/interfaces/interface.go | 2 + 11 files changed, 265 insertions(+), 118 deletions(-) create mode 100644 tests/common/make_mirror_test.go delete mode 100644 tests/e2e/ctl_v3_make_mirror_test.go diff --git a/tests/common/e2e_test.go b/tests/common/e2e_test.go index 2453740175fb..0451cd562383 100644 --- a/tests/common/e2e_test.go +++ b/tests/common/e2e_test.go @@ -122,3 +122,19 @@ func WithUnixClient() config.ClusterOption { func WithTCPClient() config.ClusterOption { return e2e.WithTCPClient() } + +func WithBasePort(port int) config.ClusterOption { + return e2e.WithBasePort() +} + +func configureMirrorDestTLS(mm *config.MakeMirrorOptions, tls config.TLSConfig) { + switch tls { + case config.ManualTLS: + mm.DestCert = e2e.CertPath + mm.DestKey = e2e.PrivateKeyPath + mm.DestCACert = e2e.CaPath + mm.DestInsecureTransport = false + default: + mm.DestInsecureTransport = true + } +} diff --git a/tests/common/integration_test.go b/tests/common/integration_test.go index 53234cd04bc3..d63b0ac124b8 100644 --- a/tests/common/integration_test.go +++ b/tests/common/integration_test.go @@ -67,3 +67,17 @@ func WithUnixClient() config.ClusterOption { func WithTCPClient() config.ClusterOption { return integration.WithTCPClient() } + +func WithBasePort(port int) config.ClusterOption { + return func(c *config.ClusterConfig) {} +} + +func ensureIntegrationClusterContext(c *config.ClusterConfig) *integration.ClusterContext { + ctx, _ := c.ClusterContext.(*integration.ClusterContext) + if ctx == nil { + ctx = &integration.ClusterContext{} + } + return ctx +} + +func configureMirrorDestTLS(mm *config.MakeMirrorOptions, _ config.TLSConfig) {} diff --git a/tests/common/make_mirror_test.go b/tests/common/make_mirror_test.go new file mode 100644 index 000000000000..7562267de6af --- /dev/null +++ b/tests/common/make_mirror_test.go @@ -0,0 +1,159 @@ +// Copyright 2016 The etcd Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package common + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/stretchr/testify/require" + + "go.etcd.io/etcd/tests/v3/framework/config" + "go.etcd.io/etcd/tests/v3/framework/testutils" +) + +func TestMakeMirrorModifyDestPrefix(t *testing.T) { + testRunner.BeforeTest(t) + for _, srcTC := range clusterTestCases() { + for _, destTC := range clusterTestCases() { + t.Run(fmt.Sprintf("Source=%s/Destination=%s", srcTC.name, destTC.name), func(t *testing.T) { + var ( + mmOpts = config.MakeMirrorOptions{Prefix: "o_", DestPrefix: "d_"} + sourcekvs = []testutils.KV{{Key: "o_key1", Val: "val1"}, {Key: "o_key2", Val: "val2"}, {Key: "o_key3", Val: "val3"}} + destkvs = []testutils.KV{{Key: "d_key1", Val: "val1"}, {Key: "d_key2", Val: "val2"}, {Key: "d_key3", Val: "val3"}} + srcprefix = "o_" + destprefix = "d_" + ) + + testMirror(t, srcTC, destTC, mmOpts, sourcekvs, destkvs, srcprefix, destprefix) + }) + } + } +} + +func TestMakeMirrorNoDestPrefix(t *testing.T) { + testRunner.BeforeTest(t) + for _, srcTC := range clusterTestCases() { + for _, destTC := range clusterTestCases() { + t.Run(fmt.Sprintf("Source=%s/Destination=%s", srcTC.name, destTC.name), func(t *testing.T) { + var ( + mmOpts = config.MakeMirrorOptions{Prefix: "o_", NoDestPrefix: true} + sourcekvs = []testutils.KV{{Key: "o_key1", Val: "val1"}, {Key: "o_key2", Val: "val2"}, {Key: "o_key3", Val: "val3"}} + destkvs = []testutils.KV{{Key: "key1", Val: "val1"}, {Key: "key2", Val: "val2"}, {Key: "key3", Val: "val3"}} + srcprefix = "o_" + destprefix = "key" + ) + + testMirror(t, srcTC, destTC, mmOpts, sourcekvs, destkvs, srcprefix, destprefix) + }) + } + } +} + +func TestMakeMirror(t *testing.T) { + testRunner.BeforeTest(t) + for _, srcTC := range clusterTestCases() { + for _, destTC := range clusterTestCases() { + t.Run(fmt.Sprintf("Source=%s/Destination=%s", srcTC.name, destTC.name), func(t *testing.T) { + var ( + mmOpts config.MakeMirrorOptions + sourcekvs = []testutils.KV{{Key: "key1", Val: "val1"}, {Key: "key2", Val: "val2"}, {Key: "key3", Val: "val3"}} + destkvs = []testutils.KV{{Key: "key1", Val: "val1"}, {Key: "key2", Val: "val2"}, {Key: "key3", Val: "val3"}} + prefix = "key" + ) + + testMirror(t, srcTC, destTC, mmOpts, sourcekvs, destkvs, prefix, prefix) + }) + } + } +} + +func TestMakeMirrorWithWatchRev(t *testing.T) { + testRunner.BeforeTest(t) + for _, srcTC := range clusterTestCases() { + for _, destTC := range clusterTestCases() { + t.Run(fmt.Sprintf("Source=%s/Destination=%s", srcTC.name, destTC.name), func(t *testing.T) { + var ( + mmOpts = config.MakeMirrorOptions{Prefix: "o_", NoDestPrefix: true, Rev: 4} + sourcekvs = []testutils.KV{{Key: "o_key1", Val: "val1"}, {Key: "o_key2", Val: "val2"}, {Key: "o_key3", Val: "val3"}, {Key: "o_key4", Val: "val4"}} + destkvs = []testutils.KV{{Key: "key3", Val: "val3"}, {Key: "key4", Val: "val4"}} + srcprefix = "o_" + destprefix = "key" + ) + + testMirror(t, srcTC, destTC, mmOpts, sourcekvs, destkvs, srcprefix, destprefix) + }) + } + } +} + +func testMirror(t *testing.T, srcTC, destTC testCase, mmOpts config.MakeMirrorOptions, sourcekvs, destkvs []testutils.KV, srcprefix, destprefix string) { + t.Helper() + + if destTC.config.ClientTLS == config.AutoTLS { + t.Skip("Skipping: destingation uses Client AutoTLS, but the test cannot expose the dest CA for etcdctl (--dest-cacert); tLS verification would fail.") + } + + ctx, cancel := context.WithTimeout(t.Context(), 30*time.Second) + defer cancel() + // Source cluster + src := testRunner.NewCluster(ctx, t, config.WithClusterConfig(srcTC.config)) + defer src.Close() + srcClient := testutils.MustClient(src.Client()) + + // Destination cluster + destCfg := destTC.config + dest := testRunner.NewCluster(ctx, t, config.WithClusterConfig(destTC.config), WithBasePort(10000)) // destTC.BasePort + defer dest.Close() + destClient := testutils.MustClient(dest.Client()) + + // Configure TLS for destination before starting make-mirror + configureMirrorDestTLS(&mmOpts, destCfg.ClientTLS) + + // Start make mirror + errCh := make(chan error) + go func(opts config.MakeMirrorOptions) { + errCh <- srcClient.MakeMirror(ctx, dest.Endpoints()[0], opts) + }(mmOpts) + defer func() { + // Need to cancel the context to ensure the MakeMirror goroutine is cancelled before catching the error. + cancel() + require.NoError(t, <-errCh) + }() + + // Write to source + for i := range sourcekvs { + require.NoError(t, srcClient.Put(ctx, sourcekvs[i].Key, sourcekvs[i].Val, config.PutOptions{})) + } + + // Source assertion + srcResp, err := srcClient.Get(ctx, srcprefix, config.GetOptions{Prefix: true}) + require.NoError(t, err) + require.Equal(t, sourcekvs, testutils.KeyValuesFromGetResponse(srcResp)) + + // Destination assertion + wCtx, wCancel := context.WithCancel(ctx) + defer wCancel() + + watchChan := destClient.Watch(wCtx, destprefix, config.WatchOptions{Prefix: true, Revision: 1}) + + // Compare the result of what we obtained using the make-mirror command versus what we had using + // the Watch command + destResp, err := testutils.KeyValuesFromWatchChan(watchChan, len(destkvs), 10*time.Second) + require.NoError(t, err) + require.Equal(t, destkvs, destResp) +} diff --git a/tests/common/unit_test.go b/tests/common/unit_test.go index 52fe5d16b3ba..864862f6e26f 100644 --- a/tests/common/unit_test.go +++ b/tests/common/unit_test.go @@ -56,3 +56,9 @@ func WithTCPClient() config.ClusterOption { func WithUnixClient() config.ClusterOption { return func(c *config.ClusterConfig) {} } + +func WithBasePort(port int) config.ClusterOption { + return func(c *config.ClusterConfig) {} +} + +func configureMirrorDestTLS(mm *config.MakeMirrorOptions, _ config.TLSConfig) {} diff --git a/tests/e2e/ctl_v3_make_mirror_test.go b/tests/e2e/ctl_v3_make_mirror_test.go deleted file mode 100644 index 720c4f22de46..000000000000 --- a/tests/e2e/ctl_v3_make_mirror_test.go +++ /dev/null @@ -1,115 +0,0 @@ -// Copyright 2016 The etcd Authors -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package e2e - -import ( - "context" - "fmt" - "testing" - "time" - - "github.com/stretchr/testify/require" - - "go.etcd.io/etcd/tests/v3/framework/e2e" -) - -func TestCtlV3MakeMirror(t *testing.T) { testCtl(t, makeMirrorTest) } -func TestCtlV3MakeMirrorModifyDestPrefix(t *testing.T) { testCtl(t, makeMirrorModifyDestPrefixTest) } -func TestCtlV3MakeMirrorNoDestPrefix(t *testing.T) { testCtl(t, makeMirrorNoDestPrefixTest) } -func TestCtlV3MakeMirrorWithWatchRev(t *testing.T) { testCtl(t, makeMirrorWithWatchRev) } - -func makeMirrorTest(cx ctlCtx) { - var ( - flags []string - kvs = []kv{{"key1", "val1"}, {"key2", "val2"}, {"key3", "val3"}} - kvs2 = []kvExec{{key: "key1", val: "val1"}, {key: "key2", val: "val2"}, {key: "key3", val: "val3"}} - prefix = "key" - ) - testMirrorCommand(cx, flags, kvs, kvs2, prefix, prefix) -} - -func makeMirrorModifyDestPrefixTest(cx ctlCtx) { - var ( - flags = []string{"--prefix", "o_", "--dest-prefix", "d_"} - kvs = []kv{{"o_key1", "val1"}, {"o_key2", "val2"}, {"o_key3", "val3"}} - kvs2 = []kvExec{{key: "d_key1", val: "val1"}, {key: "d_key2", val: "val2"}, {key: "d_key3", val: "val3"}} - srcprefix = "o_" - destprefix = "d_" - ) - testMirrorCommand(cx, flags, kvs, kvs2, srcprefix, destprefix) -} - -func makeMirrorNoDestPrefixTest(cx ctlCtx) { - var ( - flags = []string{"--prefix", "o_", "--no-dest-prefix"} - kvs = []kv{{"o_key1", "val1"}, {"o_key2", "val2"}, {"o_key3", "val3"}} - kvs2 = []kvExec{{key: "key1", val: "val1"}, {key: "key2", val: "val2"}, {key: "key3", val: "val3"}} - srcprefix = "o_" - destprefix = "key" - ) - - testMirrorCommand(cx, flags, kvs, kvs2, srcprefix, destprefix) -} - -func makeMirrorWithWatchRev(cx ctlCtx) { - var ( - flags = []string{"--prefix", "o_", "--no-dest-prefix", "--rev", "4"} - kvs = []kv{{"o_key1", "val1"}, {"o_key2", "val2"}, {"o_key3", "val3"}, {"o_key4", "val4"}} - kvs2 = []kvExec{{key: "key3", val: "val3"}, {key: "key4", val: "val4"}} - srcprefix = "o_" - destprefix = "key" - ) - - testMirrorCommand(cx, flags, kvs, kvs2, srcprefix, destprefix) -} - -func testMirrorCommand(cx ctlCtx, flags []string, sourcekvs []kv, destkvs []kvExec, srcprefix, destprefix string) { - // set up another cluster to mirror with - mirrorcfg := e2e.NewConfigAutoTLS() - mirrorcfg.ClusterSize = 1 - mirrorcfg.BasePort = 10000 - mirrorctx := ctlCtx{ - t: cx.t, - cfg: *mirrorcfg, - dialTimeout: 7 * time.Second, - } - - mirrorepc, err := e2e.NewEtcdProcessCluster(context.TODO(), cx.t, e2e.WithConfig(&mirrorctx.cfg)) - if err != nil { - cx.t.Fatalf("could not start etcd process cluster (%v)", err) - } - mirrorctx.epc = mirrorepc - - defer func() { - if err = mirrorctx.epc.Close(); err != nil { - cx.t.Fatalf("error closing etcd processes (%v)", err) - } - }() - - cmdArgs := append(cx.PrefixArgs(), "make-mirror") - cmdArgs = append(cmdArgs, flags...) - cmdArgs = append(cmdArgs, fmt.Sprintf("localhost:%d", mirrorcfg.BasePort)) - proc, err := e2e.SpawnCmd(cmdArgs, cx.envMap) - require.NoError(cx.t, err) - defer func() { - require.NoError(cx.t, proc.Stop()) - }() - - for i := range sourcekvs { - require.NoError(cx.t, ctlV3Put(cx, sourcekvs[i].key, sourcekvs[i].val, "")) - } - require.NoError(cx.t, ctlV3Get(cx, []string{srcprefix, "--prefix"}, sourcekvs...)) - require.NoError(cx.t, ctlV3Watch(mirrorctx, []string{destprefix, "--rev", "1", "--prefix"}, destkvs...)) -} diff --git a/tests/framework/config/client.go b/tests/framework/config/client.go index ac82bd547925..634fb21c01b4 100644 --- a/tests/framework/config/client.go +++ b/tests/framework/config/client.go @@ -77,3 +77,14 @@ type WatchOptions struct { Revision int64 RangeEnd string } + +type MakeMirrorOptions struct { + Prefix string + Rev int64 + DestPrefix string + NoDestPrefix bool + DestCACert string + DestCert string + DestKey string + DestInsecureTransport bool +} diff --git a/tests/framework/e2e/config.go b/tests/framework/e2e/config.go index d05d9a1058c8..11777f3eb4d0 100644 --- a/tests/framework/e2e/config.go +++ b/tests/framework/e2e/config.go @@ -41,9 +41,10 @@ func (cv ClusterVersion) String() string { } type ClusterContext struct { - Version ClusterVersion - EnvVars map[string]string - UseUnix bool + Version ClusterVersion + EnvVars map[string]string + UseUnix bool + BasePort int } func WithHTTP2Debug() config.ClusterOption { diff --git a/tests/framework/e2e/e2e.go b/tests/framework/e2e/e2e.go index 7ab52006beac..1846be656cc8 100644 --- a/tests/framework/e2e/e2e.go +++ b/tests/framework/e2e/e2e.go @@ -66,6 +66,9 @@ func (e e2eRunner) NewCluster(ctx context.Context, tb testing.TB, opts ...config if ctx.UseUnix { e2eConfig.BaseClientScheme = "unix" } + if ctx.BasePort != 0 { + e2eConfig.BasePort = ctx.BasePort + } } switch cfg.ClientTLS { diff --git a/tests/framework/e2e/etcdctl.go b/tests/framework/e2e/etcdctl.go index c86fdb871eb0..4741ef0e6649 100644 --- a/tests/framework/e2e/etcdctl.go +++ b/tests/framework/e2e/etcdctl.go @@ -745,3 +745,48 @@ func (ctl *EtcdctlV3) Watch(ctx context.Context, key string, opts config.WatchOp return ch } + +func (ctl *EtcdctlV3) MakeMirror(ctx context.Context, destEndpoints string, opts config.MakeMirrorOptions) error { + args := ctl.cmdArgs() + args = append(args, "make-mirror") + + if opts.Prefix != "" { + args = append(args, "--prefix", opts.Prefix) + } + if opts.Rev != 0 { + args = append(args, "--rev", fmt.Sprint(opts.Rev)) + } + if opts.NoDestPrefix { + args = append(args, "--no-dest-prefix") + } + if opts.DestPrefix != "" { + args = append(args, "--dest-prefix", opts.DestPrefix) + } + if opts.DestCACert != "" { + args = append(args, "--dest-cacert", opts.DestCACert) + } + if opts.DestCert != "" { + args = append(args, "--dest-cert", opts.DestCert) + } + if opts.DestKey != "" { + args = append(args, "--dest-key", opts.DestKey) + } + if opts.DestInsecureTransport { + // Bool flags in cobra must be provided as --flag=false, not as separate args. + args = append(args, "--dest-insecure-transport", "--dest-insecure-transport=false") + } + + args = append(args, destEndpoints) + + proc, err := SpawnCmd(args, nil) + if err != nil { + return err + } + + defer proc.Stop() + + // Wait until context is cancelled or its timeout is reached so that the make-mirror command can keep running in the background. + <-ctx.Done() + + return nil +} diff --git a/tests/framework/integration/integration.go b/tests/framework/integration/integration.go index 9711f38e7bfb..123b6178796b 100644 --- a/tests/framework/integration/integration.go +++ b/tests/framework/integration/integration.go @@ -16,6 +16,7 @@ package integration import ( "context" + "errors" "fmt" "strings" "testing" @@ -475,3 +476,7 @@ func (c integrationClient) MemberList(ctx context.Context, serializable bool) (* } return c.Client.MemberList(ctx) } + +func (c integrationClient) MakeMirror(ctx context.Context, destEndpoints string, opts config.MakeMirrorOptions) error { + return errors.New("errors") +} diff --git a/tests/framework/interfaces/interface.go b/tests/framework/interfaces/interface.go index 0d334011bfbf..3c1f103b0506 100644 --- a/tests/framework/interfaces/interface.go +++ b/tests/framework/interfaces/interface.go @@ -84,6 +84,8 @@ type Client interface { MemberRemove(ctx context.Context, id uint64) (*clientv3.MemberRemoveResponse, error) Watch(ctx context.Context, key string, opts config.WatchOptions) clientv3.WatchChan + + MakeMirror(ctx context.Context, destEndpoints string, opts config.MakeMirrorOptions) error } type TemplateEndpoints interface {