Merge ac95dd739105df58a28a68aa39972ba53d04821e into c86c93ca2951338115159dcdd20711603044e1f1

This commit is contained in:
Fu Wei 2024-09-25 22:31:18 -07:00 committed by GitHub
commit 269b862de5
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
5 changed files with 202 additions and 3 deletions

View File

@ -159,7 +159,7 @@ func (sctx *serveCtx) serve(
defer func(gs *grpc.Server) {
if err != nil {
sctx.lg.Warn("stopping insecure grpc server due to error", zap.Error(err))
gs.Stop()
gs.GracefulStop()
sctx.lg.Warn("stopped insecure grpc server due to error", zap.Error(err))
}
}(gs)
@ -202,16 +202,39 @@ func (sctx *serveCtx) serve(
}
if grpcEnabled {
// TODO(XXX):
//
// WaitForHandlers is experimental function to drain
// all the inflight handlers, including stream RPCs.
// For cmux mode, we can't call GracefulStop because of
// [1].
//
// Actually, we do call http.Shutdown first in stopServers.
// We still need to drain all the inflight handlers to
// make sure that there is no leaky goroutines to
// use closed backend and panic. Add WaitForHandlers
// to force gs.Stop to drain. We can remove this option
// when we remove cmux [2].
//
// [1]: https://github.com/grpc/grpc-go/issues/1384#issuecomment-317124531
// [2]: https://github.com/etcd-io/etcd/issues/15402
gopts = append(gopts, grpc.WaitForHandlers(true))
gs = v3rpc.Server(s, tlscfg, nil, gopts...)
v3electionpb.RegisterElectionServer(gs, servElection)
v3lockpb.RegisterLockServer(gs, servLock)
if sctx.serviceRegister != nil {
sctx.serviceRegister(gs)
}
defer func(gs *grpc.Server) {
if err != nil {
sctx.lg.Warn("stopping secure grpc server due to error", zap.Error(err))
gs.Stop()
if httpEnabled {
gs.Stop()
} else {
gs.GracefulStop()
}
sctx.lg.Warn("stopped secure grpc server due to error", zap.Error(err))
}
}(gs)

View File

@ -122,6 +122,7 @@ func (ms *maintenanceServer) Defragment(ctx context.Context, sr *pb.DefragmentRe
const snapshotSendBufferSize = 32 * 1024
func (ms *maintenanceServer) Snapshot(sr *pb.SnapshotRequest, srv pb.Maintenance_SnapshotServer) error {
// gofail: var v3rpcBeforeSnapshot struct{}
ver := schema.ReadStorageVersion(ms.bg.Backend().ReadTx())
storageVersion := ""
if ver != nil {

View File

@ -118,7 +118,7 @@ func TestAuthGracefulDisable(t *testing.T) {
watchCh := rootAuthClient.Watch(wCtx, "key", config.WatchOptions{Revision: 1})
wantedLen := 1
watchTimeout := 10 * time.Second
watchTimeout := 15 * time.Second
wanted := []testutils.KV{{Key: "key", Val: "value"}}
kvs, err := testutils.KeyValuesFromWatchChan(watchCh, wantedLen, watchTimeout)
require.NoErrorf(t, err, "failed to get key-values from watch channel %s", err)

View File

@ -0,0 +1,171 @@
// Copyright 2024 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.
//go:build !cluster_proxy
package e2e
import (
"context"
"io"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
clientv3 "go.etcd.io/etcd/client/v3"
"go.etcd.io/etcd/tests/v3/framework/e2e"
)
func TestShouldDrainRequestDuringShutdown(t *testing.T) {
e2e.BeforeTest(t)
// defaultBuildSnapshotConn is to setup a database with 10 MiB and a
// inflight snapshot streaming RPC.
defaultBuildSnapshotConn := func(ctx context.Context, t *testing.T, cli *clientv3.Client) io.ReadCloser {
t.Helper()
require.NoError(t, fillEtcdWithData(ctx, cli, 10*1024*1024))
rc, err := cli.Snapshot(ctx)
require.NoError(t, err)
t.Cleanup(func() { rc.Close() })
// make sure that streaming RPC is in progress
buf := make([]byte, 1)
n, err := rc.Read(buf)
assert.NoError(t, err)
assert.Equal(t, 1, n)
return rc
}
// defaultVerifySnapshotConn is to make sure that connection is still
// working even if the server is in shutdown state.
defaultVerifySnapshotConn := func(t *testing.T, rc io.ReadCloser) {
t.Helper()
_, err := io.Copy(io.Discard, rc)
require.NoError(t, err)
}
tcs := []struct {
name string
options []e2e.EPClusterOption
cliOpt e2e.ClientConfig
buildSnapshotConn func(ctx context.Context, t *testing.T, cli *clientv3.Client) io.ReadCloser
verifySnapshotConn func(t *testing.T, rc io.ReadCloser)
}{
{
name: "no-tls",
options: []e2e.EPClusterOption{
e2e.WithClusterSize(1),
e2e.WithClientAutoTLS(false),
},
cliOpt: e2e.ClientConfig{ConnectionType: e2e.ClientNonTLS},
buildSnapshotConn: defaultBuildSnapshotConn,
verifySnapshotConn: defaultVerifySnapshotConn,
},
{
name: "auto-tls_http_separated",
options: []e2e.EPClusterOption{
e2e.WithClusterSize(1),
e2e.WithClientAutoTLS(true),
e2e.WithClientConnType(e2e.ClientTLS),
e2e.WithClientHTTPSeparate(true),
},
cliOpt: e2e.ClientConfig{
ConnectionType: e2e.ClientTLS,
AutoTLS: true,
},
buildSnapshotConn: defaultBuildSnapshotConn,
verifySnapshotConn: defaultVerifySnapshotConn,
},
{
name: "auto-tls_cmux",
options: []e2e.EPClusterOption{
e2e.WithClusterSize(1),
e2e.WithClientAutoTLS(true),
e2e.WithClientConnType(e2e.ClientTLS),
e2e.WithClientHTTPSeparate(false),
e2e.WithGoFailEnabled(true),
// NOTE: Using failpoint is to make sure that
// the RPC handler won't exit because of closed
// connection.
e2e.WithEnvVars(map[string]string{
"GOFAIL_FAILPOINTS": `v3rpcBeforeSnapshot=sleep("8s")`,
}),
},
cliOpt: e2e.ClientConfig{
ConnectionType: e2e.ClientTLS,
AutoTLS: true,
},
buildSnapshotConn: func(ctx context.Context, t *testing.T, cli *clientv3.Client) io.ReadCloser {
t.Helper()
rc, err := cli.Snapshot(ctx)
require.NoError(t, err)
t.Cleanup(func() { rc.Close() })
// make sure server receives the RPC.
time.Sleep(2 * time.Second)
return rc
},
verifySnapshotConn: func(t *testing.T, rc io.ReadCloser) {
t.Helper()
_, err := io.Copy(io.Discard, rc)
require.Error(t, err) // connection will be closed forcely
},
},
}
for _, tc := range tcs {
t.Run(tc.name, func(t *testing.T) {
ctx := context.Background()
epc, err := e2e.NewEtcdProcessCluster(ctx, t, tc.options...)
require.NoError(t, err)
t.Cleanup(func() { epc.Close() })
grpcEndpoint := epc.Procs[0].EndpointsGRPC()[0]
if tc.cliOpt.ConnectionType == e2e.ClientTLS {
grpcEndpoint = e2e.ToTLS(grpcEndpoint)
}
cli := newClient(t, []string{grpcEndpoint}, tc.cliOpt)
rc := tc.buildSnapshotConn(ctx, t, cli)
errCh := make(chan error, 1)
go func() {
defer close(errCh)
errCh <- epc.Stop()
}()
select {
case <-time.After(4 * time.Second):
case err := <-errCh:
t.Fatalf("should drain request but got error from cluster stop: %v", err)
}
tc.verifySnapshotConn(t, rc)
require.NoError(t, <-errCh)
})
}
}

View File

@ -267,6 +267,10 @@ func WithClientAutoTLS(isClientAutoTLS bool) EPClusterOption {
return func(c *EtcdProcessClusterConfig) { c.Client.AutoTLS = isClientAutoTLS }
}
func WithClientHTTPSeparate(separate bool) EPClusterOption {
return func(c *EtcdProcessClusterConfig) { c.ClientHTTPSeparate = separate }
}
func WithClientRevokeCerts(isClientCRL bool) EPClusterOption {
return func(c *EtcdProcessClusterConfig) { c.Client.RevokeCerts = isClientCRL }
}