mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
*: add max requests bytes, keepalive to server, blackhole methods to integration
Signed-off-by: Gyu-Ho Lee <gyuhox@gmail.com>
This commit is contained in:
parent
2a6d50470d
commit
939337f450
@ -21,6 +21,7 @@ import (
|
|||||||
"net/http"
|
"net/http"
|
||||||
"net/url"
|
"net/url"
|
||||||
"strings"
|
"strings"
|
||||||
|
"time"
|
||||||
|
|
||||||
"github.com/coreos/etcd/etcdserver"
|
"github.com/coreos/etcd/etcdserver"
|
||||||
"github.com/coreos/etcd/pkg/cors"
|
"github.com/coreos/etcd/pkg/cors"
|
||||||
@ -37,9 +38,13 @@ const (
|
|||||||
ClusterStateFlagNew = "new"
|
ClusterStateFlagNew = "new"
|
||||||
ClusterStateFlagExisting = "existing"
|
ClusterStateFlagExisting = "existing"
|
||||||
|
|
||||||
DefaultName = "default"
|
DefaultName = "default"
|
||||||
DefaultMaxSnapshots = 5
|
DefaultMaxSnapshots = 5
|
||||||
DefaultMaxWALs = 5
|
DefaultMaxWALs = 5
|
||||||
|
DefaultMaxRequestBytes = 1.5 * 1024 * 1024
|
||||||
|
DefaultGRPCKeepAliveMinTime = 5 * time.Second
|
||||||
|
DefaultGRPCKeepAliveInterval = 2 * time.Hour
|
||||||
|
DefaultGRPCKeepAliveTimeout = 20 * time.Second
|
||||||
|
|
||||||
DefaultListenPeerURLs = "http://localhost:2380"
|
DefaultListenPeerURLs = "http://localhost:2380"
|
||||||
DefaultListenClientURLs = "http://localhost:2379"
|
DefaultListenClientURLs = "http://localhost:2379"
|
||||||
@ -85,6 +90,24 @@ type Config struct {
|
|||||||
TickMs uint `json:"heartbeat-interval"`
|
TickMs uint `json:"heartbeat-interval"`
|
||||||
ElectionMs uint `json:"election-timeout"`
|
ElectionMs uint `json:"election-timeout"`
|
||||||
QuotaBackendBytes int64 `json:"quota-backend-bytes"`
|
QuotaBackendBytes int64 `json:"quota-backend-bytes"`
|
||||||
|
MaxRequestBytes uint `json:"max-request-bytes"`
|
||||||
|
|
||||||
|
// gRPC server options
|
||||||
|
|
||||||
|
// GRPCKeepAliveMinTime is the minimum interval that a client should
|
||||||
|
// wait before pinging server. When client pings "too fast", server
|
||||||
|
// sends goaway and closes the connection (errors: too_many_pings,
|
||||||
|
// http2.ErrCodeEnhanceYourCalm). When too slow, nothing happens.
|
||||||
|
// Server expects client pings only when there is any active streams
|
||||||
|
// (PermitWithoutStream is set false).
|
||||||
|
GRPCKeepAliveMinTime time.Duration `json:"grpc-keepalive-min-time"`
|
||||||
|
// GRPCKeepAliveInterval is the frequency of server-to-client ping
|
||||||
|
// to check if a connection is alive. Close a non-responsive connection
|
||||||
|
// after an additional duration of Timeout. 0 to disable.
|
||||||
|
GRPCKeepAliveInterval time.Duration `json:"grpc-keepalive-interval"`
|
||||||
|
// GRPCKeepAliveTimeout is the additional duration of wait
|
||||||
|
// before closing a non-responsive connection. 0 to disable.
|
||||||
|
GRPCKeepAliveTimeout time.Duration `json:"grpc-keepalive-timeout"`
|
||||||
|
|
||||||
// clustering
|
// clustering
|
||||||
|
|
||||||
@ -167,23 +190,27 @@ func NewConfig() *Config {
|
|||||||
lcurl, _ := url.Parse(DefaultListenClientURLs)
|
lcurl, _ := url.Parse(DefaultListenClientURLs)
|
||||||
acurl, _ := url.Parse(DefaultAdvertiseClientURLs)
|
acurl, _ := url.Parse(DefaultAdvertiseClientURLs)
|
||||||
cfg := &Config{
|
cfg := &Config{
|
||||||
CorsInfo: &cors.CORSInfo{},
|
CorsInfo: &cors.CORSInfo{},
|
||||||
MaxSnapFiles: DefaultMaxSnapshots,
|
MaxSnapFiles: DefaultMaxSnapshots,
|
||||||
MaxWalFiles: DefaultMaxWALs,
|
MaxWalFiles: DefaultMaxWALs,
|
||||||
Name: DefaultName,
|
Name: DefaultName,
|
||||||
SnapCount: etcdserver.DefaultSnapCount,
|
SnapCount: etcdserver.DefaultSnapCount,
|
||||||
TickMs: 100,
|
MaxRequestBytes: DefaultMaxRequestBytes,
|
||||||
ElectionMs: 1000,
|
GRPCKeepAliveMinTime: DefaultGRPCKeepAliveMinTime,
|
||||||
LPUrls: []url.URL{*lpurl},
|
GRPCKeepAliveInterval: DefaultGRPCKeepAliveInterval,
|
||||||
LCUrls: []url.URL{*lcurl},
|
GRPCKeepAliveTimeout: DefaultGRPCKeepAliveTimeout,
|
||||||
APUrls: []url.URL{*apurl},
|
TickMs: 100,
|
||||||
ACUrls: []url.URL{*acurl},
|
ElectionMs: 1000,
|
||||||
ClusterState: ClusterStateFlagNew,
|
LPUrls: []url.URL{*lpurl},
|
||||||
InitialClusterToken: "etcd-cluster",
|
LCUrls: []url.URL{*lcurl},
|
||||||
StrictReconfigCheck: true,
|
APUrls: []url.URL{*apurl},
|
||||||
Metrics: "basic",
|
ACUrls: []url.URL{*acurl},
|
||||||
EnableV2: true,
|
ClusterState: ClusterStateFlagNew,
|
||||||
AuthToken: "simple",
|
InitialClusterToken: "etcd-cluster",
|
||||||
|
StrictReconfigCheck: true,
|
||||||
|
Metrics: "basic",
|
||||||
|
EnableV2: true,
|
||||||
|
AuthToken: "simple",
|
||||||
}
|
}
|
||||||
cfg.InitialCluster = cfg.InitialClusterFromName(cfg.Name)
|
cfg.InitialCluster = cfg.InitialClusterFromName(cfg.Name)
|
||||||
return cfg
|
return cfg
|
||||||
|
@ -36,6 +36,8 @@ import (
|
|||||||
"github.com/coreos/etcd/pkg/types"
|
"github.com/coreos/etcd/pkg/types"
|
||||||
"github.com/coreos/etcd/rafthttp"
|
"github.com/coreos/etcd/rafthttp"
|
||||||
"github.com/coreos/pkg/capnslog"
|
"github.com/coreos/pkg/capnslog"
|
||||||
|
"google.golang.org/grpc"
|
||||||
|
"google.golang.org/grpc/keepalive"
|
||||||
)
|
)
|
||||||
|
|
||||||
var plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "embed")
|
var plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "embed")
|
||||||
@ -140,6 +142,7 @@ func StartEtcd(inCfg *Config) (e *Etcd, err error) {
|
|||||||
ElectionTicks: cfg.ElectionTicks(),
|
ElectionTicks: cfg.ElectionTicks(),
|
||||||
AutoCompactionRetention: cfg.AutoCompactionRetention,
|
AutoCompactionRetention: cfg.AutoCompactionRetention,
|
||||||
QuotaBackendBytes: cfg.QuotaBackendBytes,
|
QuotaBackendBytes: cfg.QuotaBackendBytes,
|
||||||
|
MaxRequestBytes: cfg.MaxRequestBytes,
|
||||||
StrictReconfigCheck: cfg.StrictReconfigCheck,
|
StrictReconfigCheck: cfg.StrictReconfigCheck,
|
||||||
ClientCertAuthEnabled: cfg.ClientTLSInfo.ClientCertAuth,
|
ClientCertAuthEnabled: cfg.ClientTLSInfo.ClientCertAuth,
|
||||||
AuthToken: cfg.AuthToken,
|
AuthToken: cfg.AuthToken,
|
||||||
@ -415,9 +418,23 @@ func (e *Etcd) serve() (err error) {
|
|||||||
}
|
}
|
||||||
h = http.Handler(&cors.CORSHandler{Handler: h, Info: e.cfg.CorsInfo})
|
h = http.Handler(&cors.CORSHandler{Handler: h, Info: e.cfg.CorsInfo})
|
||||||
|
|
||||||
|
gopts := []grpc.ServerOption{}
|
||||||
|
if e.cfg.GRPCKeepAliveMinTime > time.Duration(0) {
|
||||||
|
gopts = append(gopts, grpc.KeepaliveEnforcementPolicy(keepalive.EnforcementPolicy{
|
||||||
|
MinTime: e.cfg.GRPCKeepAliveMinTime,
|
||||||
|
PermitWithoutStream: false,
|
||||||
|
}))
|
||||||
|
}
|
||||||
|
if e.cfg.GRPCKeepAliveInterval > time.Duration(0) &&
|
||||||
|
e.cfg.GRPCKeepAliveTimeout > time.Duration(0) {
|
||||||
|
gopts = append(gopts, grpc.KeepaliveParams(keepalive.ServerParameters{
|
||||||
|
Time: e.cfg.GRPCKeepAliveInterval,
|
||||||
|
Timeout: e.cfg.GRPCKeepAliveTimeout,
|
||||||
|
}))
|
||||||
|
}
|
||||||
for _, sctx := range e.sctxs {
|
for _, sctx := range e.sctxs {
|
||||||
go func(s *serveCtx) {
|
go func(s *serveCtx) {
|
||||||
e.errHandler(s.serve(e.Server, ctlscfg, h, e.errHandler))
|
e.errHandler(s.serve(e.Server, ctlscfg, h, e.errHandler, gopts...))
|
||||||
}(sctx)
|
}(sctx)
|
||||||
}
|
}
|
||||||
return nil
|
return nil
|
||||||
|
@ -66,7 +66,12 @@ func newServeCtx() *serveCtx {
|
|||||||
// serve accepts incoming connections on the listener l,
|
// serve accepts incoming connections on the listener l,
|
||||||
// creating a new service goroutine for each. The service goroutines
|
// creating a new service goroutine for each. The service goroutines
|
||||||
// read requests and then call handler to reply to them.
|
// read requests and then call handler to reply to them.
|
||||||
func (sctx *serveCtx) serve(s *etcdserver.EtcdServer, tlscfg *tls.Config, handler http.Handler, errHandler func(error)) error {
|
func (sctx *serveCtx) serve(
|
||||||
|
s *etcdserver.EtcdServer,
|
||||||
|
tlscfg *tls.Config,
|
||||||
|
handler http.Handler,
|
||||||
|
errHandler func(error),
|
||||||
|
gopts ...grpc.ServerOption) error {
|
||||||
logger := defaultLog.New(ioutil.Discard, "etcdhttp", 0)
|
logger := defaultLog.New(ioutil.Discard, "etcdhttp", 0)
|
||||||
<-s.ReadyNotify()
|
<-s.ReadyNotify()
|
||||||
plog.Info("ready to serve client requests")
|
plog.Info("ready to serve client requests")
|
||||||
@ -77,7 +82,7 @@ func (sctx *serveCtx) serve(s *etcdserver.EtcdServer, tlscfg *tls.Config, handle
|
|||||||
servLock := v3lock.NewLockServer(v3c)
|
servLock := v3lock.NewLockServer(v3c)
|
||||||
|
|
||||||
if sctx.insecure {
|
if sctx.insecure {
|
||||||
gs := v3rpc.Server(s, nil)
|
gs := v3rpc.Server(s, nil, gopts...)
|
||||||
sctx.grpcServerC <- gs
|
sctx.grpcServerC <- gs
|
||||||
v3electionpb.RegisterElectionServer(gs, servElection)
|
v3electionpb.RegisterElectionServer(gs, servElection)
|
||||||
v3lockpb.RegisterLockServer(gs, servLock)
|
v3lockpb.RegisterLockServer(gs, servLock)
|
||||||
@ -107,7 +112,7 @@ func (sctx *serveCtx) serve(s *etcdserver.EtcdServer, tlscfg *tls.Config, handle
|
|||||||
}
|
}
|
||||||
|
|
||||||
if sctx.secure {
|
if sctx.secure {
|
||||||
gs := v3rpc.Server(s, tlscfg)
|
gs := v3rpc.Server(s, tlscfg, gopts...)
|
||||||
sctx.grpcServerC <- gs
|
sctx.grpcServerC <- gs
|
||||||
v3electionpb.RegisterElectionServer(gs, servElection)
|
v3electionpb.RegisterElectionServer(gs, servElection)
|
||||||
v3lockpb.RegisterLockServer(gs, servLock)
|
v3lockpb.RegisterLockServer(gs, servLock)
|
||||||
|
@ -138,6 +138,10 @@ func newConfig() *config {
|
|||||||
fs.UintVar(&cfg.TickMs, "heartbeat-interval", cfg.TickMs, "Time (in milliseconds) of a heartbeat interval.")
|
fs.UintVar(&cfg.TickMs, "heartbeat-interval", cfg.TickMs, "Time (in milliseconds) of a heartbeat interval.")
|
||||||
fs.UintVar(&cfg.ElectionMs, "election-timeout", cfg.ElectionMs, "Time (in milliseconds) for an election to timeout.")
|
fs.UintVar(&cfg.ElectionMs, "election-timeout", cfg.ElectionMs, "Time (in milliseconds) for an election to timeout.")
|
||||||
fs.Int64Var(&cfg.QuotaBackendBytes, "quota-backend-bytes", cfg.QuotaBackendBytes, "Raise alarms when backend size exceeds the given quota. 0 means use the default quota.")
|
fs.Int64Var(&cfg.QuotaBackendBytes, "quota-backend-bytes", cfg.QuotaBackendBytes, "Raise alarms when backend size exceeds the given quota. 0 means use the default quota.")
|
||||||
|
fs.UintVar(&cfg.MaxRequestBytes, "max-request-bytes", cfg.MaxRequestBytes, "Maximum client request size in bytes the server will accept.")
|
||||||
|
fs.DurationVar(&cfg.GRPCKeepAliveMinTime, "grpc-keepalive-min-time", cfg.Config.GRPCKeepAliveMinTime, "Minimum interval duration that a client should wait before pinging server.")
|
||||||
|
fs.DurationVar(&cfg.GRPCKeepAliveInterval, "grpc-keepalive-interval", cfg.Config.GRPCKeepAliveInterval, "Frequency duration of server-to-client ping to check if a connection is alive (0 to disable).")
|
||||||
|
fs.DurationVar(&cfg.GRPCKeepAliveTimeout, "grpc-keepalive-timeout", cfg.Config.GRPCKeepAliveTimeout, "Additional duration of wait before closing a non-responsive connection (0 to disable).")
|
||||||
|
|
||||||
// clustering
|
// clustering
|
||||||
fs.Var(flags.NewURLsValue(embed.DefaultInitialAdvertisePeerURLs), "initial-advertise-peer-urls", "List of this member's peer URLs to advertise to the rest of the cluster.")
|
fs.Var(flags.NewURLsValue(embed.DefaultInitialAdvertisePeerURLs), "initial-advertise-peer-urls", "List of this member's peer URLs to advertise to the rest of the cluster.")
|
||||||
|
@ -66,6 +66,14 @@ member flags:
|
|||||||
comma-separated whitelist of origins for CORS (cross-origin resource sharing).
|
comma-separated whitelist of origins for CORS (cross-origin resource sharing).
|
||||||
--quota-backend-bytes '0'
|
--quota-backend-bytes '0'
|
||||||
raise alarms when backend size exceeds the given quota (0 defaults to low space quota).
|
raise alarms when backend size exceeds the given quota (0 defaults to low space quota).
|
||||||
|
--max-request-bytes '1572864'
|
||||||
|
maximum client request size in bytes the server will accept.
|
||||||
|
--grpc-keepalive-min-time '5s'
|
||||||
|
minimum duration interval that a client should wait before pinging server.
|
||||||
|
--grpc-keepalive-interval '2h'
|
||||||
|
frequency duration of server-to-client ping to check if a connection is alive (0 to disable).
|
||||||
|
--grpc-keepalive-timeout '20s'
|
||||||
|
additional duration of wait before closing a non-responsive connection (0 to disable).
|
||||||
|
|
||||||
clustering flags:
|
clustering flags:
|
||||||
|
|
||||||
|
@ -25,13 +25,17 @@ import (
|
|||||||
"google.golang.org/grpc/grpclog"
|
"google.golang.org/grpc/grpclog"
|
||||||
)
|
)
|
||||||
|
|
||||||
const maxStreams = math.MaxUint32
|
const (
|
||||||
|
grpcOverheadBytes = 512 * 1024
|
||||||
|
maxStreams = math.MaxUint32
|
||||||
|
maxSendBytes = math.MaxInt32
|
||||||
|
)
|
||||||
|
|
||||||
func init() {
|
func init() {
|
||||||
grpclog.SetLogger(plog)
|
grpclog.SetLogger(plog)
|
||||||
}
|
}
|
||||||
|
|
||||||
func Server(s *etcdserver.EtcdServer, tls *tls.Config) *grpc.Server {
|
func Server(s *etcdserver.EtcdServer, tls *tls.Config, gopts ...grpc.ServerOption) *grpc.Server {
|
||||||
var opts []grpc.ServerOption
|
var opts []grpc.ServerOption
|
||||||
opts = append(opts, grpc.CustomCodec(&codec{}))
|
opts = append(opts, grpc.CustomCodec(&codec{}))
|
||||||
if tls != nil {
|
if tls != nil {
|
||||||
@ -39,8 +43,10 @@ func Server(s *etcdserver.EtcdServer, tls *tls.Config) *grpc.Server {
|
|||||||
}
|
}
|
||||||
opts = append(opts, grpc.UnaryInterceptor(newUnaryInterceptor(s)))
|
opts = append(opts, grpc.UnaryInterceptor(newUnaryInterceptor(s)))
|
||||||
opts = append(opts, grpc.StreamInterceptor(newStreamInterceptor(s)))
|
opts = append(opts, grpc.StreamInterceptor(newStreamInterceptor(s)))
|
||||||
|
opts = append(opts, grpc.MaxRecvMsgSize(int(s.Cfg.MaxRequestBytes+grpcOverheadBytes)))
|
||||||
|
opts = append(opts, grpc.MaxSendMsgSize(maxSendBytes))
|
||||||
opts = append(opts, grpc.MaxConcurrentStreams(maxStreams))
|
opts = append(opts, grpc.MaxConcurrentStreams(maxStreams))
|
||||||
grpcServer := grpc.NewServer(opts...)
|
grpcServer := grpc.NewServer(append(opts, gopts...)...)
|
||||||
|
|
||||||
pb.RegisterKVServer(grpcServer, NewQuotaKVServer(s))
|
pb.RegisterKVServer(grpcServer, NewQuotaKVServer(s))
|
||||||
pb.RegisterWatchServer(grpcServer, NewWatchServer(s))
|
pb.RegisterWatchServer(grpcServer, NewWatchServer(s))
|
||||||
|
@ -17,6 +17,7 @@ package rpctypes
|
|||||||
import (
|
import (
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/codes"
|
"google.golang.org/grpc/codes"
|
||||||
|
"google.golang.org/grpc/status"
|
||||||
)
|
)
|
||||||
|
|
||||||
var (
|
var (
|
||||||
@ -188,3 +189,10 @@ func Error(err error) error {
|
|||||||
}
|
}
|
||||||
return EtcdError{code: grpc.Code(verr), desc: grpc.ErrorDesc(verr)}
|
return EtcdError{code: grpc.Code(verr), desc: grpc.ErrorDesc(verr)}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func ErrorDesc(err error) string {
|
||||||
|
if s, ok := status.FromError(err); ok {
|
||||||
|
return s.Message()
|
||||||
|
}
|
||||||
|
return err.Error()
|
||||||
|
}
|
||||||
|
@ -55,6 +55,9 @@ type ServerConfig struct {
|
|||||||
AutoCompactionRetention int
|
AutoCompactionRetention int
|
||||||
QuotaBackendBytes int64
|
QuotaBackendBytes int64
|
||||||
|
|
||||||
|
// MaxRequestBytes is the maximum request size to send over raft.
|
||||||
|
MaxRequestBytes uint
|
||||||
|
|
||||||
StrictReconfigCheck bool
|
StrictReconfigCheck bool
|
||||||
|
|
||||||
// ClientCertAuthEnabled is true when cert has been signed by the client CA.
|
// ClientCertAuthEnabled is true when cert has been signed by the client CA.
|
||||||
|
@ -82,7 +82,8 @@ const (
|
|||||||
releaseDelayAfterSnapshot = 30 * time.Second
|
releaseDelayAfterSnapshot = 30 * time.Second
|
||||||
|
|
||||||
// maxPendingRevokes is the maximum number of outstanding expired lease revocations.
|
// maxPendingRevokes is the maximum number of outstanding expired lease revocations.
|
||||||
maxPendingRevokes = 16
|
maxPendingRevokes = 16
|
||||||
|
recommendedMaxRequestBytes = 10 * 1024 * 1024
|
||||||
)
|
)
|
||||||
|
|
||||||
var (
|
var (
|
||||||
@ -259,6 +260,10 @@ func NewServer(cfg *ServerConfig) (srv *EtcdServer, err error) {
|
|||||||
cl *membership.RaftCluster
|
cl *membership.RaftCluster
|
||||||
)
|
)
|
||||||
|
|
||||||
|
if cfg.MaxRequestBytes > recommendedMaxRequestBytes {
|
||||||
|
plog.Warningf("MaxRequestBytes %v exceeds maximum recommended size %v", cfg.MaxRequestBytes, recommendedMaxRequestBytes)
|
||||||
|
}
|
||||||
|
|
||||||
if terr := fileutil.TouchDirAll(cfg.DataDir); terr != nil {
|
if terr := fileutil.TouchDirAll(cfg.DataDir); terr != nil {
|
||||||
return nil, fmt.Errorf("cannot access data directory: %v", terr)
|
return nil, fmt.Errorf("cannot access data directory: %v", terr)
|
||||||
}
|
}
|
||||||
|
@ -33,12 +33,6 @@ import (
|
|||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
// the max request size that raft accepts.
|
|
||||||
// TODO: make this a flag? But we probably do not want to
|
|
||||||
// accept large request which might block raft stream. User
|
|
||||||
// specify a large value might end up with shooting in the foot.
|
|
||||||
maxRequestBytes = 1.5 * 1024 * 1024
|
|
||||||
|
|
||||||
// In the health case, there might be a small gap (10s of entries) between
|
// In the health case, there might be a small gap (10s of entries) between
|
||||||
// the applied index and committed index.
|
// the applied index and committed index.
|
||||||
// However, if the committed entries are very heavy to apply, the gap might grow.
|
// However, if the committed entries are very heavy to apply, the gap might grow.
|
||||||
@ -556,7 +550,7 @@ func (s *EtcdServer) processInternalRaftRequestOnce(ctx context.Context, r pb.In
|
|||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
if len(data) > maxRequestBytes {
|
if len(data) > int(s.Cfg.MaxRequestBytes) {
|
||||||
return nil, ErrRequestTooLarge
|
return nil, ErrRequestTooLarge
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -17,6 +17,7 @@ package integration
|
|||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"io"
|
"io"
|
||||||
|
"io/ioutil"
|
||||||
"net"
|
"net"
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
@ -31,9 +32,10 @@ type bridge struct {
|
|||||||
l net.Listener
|
l net.Listener
|
||||||
conns map[*bridgeConn]struct{}
|
conns map[*bridgeConn]struct{}
|
||||||
|
|
||||||
stopc chan struct{}
|
stopc chan struct{}
|
||||||
pausec chan struct{}
|
pausec chan struct{}
|
||||||
wg sync.WaitGroup
|
blackholec chan struct{}
|
||||||
|
wg sync.WaitGroup
|
||||||
|
|
||||||
mu sync.Mutex
|
mu sync.Mutex
|
||||||
}
|
}
|
||||||
@ -41,11 +43,12 @@ type bridge struct {
|
|||||||
func newBridge(addr string) (*bridge, error) {
|
func newBridge(addr string) (*bridge, error) {
|
||||||
b := &bridge{
|
b := &bridge{
|
||||||
// bridge "port" is ("%05d%05d0", port, pid) since go1.8 expects the port to be a number
|
// bridge "port" is ("%05d%05d0", port, pid) since go1.8 expects the port to be a number
|
||||||
inaddr: addr + "0",
|
inaddr: addr + "0",
|
||||||
outaddr: addr,
|
outaddr: addr,
|
||||||
conns: make(map[*bridgeConn]struct{}),
|
conns: make(map[*bridgeConn]struct{}),
|
||||||
stopc: make(chan struct{}),
|
stopc: make(chan struct{}),
|
||||||
pausec: make(chan struct{}),
|
pausec: make(chan struct{}),
|
||||||
|
blackholec: make(chan struct{}),
|
||||||
}
|
}
|
||||||
close(b.pausec)
|
close(b.pausec)
|
||||||
|
|
||||||
@ -152,12 +155,12 @@ func (b *bridge) serveConn(bc *bridgeConn) {
|
|||||||
var wg sync.WaitGroup
|
var wg sync.WaitGroup
|
||||||
wg.Add(2)
|
wg.Add(2)
|
||||||
go func() {
|
go func() {
|
||||||
io.Copy(bc.out, bc.in)
|
b.ioCopy(bc, bc.out, bc.in)
|
||||||
bc.close()
|
bc.close()
|
||||||
wg.Done()
|
wg.Done()
|
||||||
}()
|
}()
|
||||||
go func() {
|
go func() {
|
||||||
io.Copy(bc.in, bc.out)
|
b.ioCopy(bc, bc.in, bc.out)
|
||||||
bc.close()
|
bc.close()
|
||||||
wg.Done()
|
wg.Done()
|
||||||
}()
|
}()
|
||||||
@ -179,3 +182,47 @@ func (bc *bridgeConn) close() {
|
|||||||
bc.in.Close()
|
bc.in.Close()
|
||||||
bc.out.Close()
|
bc.out.Close()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (b *bridge) Blackhole() {
|
||||||
|
b.mu.Lock()
|
||||||
|
close(b.blackholec)
|
||||||
|
b.mu.Unlock()
|
||||||
|
}
|
||||||
|
|
||||||
|
func (b *bridge) Unblackhole() {
|
||||||
|
b.mu.Lock()
|
||||||
|
for bc := range b.conns {
|
||||||
|
bc.Close()
|
||||||
|
}
|
||||||
|
b.conns = make(map[*bridgeConn]struct{})
|
||||||
|
b.blackholec = make(chan struct{})
|
||||||
|
b.mu.Unlock()
|
||||||
|
}
|
||||||
|
|
||||||
|
// ref. https://github.com/golang/go/blob/master/src/io/io.go copyBuffer
|
||||||
|
func (b *bridge) ioCopy(bc *bridgeConn, dst io.Writer, src io.Reader) (err error) {
|
||||||
|
buf := make([]byte, 32*1024)
|
||||||
|
for {
|
||||||
|
select {
|
||||||
|
case <-b.blackholec:
|
||||||
|
io.Copy(ioutil.Discard, src)
|
||||||
|
return nil
|
||||||
|
default:
|
||||||
|
}
|
||||||
|
nr, er := src.Read(buf)
|
||||||
|
if nr > 0 {
|
||||||
|
nw, ew := dst.Write(buf[0:nr])
|
||||||
|
if ew != nil {
|
||||||
|
return ew
|
||||||
|
}
|
||||||
|
if nr != nw {
|
||||||
|
return io.ErrShortWrite
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if er != nil {
|
||||||
|
err = er
|
||||||
|
break
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return
|
||||||
|
}
|
||||||
|
@ -31,11 +31,9 @@ import (
|
|||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"golang.org/x/net/context"
|
|
||||||
"google.golang.org/grpc"
|
|
||||||
|
|
||||||
"github.com/coreos/etcd/client"
|
"github.com/coreos/etcd/client"
|
||||||
"github.com/coreos/etcd/clientv3"
|
"github.com/coreos/etcd/clientv3"
|
||||||
|
"github.com/coreos/etcd/embed"
|
||||||
"github.com/coreos/etcd/etcdserver"
|
"github.com/coreos/etcd/etcdserver"
|
||||||
"github.com/coreos/etcd/etcdserver/api/etcdhttp"
|
"github.com/coreos/etcd/etcdserver/api/etcdhttp"
|
||||||
"github.com/coreos/etcd/etcdserver/api/v2http"
|
"github.com/coreos/etcd/etcdserver/api/v2http"
|
||||||
@ -50,7 +48,11 @@ import (
|
|||||||
"github.com/coreos/etcd/pkg/transport"
|
"github.com/coreos/etcd/pkg/transport"
|
||||||
"github.com/coreos/etcd/pkg/types"
|
"github.com/coreos/etcd/pkg/types"
|
||||||
"github.com/coreos/etcd/rafthttp"
|
"github.com/coreos/etcd/rafthttp"
|
||||||
|
|
||||||
"github.com/coreos/pkg/capnslog"
|
"github.com/coreos/pkg/capnslog"
|
||||||
|
"golang.org/x/net/context"
|
||||||
|
"google.golang.org/grpc"
|
||||||
|
"google.golang.org/grpc/keepalive"
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
@ -88,12 +90,18 @@ var (
|
|||||||
)
|
)
|
||||||
|
|
||||||
type ClusterConfig struct {
|
type ClusterConfig struct {
|
||||||
Size int
|
Size int
|
||||||
PeerTLS *transport.TLSInfo
|
PeerTLS *transport.TLSInfo
|
||||||
ClientTLS *transport.TLSInfo
|
ClientTLS *transport.TLSInfo
|
||||||
DiscoveryURL string
|
DiscoveryURL string
|
||||||
UseGRPC bool
|
UseGRPC bool
|
||||||
QuotaBackendBytes int64
|
QuotaBackendBytes int64
|
||||||
|
MaxRequestBytes uint
|
||||||
|
GRPCKeepAliveMinTime time.Duration
|
||||||
|
GRPCKeepAliveInterval time.Duration
|
||||||
|
GRPCKeepAliveTimeout time.Duration
|
||||||
|
// SkipCreatingClient to skip creating clients for each member.
|
||||||
|
SkipCreatingClient bool
|
||||||
}
|
}
|
||||||
|
|
||||||
type cluster struct {
|
type cluster struct {
|
||||||
@ -221,10 +229,14 @@ func (c *cluster) HTTPMembers() []client.Member {
|
|||||||
func (c *cluster) mustNewMember(t *testing.T) *member {
|
func (c *cluster) mustNewMember(t *testing.T) *member {
|
||||||
m := mustNewMember(t,
|
m := mustNewMember(t,
|
||||||
memberConfig{
|
memberConfig{
|
||||||
name: c.name(rand.Int()),
|
name: c.name(rand.Int()),
|
||||||
peerTLS: c.cfg.PeerTLS,
|
peerTLS: c.cfg.PeerTLS,
|
||||||
clientTLS: c.cfg.ClientTLS,
|
clientTLS: c.cfg.ClientTLS,
|
||||||
quotaBackendBytes: c.cfg.QuotaBackendBytes,
|
quotaBackendBytes: c.cfg.QuotaBackendBytes,
|
||||||
|
maxRequestBytes: c.cfg.MaxRequestBytes,
|
||||||
|
grpcKeepAliveMinTime: c.cfg.GRPCKeepAliveMinTime,
|
||||||
|
grpcKeepAliveInterval: c.cfg.GRPCKeepAliveInterval,
|
||||||
|
grpcKeepAliveTimeout: c.cfg.GRPCKeepAliveTimeout,
|
||||||
})
|
})
|
||||||
m.DiscoveryURL = c.cfg.DiscoveryURL
|
m.DiscoveryURL = c.cfg.DiscoveryURL
|
||||||
if c.cfg.UseGRPC {
|
if c.cfg.UseGRPC {
|
||||||
@ -474,9 +486,10 @@ type member struct {
|
|||||||
s *etcdserver.EtcdServer
|
s *etcdserver.EtcdServer
|
||||||
hss []*httptest.Server
|
hss []*httptest.Server
|
||||||
|
|
||||||
grpcServer *grpc.Server
|
grpcServerOpts []grpc.ServerOption
|
||||||
grpcAddr string
|
grpcServer *grpc.Server
|
||||||
grpcBridge *bridge
|
grpcAddr string
|
||||||
|
grpcBridge *bridge
|
||||||
|
|
||||||
// serverClient is a clientv3 that directly calls the etcdserver.
|
// serverClient is a clientv3 that directly calls the etcdserver.
|
||||||
serverClient *clientv3.Client
|
serverClient *clientv3.Client
|
||||||
@ -487,10 +500,14 @@ type member struct {
|
|||||||
func (m *member) GRPCAddr() string { return m.grpcAddr }
|
func (m *member) GRPCAddr() string { return m.grpcAddr }
|
||||||
|
|
||||||
type memberConfig struct {
|
type memberConfig struct {
|
||||||
name string
|
name string
|
||||||
peerTLS *transport.TLSInfo
|
peerTLS *transport.TLSInfo
|
||||||
clientTLS *transport.TLSInfo
|
clientTLS *transport.TLSInfo
|
||||||
quotaBackendBytes int64
|
quotaBackendBytes int64
|
||||||
|
maxRequestBytes uint
|
||||||
|
grpcKeepAliveMinTime time.Duration
|
||||||
|
grpcKeepAliveInterval time.Duration
|
||||||
|
grpcKeepAliveTimeout time.Duration
|
||||||
}
|
}
|
||||||
|
|
||||||
// mustNewMember return an inited member with the given name. If peerTLS is
|
// mustNewMember return an inited member with the given name. If peerTLS is
|
||||||
@ -538,7 +555,26 @@ func mustNewMember(t *testing.T, mcfg memberConfig) *member {
|
|||||||
m.ElectionTicks = electionTicks
|
m.ElectionTicks = electionTicks
|
||||||
m.TickMs = uint(tickDuration / time.Millisecond)
|
m.TickMs = uint(tickDuration / time.Millisecond)
|
||||||
m.QuotaBackendBytes = mcfg.quotaBackendBytes
|
m.QuotaBackendBytes = mcfg.quotaBackendBytes
|
||||||
|
m.MaxRequestBytes = mcfg.maxRequestBytes
|
||||||
|
if m.MaxRequestBytes == 0 {
|
||||||
|
m.MaxRequestBytes = embed.DefaultMaxRequestBytes
|
||||||
|
}
|
||||||
m.AuthToken = "simple" // for the purpose of integration testing, simple token is enough
|
m.AuthToken = "simple" // for the purpose of integration testing, simple token is enough
|
||||||
|
|
||||||
|
m.grpcServerOpts = []grpc.ServerOption{}
|
||||||
|
if mcfg.grpcKeepAliveMinTime > time.Duration(0) {
|
||||||
|
m.grpcServerOpts = append(m.grpcServerOpts, grpc.KeepaliveEnforcementPolicy(keepalive.EnforcementPolicy{
|
||||||
|
MinTime: mcfg.grpcKeepAliveMinTime,
|
||||||
|
PermitWithoutStream: false,
|
||||||
|
}))
|
||||||
|
}
|
||||||
|
if mcfg.grpcKeepAliveInterval > time.Duration(0) &&
|
||||||
|
mcfg.grpcKeepAliveTimeout > time.Duration(0) {
|
||||||
|
m.grpcServerOpts = append(m.grpcServerOpts, grpc.KeepaliveParams(keepalive.ServerParameters{
|
||||||
|
Time: mcfg.grpcKeepAliveInterval,
|
||||||
|
Timeout: mcfg.grpcKeepAliveTimeout,
|
||||||
|
}))
|
||||||
|
}
|
||||||
return m
|
return m
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -567,6 +603,8 @@ func (m *member) electionTimeout() time.Duration {
|
|||||||
func (m *member) DropConnections() { m.grpcBridge.Reset() }
|
func (m *member) DropConnections() { m.grpcBridge.Reset() }
|
||||||
func (m *member) PauseConnections() { m.grpcBridge.Pause() }
|
func (m *member) PauseConnections() { m.grpcBridge.Pause() }
|
||||||
func (m *member) UnpauseConnections() { m.grpcBridge.Unpause() }
|
func (m *member) UnpauseConnections() { m.grpcBridge.Unpause() }
|
||||||
|
func (m *member) Blackhole() { m.grpcBridge.Blackhole() }
|
||||||
|
func (m *member) Unblackhole() { m.grpcBridge.Unblackhole() }
|
||||||
|
|
||||||
// NewClientV3 creates a new grpc client connection to the member
|
// NewClientV3 creates a new grpc client connection to the member
|
||||||
func NewClientV3(m *member) (*clientv3.Client, error) {
|
func NewClientV3(m *member) (*clientv3.Client, error) {
|
||||||
@ -676,7 +714,7 @@ func (m *member) Launch() error {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
m.grpcServer = v3rpc.Server(m.s, tlscfg)
|
m.grpcServer = v3rpc.Server(m.s, tlscfg, m.grpcServerOpts...)
|
||||||
m.serverClient = v3client.New(m.s)
|
m.serverClient = v3client.New(m.s)
|
||||||
lockpb.RegisterLockServer(m.grpcServer, v3lock.NewLockServer(m.serverClient))
|
lockpb.RegisterLockServer(m.grpcServer, v3lock.NewLockServer(m.serverClient))
|
||||||
epb.RegisterElectionServer(m.grpcServer, v3election.NewElectionServer(m.serverClient))
|
epb.RegisterElectionServer(m.grpcServer, v3election.NewElectionServer(m.serverClient))
|
||||||
@ -824,7 +862,7 @@ func (m *member) Metric(metricName string) (string, error) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// InjectPartition drops connections from m to others, vice versa.
|
// InjectPartition drops connections from m to others, vice versa.
|
||||||
func (m *member) InjectPartition(t *testing.T, others []*member) {
|
func (m *member) InjectPartition(t *testing.T, others ...*member) {
|
||||||
for _, other := range others {
|
for _, other := range others {
|
||||||
m.s.CutPeer(other.s.ID())
|
m.s.CutPeer(other.s.ID())
|
||||||
other.s.CutPeer(m.s.ID())
|
other.s.CutPeer(m.s.ID())
|
||||||
@ -832,7 +870,7 @@ func (m *member) InjectPartition(t *testing.T, others []*member) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// RecoverPartition recovers connections from m to others, vice versa.
|
// RecoverPartition recovers connections from m to others, vice versa.
|
||||||
func (m *member) RecoverPartition(t *testing.T, others []*member) {
|
func (m *member) RecoverPartition(t *testing.T, others ...*member) {
|
||||||
for _, other := range others {
|
for _, other := range others {
|
||||||
m.s.MendPeer(other.s.ID())
|
m.s.MendPeer(other.s.ID())
|
||||||
other.s.MendPeer(m.s.ID())
|
other.s.MendPeer(m.s.ID())
|
||||||
@ -884,12 +922,15 @@ func NewClusterV3(t *testing.T, cfg *ClusterConfig) *ClusterV3 {
|
|||||||
cluster: NewClusterByConfig(t, cfg),
|
cluster: NewClusterByConfig(t, cfg),
|
||||||
}
|
}
|
||||||
clus.Launch(t)
|
clus.Launch(t)
|
||||||
for _, m := range clus.Members {
|
|
||||||
client, err := NewClientV3(m)
|
if !cfg.SkipCreatingClient {
|
||||||
if err != nil {
|
for _, m := range clus.Members {
|
||||||
t.Fatalf("cannot create client: %v", err)
|
client, err := NewClientV3(m)
|
||||||
|
if err != nil {
|
||||||
|
t.Fatalf("cannot create client: %v", err)
|
||||||
|
}
|
||||||
|
clus.clients = append(clus.clients, client)
|
||||||
}
|
}
|
||||||
clus.clients = append(clus.clients, client)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return clus
|
return clus
|
||||||
|
@ -149,12 +149,12 @@ func getMembersByIndexSlice(clus *cluster, idxs []int) []*member {
|
|||||||
|
|
||||||
func injectPartition(t *testing.T, src, others []*member) {
|
func injectPartition(t *testing.T, src, others []*member) {
|
||||||
for _, m := range src {
|
for _, m := range src {
|
||||||
m.InjectPartition(t, others)
|
m.InjectPartition(t, others...)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func recoverPartition(t *testing.T, src, others []*member) {
|
func recoverPartition(t *testing.T, src, others []*member) {
|
||||||
for _, m := range src {
|
for _, m := range src {
|
||||||
m.RecoverPartition(t, others)
|
m.RecoverPartition(t, others...)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1372,7 +1372,7 @@ func TestTLSGRPCRejectInsecureClient(t *testing.T) {
|
|||||||
// nil out TLS field so client will use an insecure connection
|
// nil out TLS field so client will use an insecure connection
|
||||||
clus.Members[0].ClientTLSInfo = nil
|
clus.Members[0].ClientTLSInfo = nil
|
||||||
client, err := NewClientV3(clus.Members[0])
|
client, err := NewClientV3(clus.Members[0])
|
||||||
if err != nil && err != grpc.ErrClientConnTimeout {
|
if err != nil && err != context.DeadlineExceeded {
|
||||||
t.Fatalf("unexpected error (%v)", err)
|
t.Fatalf("unexpected error (%v)", err)
|
||||||
} else if client == nil {
|
} else if client == nil {
|
||||||
// Ideally, no client would be returned. However, grpc will
|
// Ideally, no client would be returned. However, grpc will
|
||||||
@ -1408,7 +1408,7 @@ func TestTLSGRPCRejectSecureClient(t *testing.T) {
|
|||||||
client, err := NewClientV3(clus.Members[0])
|
client, err := NewClientV3(clus.Members[0])
|
||||||
if client != nil || err == nil {
|
if client != nil || err == nil {
|
||||||
t.Fatalf("expected no client")
|
t.Fatalf("expected no client")
|
||||||
} else if err != grpc.ErrClientConnTimeout {
|
} else if err != context.DeadlineExceeded {
|
||||||
t.Fatalf("unexpected error (%v)", err)
|
t.Fatalf("unexpected error (%v)", err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -1565,8 +1565,8 @@ func testTLSReload(t *testing.T, cloneFunc func() transport.TLSInfo, replaceFunc
|
|||||||
// 5. expect dial time-out when loading expired certs
|
// 5. expect dial time-out when loading expired certs
|
||||||
select {
|
select {
|
||||||
case gerr := <-errc:
|
case gerr := <-errc:
|
||||||
if gerr != grpc.ErrClientConnTimeout {
|
if gerr != context.DeadlineExceeded {
|
||||||
t.Fatalf("expected %v, got %v", grpc.ErrClientConnTimeout, gerr)
|
t.Fatalf("expected %v, got %v", context.DeadlineExceeded, gerr)
|
||||||
}
|
}
|
||||||
case <-time.After(5 * time.Second):
|
case <-time.After(5 * time.Second):
|
||||||
t.Fatal("failed to receive dial timeout error")
|
t.Fatal("failed to receive dial timeout error")
|
||||||
@ -1611,7 +1611,7 @@ func TestGRPCRequireLeader(t *testing.T) {
|
|||||||
time.Sleep(time.Duration(3*electionTicks) * tickDuration)
|
time.Sleep(time.Duration(3*electionTicks) * tickDuration)
|
||||||
|
|
||||||
md := metadata.Pairs(rpctypes.MetadataRequireLeaderKey, rpctypes.MetadataHasLeader)
|
md := metadata.Pairs(rpctypes.MetadataRequireLeaderKey, rpctypes.MetadataHasLeader)
|
||||||
ctx := metadata.NewContext(context.Background(), md)
|
ctx := metadata.NewOutgoingContext(context.Background(), md)
|
||||||
reqput := &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}
|
reqput := &pb.PutRequest{Key: []byte("foo"), Value: []byte("bar")}
|
||||||
if _, err := toGRPC(client).KV.Put(ctx, reqput); grpc.ErrorDesc(err) != rpctypes.ErrNoLeader.Error() {
|
if _, err := toGRPC(client).KV.Put(ctx, reqput); grpc.ErrorDesc(err) != rpctypes.ErrNoLeader.Error() {
|
||||||
t.Errorf("err = %v, want %v", err, rpctypes.ErrNoLeader)
|
t.Errorf("err = %v, want %v", err, rpctypes.ErrNoLeader)
|
||||||
@ -1633,7 +1633,7 @@ func TestGRPCStreamRequireLeader(t *testing.T) {
|
|||||||
|
|
||||||
wAPI := toGRPC(client).Watch
|
wAPI := toGRPC(client).Watch
|
||||||
md := metadata.Pairs(rpctypes.MetadataRequireLeaderKey, rpctypes.MetadataHasLeader)
|
md := metadata.Pairs(rpctypes.MetadataRequireLeaderKey, rpctypes.MetadataHasLeader)
|
||||||
ctx := metadata.NewContext(context.Background(), md)
|
ctx := metadata.NewOutgoingContext(context.Background(), md)
|
||||||
wStream, err := wAPI.Watch(ctx)
|
wStream, err := wAPI.Watch(ctx)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
t.Fatalf("wAPI.Watch error: %v", err)
|
t.Fatalf("wAPI.Watch error: %v", err)
|
||||||
@ -1680,6 +1680,35 @@ func TestGRPCStreamRequireLeader(t *testing.T) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// TestV3PutLargeRequests ensures that configurable MaxRequestBytes works as intended.
|
||||||
|
func TestV3PutLargeRequests(t *testing.T) {
|
||||||
|
defer testutil.AfterTest(t)
|
||||||
|
tests := []struct {
|
||||||
|
key string
|
||||||
|
maxRequestBytes uint
|
||||||
|
valueSize int
|
||||||
|
expectError error
|
||||||
|
}{
|
||||||
|
// don't set to 0. use 0 as the default.
|
||||||
|
{"foo", 1, 1024, rpctypes.ErrGRPCRequestTooLarge},
|
||||||
|
{"foo", 10 * 1024 * 1024, 9 * 1024 * 1024, nil},
|
||||||
|
{"foo", 10 * 1024 * 1024, 10 * 1024 * 1024, rpctypes.ErrGRPCRequestTooLarge},
|
||||||
|
{"foo", 10 * 1024 * 1024, 10*1024*1024 + 5, rpctypes.ErrGRPCRequestTooLarge},
|
||||||
|
}
|
||||||
|
for i, test := range tests {
|
||||||
|
clus := NewClusterV3(t, &ClusterConfig{Size: 1, MaxRequestBytes: test.maxRequestBytes})
|
||||||
|
kvcli := toGRPC(clus.Client(0)).KV
|
||||||
|
reqput := &pb.PutRequest{Key: []byte(test.key), Value: make([]byte, test.valueSize)}
|
||||||
|
_, err := kvcli.Put(context.TODO(), reqput)
|
||||||
|
|
||||||
|
if !eqErrGRPC(err, test.expectError) {
|
||||||
|
t.Errorf("#%d: expected error %v, got %v", i, test.expectError, err)
|
||||||
|
}
|
||||||
|
|
||||||
|
clus.Terminate(t)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
func eqErrGRPC(err1 error, err2 error) bool {
|
func eqErrGRPC(err1 error, err2 error) bool {
|
||||||
return !(err1 == nil && err2 != nil) || err1.Error() == err2.Error()
|
return !(err1 == nil && err2 != nil) || err1.Error() == err2.Error()
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user