mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00

- most important: unix's socket transport should not keep idle
connections. For top-level Transport we close them using:
f3c518025e/server/etcdserver/api/rafthttp/transport.go (L226)
but currently we don't have access to close them witing the nest (unix) transport. Short idle deadline is good enough.
- Use dialContext (instead of dial) to make sure context is passed down the stack
- Make sure Context is cancelled as soon as the operation is done in pipeline
- nit: use dedicated method to yeld goroutines.
Tested with:
```
d=$(date +"%Y%m%d_%H%M")
(cd tests && go test --timeout=60m ./integration/snapshot -run TestSnapshotV3RestoreMultiMemberAdd -v --count=180 2>&1 | tee log_${d}.log)
```
There were transports & cmux leaked:
```
leak.go:118: Test appears to have leaked a Transport:
internal/poll.runtime_pollWait(0x7f6c5c3784c8, 0x72, 0xffffffffffffffff)
/usr/lib/google-golang/src/runtime/netpoll.go:222 +0x55
internal/poll.(*pollDesc).wait(0xc003296298, 0x72, 0x0, 0x18, 0xffffffffffffffff)
/usr/lib/google-golang/src/internal/poll/fd_poll_runtime.go:87 +0x45
internal/poll.(*pollDesc).waitRead(...)
/usr/lib/google-golang/src/internal/poll/fd_poll_runtime.go:92
internal/poll.(*FD).Read(0xc003296280, 0xc0031f60a8, 0x18, 0x18, 0x0, 0x0, 0x0)
/usr/lib/google-golang/src/internal/poll/fd_unix.go:166 +0x1d5
net.(*netFD).Read(0xc003296280, 0xc0031f60a8, 0x18, 0x18, 0x18, 0xc0009056e2, 0x203000)
/usr/lib/google-golang/src/net/fd_posix.go:55 +0x4f
net.(*conn).Read(0xc000010258, 0xc0031f60a8, 0x18, 0x18, 0x0, 0x0, 0x0)
/usr/lib/google-golang/src/net/net.go:183 +0x91
github.com/soheilhy/cmux.(*bufferedReader).Read(0xc0003d24e0, 0xc0031f60a8, 0x18, 0x18, 0xc0003d24d0, 0xc0009056e2, 0xc000278400)
/home/ptab/private/golang/pkg/mod/github.com/soheilhy/cmux@v0.1.5/buffer.go:53 +0x12d
github.com/soheilhy/cmux.hasHTTP2Preface(0x1367e20, 0xc0003d24e0, 0x7f6c5c699f40)
/home/ptab/private/golang/pkg/mod/github.com/soheilhy/cmux@v0.1.5/matchers.go:195 +0x8a
github.com/soheilhy/cmux.matchersToMatchWriters.func1(0x7f6c5c699f40, 0xc000010258, 0x1367e20, 0xc0003d24e0, 0xc000010258)
/home/ptab/private/golang/pkg/mod/github.com/soheilhy/cmux@v0.1.5/cmux.go:128 +0x39
github.com/soheilhy/cmux.(*cMux).serve(0xc003228690, 0x138c410, 0xc000010258, 0xc00327f740, 0xc0059ba860)
/home/ptab/private/golang/pkg/mod/github.com/soheilhy/cmux@v0.1.5/cmux.go:192 +0x1e7
created by github.com/soheilhy/cmux.(*cMux).Serve
/home/ptab/private/golang/pkg/mod/github.com/soheilhy/cmux@v0.1.5/cmux.go:179 +0x191
internal/poll.runtime_pollWait(0x7f6c5c60f3f0, 0x72, 0xffffffffffffffff)
/usr/lib/google-golang/src/runtime/netpoll.go:222 +0x55
internal/poll.(*pollDesc).wait(0xc000d53018, 0x72, 0x1000, 0x1000, 0xffffffffffffffff)
/usr/lib/google-golang/src/internal/poll/fd_poll_runtime.go:87 +0x45
internal/poll.(*pollDesc).waitRead(...)
/usr/lib/google-golang/src/internal/poll/fd_poll_runtime.go:92
internal/poll.(*FD).Read(0xc000d53000, 0xc000cfd000, 0x1000, 0x1000, 0x0, 0x0, 0x0)
/usr/lib/google-golang/src/internal/poll/fd_unix.go:166 +0x1d5
net.(*netFD).Read(0xc000d53000, 0xc000cfd000, 0x1000, 0x1000, 0x3, 0x3, 0x1000000000001)
/usr/lib/google-golang/src/net/fd_posix.go:55 +0x4f
net.(*conn).Read(0xc00031a570, 0xc000cfd000, 0x1000, 0x1000, 0x0, 0x0, 0x0)
/usr/lib/google-golang/src/net/net.go:183 +0x91
net/http.(*persistConn).Read(0xc00093b320, 0xc000cfd000, 0x1000, 0x1000, 0x577750, 0x60, 0x0)
/usr/lib/google-golang/src/net/http/transport.go:1933 +0x77
bufio.(*Reader).fill(0xc005702fc0)
/usr/lib/google-golang/src/bufio/bufio.go:101 +0x108
bufio.(*Reader).Peek(0xc005702fc0, 0x1, 0xc00077c660, 0xc003b082a0, 0xc000d08de0, 0x5ae586, 0x11dd6c0)
/usr/lib/google-golang/src/bufio/bufio.go:139 +0x4f
net/http.(*persistConn).readLoop(0xc00093b320)
/usr/lib/google-golang/src/net/http/transport.go:2094 +0x1a8
created by net/http.(*Transport).dialConn
/usr/lib/google-golang/src/net/http/transport.go:1754 +0xdaa
net/http.(*persistConn).writeLoop(0xc00093b320)
/usr/lib/google-golang/src/net/http/transport.go:2393 +0xf7
created by net/http.(*Transport).dialConn
/usr/lib/google-golang/src/net/http/transport.go:1755 +0xdcf
sync.runtime_Semacquire(0xc0059ba868)
/usr/lib/google-golang/src/runtime/sema.go:56 +0x45
sync.(*WaitGroup).Wait(0xc0059ba860)
/usr/lib/google-golang/src/sync/waitgroup.go:130 +0x65
github.com/soheilhy/cmux.(*cMux).Serve.func1(0xc003228690, 0xc0059ba860)
/home/ptab/private/golang/pkg/mod/github.com/soheilhy/cmux@v0.1.5/cmux.go:158 +0x56
github.com/soheilhy/cmux.(*cMux).Serve(0xc003228690, 0x13698c0, 0xc00377a0f0)
/home/ptab/private/golang/pkg/mod/github.com/soheilhy/cmux@v0.1.5/cmux.go:173 +0x115
go.etcd.io/etcd/server/v3/embed.(*Etcd).servePeers.func1(0xc0007cc360, 0x122b75f)
/home/ptab/corp/etcd/server/embed/etcd.go:518 +0x2b9
go.etcd.io/etcd/server/v3/embed.(*Etcd).servePeers.func3(0xc00036d080, 0xc0059330a0)
/home/ptab/corp/etcd/server/embed/etcd.go:549 +0x182
created by go.etcd.io/etcd/server/v3/embed.(*Etcd).servePeers
/home/ptab/corp/etcd/server/embed/etcd.go:543 +0x73a
--- FAIL: TestSnapshotV3RestoreMultiMemberAdd (17.74s)
```
179 lines
4.3 KiB
Go
179 lines
4.3 KiB
Go
// Copyright 2015 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 rafthttp
|
|
|
|
import (
|
|
"bytes"
|
|
"context"
|
|
"errors"
|
|
"io/ioutil"
|
|
"runtime"
|
|
"sync"
|
|
"time"
|
|
|
|
"go.etcd.io/etcd/client/pkg/v3/types"
|
|
"go.etcd.io/etcd/pkg/v3/pbutil"
|
|
"go.etcd.io/etcd/raft/v3"
|
|
"go.etcd.io/etcd/raft/v3/raftpb"
|
|
stats "go.etcd.io/etcd/server/v3/etcdserver/api/v2stats"
|
|
|
|
"go.uber.org/zap"
|
|
)
|
|
|
|
const (
|
|
connPerPipeline = 4
|
|
// pipelineBufSize is the size of pipeline buffer, which helps hold the
|
|
// temporary network latency.
|
|
// The size ensures that pipeline does not drop messages when the network
|
|
// is out of work for less than 1 second in good path.
|
|
pipelineBufSize = 64
|
|
)
|
|
|
|
var errStopped = errors.New("stopped")
|
|
|
|
type pipeline struct {
|
|
peerID types.ID
|
|
|
|
tr *Transport
|
|
picker *urlPicker
|
|
status *peerStatus
|
|
raft Raft
|
|
errorc chan error
|
|
// deprecate when we depercate v2 API
|
|
followerStats *stats.FollowerStats
|
|
|
|
msgc chan raftpb.Message
|
|
// wait for the handling routines
|
|
wg sync.WaitGroup
|
|
stopc chan struct{}
|
|
}
|
|
|
|
func (p *pipeline) start() {
|
|
p.stopc = make(chan struct{})
|
|
p.msgc = make(chan raftpb.Message, pipelineBufSize)
|
|
p.wg.Add(connPerPipeline)
|
|
for i := 0; i < connPerPipeline; i++ {
|
|
go p.handle()
|
|
}
|
|
|
|
if p.tr != nil && p.tr.Logger != nil {
|
|
p.tr.Logger.Info(
|
|
"started HTTP pipelining with remote peer",
|
|
zap.String("local-member-id", p.tr.ID.String()),
|
|
zap.String("remote-peer-id", p.peerID.String()),
|
|
)
|
|
}
|
|
}
|
|
|
|
func (p *pipeline) stop() {
|
|
close(p.stopc)
|
|
p.wg.Wait()
|
|
|
|
if p.tr != nil && p.tr.Logger != nil {
|
|
p.tr.Logger.Info(
|
|
"stopped HTTP pipelining with remote peer",
|
|
zap.String("local-member-id", p.tr.ID.String()),
|
|
zap.String("remote-peer-id", p.peerID.String()),
|
|
)
|
|
}
|
|
}
|
|
|
|
func (p *pipeline) handle() {
|
|
defer p.wg.Done()
|
|
|
|
for {
|
|
select {
|
|
case m := <-p.msgc:
|
|
start := time.Now()
|
|
err := p.post(pbutil.MustMarshal(&m))
|
|
end := time.Now()
|
|
|
|
if err != nil {
|
|
p.status.deactivate(failureType{source: pipelineMsg, action: "write"}, err.Error())
|
|
|
|
if m.Type == raftpb.MsgApp && p.followerStats != nil {
|
|
p.followerStats.Fail()
|
|
}
|
|
p.raft.ReportUnreachable(m.To)
|
|
if isMsgSnap(m) {
|
|
p.raft.ReportSnapshot(m.To, raft.SnapshotFailure)
|
|
}
|
|
sentFailures.WithLabelValues(types.ID(m.To).String()).Inc()
|
|
continue
|
|
}
|
|
|
|
p.status.activate()
|
|
if m.Type == raftpb.MsgApp && p.followerStats != nil {
|
|
p.followerStats.Succ(end.Sub(start))
|
|
}
|
|
if isMsgSnap(m) {
|
|
p.raft.ReportSnapshot(m.To, raft.SnapshotFinish)
|
|
}
|
|
sentBytes.WithLabelValues(types.ID(m.To).String()).Add(float64(m.Size()))
|
|
case <-p.stopc:
|
|
return
|
|
}
|
|
}
|
|
}
|
|
|
|
// post POSTs a data payload to a url. Returns nil if the POST succeeds,
|
|
// error on any failure.
|
|
func (p *pipeline) post(data []byte) (err error) {
|
|
u := p.picker.pick()
|
|
req := createPostRequest(p.tr.Logger, u, RaftPrefix, bytes.NewBuffer(data), "application/protobuf", p.tr.URLs, p.tr.ID, p.tr.ClusterID)
|
|
|
|
done := make(chan struct{}, 1)
|
|
ctx, cancel := context.WithCancel(context.Background())
|
|
req = req.WithContext(ctx)
|
|
go func() {
|
|
select {
|
|
case <-done:
|
|
cancel()
|
|
case <-p.stopc:
|
|
waitSchedule()
|
|
cancel()
|
|
}
|
|
}()
|
|
|
|
resp, err := p.tr.pipelineRt.RoundTrip(req)
|
|
done <- struct{}{}
|
|
if err != nil {
|
|
p.picker.unreachable(u)
|
|
return err
|
|
}
|
|
defer resp.Body.Close()
|
|
b, err := ioutil.ReadAll(resp.Body)
|
|
if err != nil {
|
|
p.picker.unreachable(u)
|
|
return err
|
|
}
|
|
|
|
err = checkPostResponse(p.tr.Logger, resp, b, req, p.peerID)
|
|
if err != nil {
|
|
p.picker.unreachable(u)
|
|
// errMemberRemoved is a critical error since a removed member should
|
|
// always be stopped. So we use reportCriticalError to report it to errorc.
|
|
if err == errMemberRemoved {
|
|
reportCriticalError(err, p.errorc)
|
|
}
|
|
return err
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
// waitSchedule waits other goroutines to be scheduled for a while
|
|
func waitSchedule() { runtime.Gosched() }
|