Skip to content

Commit 21d2590

Browse files
hicqutiancaiamao
authored andcommitted
tikvclient: fix a bug that double close channels. (#10991)
1 parent 407e50d commit 21d2590

File tree

8 files changed

+239
-55
lines changed

8 files changed

+239
-55
lines changed

go.mod

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -33,15 +33,13 @@ require (
3333
github.com/myesui/uuid v1.0.0 // indirect
3434
github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7
3535
github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef
36-
github.com/onsi/ginkgo v1.7.0 // indirect
37-
github.com/onsi/gomega v1.4.3 // indirect
3836
github.com/opentracing/basictracer-go v1.0.0
3937
github.com/opentracing/opentracing-go v1.0.2
4038
github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8
4139
github.com/pingcap/errors v0.11.4
4240
github.com/pingcap/failpoint v0.0.0-20190512135322-30cc7431d99c
4341
github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e
44-
github.com/pingcap/kvproto v0.0.0-20190619024611-a4759dfe3753
42+
github.com/pingcap/kvproto v0.0.0-20190703131923-d9830856b531
4543
github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596
4644
github.com/pingcap/parser v0.0.0-20190701123046-5768e68c1e65
4745
github.com/pingcap/pd v0.0.0-20190617100349-293d4b5189bf

go.sum

Lines changed: 4 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -137,12 +137,10 @@ github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef h1:K0Fn+DoFqNqktdZtdV3
137137
github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef/go.mod h1:7WjlapSfwQyo6LNmIvEWzsW1hbBQfpUO4JWnuQRmva8=
138138
github.com/nicksnyder/go-i18n v1.10.0/go.mod h1:HrK7VCrbOvQoUAQ7Vpy7i87N7JZZZ7R2xBGjv0j365Q=
139139
github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo=
140+
github.com/onsi/ginkgo v1.6.0 h1:Ix8l273rp3QzYgXSR+c8d1fTG7UPgYkOSELPhiY/YGw=
140141
github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
141-
github.com/onsi/ginkgo v1.7.0 h1:WSHQ+IS43OoUrWtD1/bbclrwK8TTH5hzp+umCiuxHgs=
142-
github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
142+
github.com/onsi/gomega v1.4.2 h1:3mYCb7aPxS/RU7TI1y4rkEn1oKmPRjNJLNEXgw7MH2I=
143143
github.com/onsi/gomega v1.4.2/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
144-
github.com/onsi/gomega v1.4.3 h1:RE1xgDvH7imwFD45h+u2SgIfERHlS2yNG4DObb5BSKU=
145-
github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
146144
github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo=
147145
github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74=
148146
github.com/opentracing/opentracing-go v1.0.2 h1:3jA2P6O1F9UOrWVpwrIo17pu01KWvNWg4X946/Y5Zwg=
@@ -161,8 +159,8 @@ github.com/pingcap/failpoint v0.0.0-20190512135322-30cc7431d99c/go.mod h1:DNS3Qg
161159
github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e h1:P73/4dPCL96rGrobssy1nVy2VaVpNCuLpCbr+FEaTA8=
162160
github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw=
163161
github.com/pingcap/kvproto v0.0.0-20190516013202-4cf58ad90b6c/go.mod h1:QMdbTAXCHzzygQzqcG9uVUgU2fKeSN1GmfMiykdSzzY=
164-
github.com/pingcap/kvproto v0.0.0-20190619024611-a4759dfe3753 h1:92t0y430CJF0tN1lvUhP5fhnYTFmssATJqwxQtvixYU=
165-
github.com/pingcap/kvproto v0.0.0-20190619024611-a4759dfe3753/go.mod h1:QMdbTAXCHzzygQzqcG9uVUgU2fKeSN1GmfMiykdSzzY=
162+
github.com/pingcap/kvproto v0.0.0-20190703131923-d9830856b531 h1:8xk2HobDwClB5E3Hv9TEPiS7K7bv3ykWHLyZzuUYywI=
163+
github.com/pingcap/kvproto v0.0.0-20190703131923-d9830856b531/go.mod h1:QMdbTAXCHzzygQzqcG9uVUgU2fKeSN1GmfMiykdSzzY=
166164
github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w=
167165
github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ=
168166
github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw=

metrics/tikvclient.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -180,13 +180,13 @@ var (
180180
})
181181

182182
// TiKVPendingBatchRequests indicates the number of requests pending in the batch channel.
183-
TiKVPendingBatchRequests = prometheus.NewGauge(
183+
TiKVPendingBatchRequests = prometheus.NewGaugeVec(
184184
prometheus.GaugeOpts{
185185
Namespace: "tidb",
186186
Subsystem: "tikvclient",
187187
Name: "pending_batch_requests",
188188
Help: "Pending batch requests",
189-
})
189+
}, []string{"store"})
190190

191191
TiKVBatchWaitDuration = prometheus.NewHistogram(
192192
prometheus.HistogramOpts{

store/tikv/client.go

Lines changed: 78 additions & 42 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ import (
2525

2626
"github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
2727
"github.com/pingcap/errors"
28+
"github.com/pingcap/failpoint"
2829
"github.com/pingcap/kvproto/pkg/coprocessor"
2930
"github.com/pingcap/kvproto/pkg/debugpb"
3031
"github.com/pingcap/kvproto/pkg/tikvpb"
@@ -33,6 +34,7 @@ import (
3334
"github.com/pingcap/tidb/metrics"
3435
"github.com/pingcap/tidb/store/tikv/tikvrpc"
3536
"github.com/pingcap/tidb/util/logutil"
37+
"github.com/prometheus/client_golang/prometheus"
3638
"go.uber.org/zap"
3739
"google.golang.org/grpc"
3840
"google.golang.org/grpc/credentials"
@@ -78,11 +80,14 @@ type connArray struct {
7880
batchCommandsCh chan *batchCommandsEntry
7981
batchCommandsClients []*batchCommandsClient
8082
tikvTransportLayerLoad uint64
83+
closed chan struct{}
8184

8285
// Notify rpcClient to check the idle flag
8386
idleNotify *uint32
8487
idle bool
8588
idleDetect *time.Timer
89+
90+
pendingRequests prometheus.Gauge
8691
}
8792

8893
type batchCommandsClient struct {
@@ -105,17 +110,70 @@ func (c *batchCommandsClient) isStopped() bool {
105110
return atomic.LoadInt32(&c.closed) != 0
106111
}
107112

113+
func (c *batchCommandsClient) send(request *tikvpb.BatchCommandsRequest, entries []*batchCommandsEntry) {
114+
// Use the lock to protect the stream client won't be replaced by RecvLoop,
115+
// and new added request won't be removed by `failPendingRequests`.
116+
c.clientLock.Lock()
117+
defer c.clientLock.Unlock()
118+
for i, requestID := range request.RequestIds {
119+
c.batched.Store(requestID, entries[i])
120+
}
121+
if err := c.client.Send(request); err != nil {
122+
logutil.BgLogger().Error(
123+
"batch commands send error",
124+
zap.String("target", c.target),
125+
zap.Error(err),
126+
)
127+
c.failPendingRequests(err)
128+
}
129+
}
130+
131+
func (c *batchCommandsClient) recv() (*tikvpb.BatchCommandsResponse, error) {
132+
failpoint.Inject("gotErrorInRecvLoop", func(_ failpoint.Value) (*tikvpb.BatchCommandsResponse, error) {
133+
return nil, errors.New("injected error in batchRecvLoop")
134+
})
135+
// When `conn.Close()` is called, `client.Recv()` will return an error.
136+
return c.client.Recv()
137+
}
138+
139+
// `failPendingRequests` must be called in locked contexts in order to avoid double closing channels.
108140
func (c *batchCommandsClient) failPendingRequests(err error) {
141+
failpoint.Inject("panicInFailPendingRequests", nil)
109142
c.batched.Range(func(key, value interface{}) bool {
110143
id, _ := key.(uint64)
111144
entry, _ := value.(*batchCommandsEntry)
112145
entry.err = err
113-
close(entry.res)
114146
c.batched.Delete(id)
147+
close(entry.res)
115148
return true
116149
})
117150
}
118151

152+
func (c *batchCommandsClient) reCreateStreamingClient(err error) bool {
153+
// Hold the lock to forbid batchSendLoop using the old client.
154+
c.clientLock.Lock()
155+
defer c.clientLock.Unlock()
156+
c.failPendingRequests(err) // fail all pending requests.
157+
158+
// Re-establish a application layer stream. TCP layer is handled by gRPC.
159+
tikvClient := tikvpb.NewTikvClient(c.conn)
160+
streamClient, err := tikvClient.BatchCommands(context.TODO())
161+
if err == nil {
162+
logutil.BgLogger().Info(
163+
"batchRecvLoop re-create streaming success",
164+
zap.String("target", c.target),
165+
)
166+
c.client = streamClient
167+
return true
168+
}
169+
logutil.BgLogger().Error(
170+
"batchRecvLoop re-create streaming fail",
171+
zap.String("target", c.target),
172+
zap.Error(err),
173+
)
174+
return false
175+
}
176+
119177
func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient) {
120178
defer func() {
121179
if r := recover(); r != nil {
@@ -129,8 +187,7 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient) {
129187
}()
130188

131189
for {
132-
// When `conn.Close()` is called, `client.Recv()` will return an error.
133-
resp, err := c.client.Recv()
190+
resp, err := c.recv()
134191
if err != nil {
135192
now := time.Now()
136193
for { // try to re-create the streaming in the loop.
@@ -143,28 +200,10 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient) {
143200
zap.Error(err),
144201
)
145202

146-
// Hold the lock to forbid batchSendLoop using the old client.
147-
c.clientLock.Lock()
148-
c.failPendingRequests(err) // fail all pending requests.
149-
150-
// Re-establish a application layer stream. TCP layer is handled by gRPC.
151-
tikvClient := tikvpb.NewTikvClient(c.conn)
152-
streamClient, err := tikvClient.BatchCommands(context.TODO())
153-
c.clientLock.Unlock()
154-
155-
if err == nil {
156-
logutil.BgLogger().Info(
157-
"batchRecvLoop re-create streaming success",
158-
zap.String("target", c.target),
159-
)
160-
c.client = streamClient
203+
if c.reCreateStreamingClient(err) {
161204
break
162205
}
163-
logutil.BgLogger().Error(
164-
"batchRecvLoop re-create streaming fail",
165-
zap.String("target", c.target),
166-
zap.Error(err),
167-
)
206+
168207
// TODO: Use a more smart backoff strategy.
169208
time.Sleep(time.Second)
170209
}
@@ -208,6 +247,7 @@ func newConnArray(maxSize uint, addr string, security config.Security, idleNotif
208247
batchCommandsCh: make(chan *batchCommandsEntry, cfg.TiKVClient.MaxBatchSize),
209248
batchCommandsClients: make([]*batchCommandsClient, 0, maxSize),
210249
tikvTransportLayerLoad: 0,
250+
closed: make(chan struct{}),
211251

212252
idleNotify: idleNotify,
213253
idleDetect: time.NewTimer(idleTimeout),
@@ -220,6 +260,7 @@ func newConnArray(maxSize uint, addr string, security config.Security, idleNotif
220260

221261
func (a *connArray) Init(addr string, security config.Security) error {
222262
a.target = addr
263+
a.pendingRequests = metrics.TiKVPendingBatchRequests.WithLabelValues(a.target)
223264

224265
opt := grpc.WithInsecure()
225266
if len(security.ClusterSSLCA) != 0 {
@@ -309,7 +350,10 @@ func (a *connArray) Close() {
309350
// After connections are closed, `batchRecvLoop`s will check the flag.
310351
atomic.StoreInt32(&c.closed, 1)
311352
}
312-
close(a.batchCommandsCh)
353+
// Don't close(batchCommandsCh) because when Close() is called, someone maybe
354+
// calling SendRequest and writing batchCommandsCh, if we close it here the
355+
// writing goroutine will panic.
356+
close(a.closed)
313357

314358
for i, c := range a.v {
315359
if c != nil {
@@ -356,6 +400,8 @@ func (a *connArray) fetchAllPendingRequests(
356400
atomic.CompareAndSwapUint32(a.idleNotify, 0, 1)
357401
// This connArray to be recycled
358402
return
403+
case <-a.closed:
404+
return
359405
}
360406
if headEntry == nil {
361407
return
@@ -450,7 +496,7 @@ func (a *connArray) batchSendLoop(cfg config.TiKVClient) {
450496
requests = requests[:0]
451497
requestIDs = requestIDs[:0]
452498

453-
metrics.TiKVPendingBatchRequests.Set(float64(len(a.batchCommandsCh)))
499+
a.pendingRequests.Set(float64(len(a.batchCommandsCh)))
454500
a.fetchAllPendingRequests(int(cfg.MaxBatchSize), &entries, &requests)
455501

456502
if len(entries) < int(cfg.MaxBatchSize) && cfg.MaxBatchWaitTime > 0 {
@@ -484,27 +530,12 @@ func (a *connArray) batchSendLoop(cfg config.TiKVClient) {
484530
requestIDs = append(requestIDs, requestID)
485531
}
486532

487-
request := &tikvpb.BatchCommandsRequest{
533+
req := &tikvpb.BatchCommandsRequest{
488534
Requests: requests,
489535
RequestIds: requestIDs,
490536
}
491537

492-
// Use the lock to protect the stream client won't be replaced by RecvLoop,
493-
// and new added request won't be removed by `failPendingRequests`.
494-
batchCommandsClient.clientLock.Lock()
495-
for i, requestID := range request.RequestIds {
496-
batchCommandsClient.batched.Store(requestID, entries[i])
497-
}
498-
err := batchCommandsClient.client.Send(request)
499-
batchCommandsClient.clientLock.Unlock()
500-
if err != nil {
501-
logutil.BgLogger().Error(
502-
"batch commands send error",
503-
zap.String("target", a.target),
504-
zap.Error(err),
505-
)
506-
batchCommandsClient.failPendingRequests(err)
507-
}
538+
batchCommandsClient.send(req, entries)
508539
}
509540
}
510541

@@ -547,6 +578,11 @@ func newRPCClient(security config.Security) *rpcClient {
547578
}
548579
}
549580

581+
// NewTestRPCClient is for some external tests.
582+
func NewTestRPCClient() Client {
583+
return newRPCClient(config.Security{})
584+
}
585+
550586
func (c *rpcClient) getConnArray(addr string) (*connArray, error) {
551587
c.RLock()
552588
if c.isClosed {

store/tikv/client_fail_test.go

Lines changed: 62 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,62 @@
1+
// Copyright 2019 PingCAP, Inc.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// See the License for the specific language governing permissions and
12+
// limitations under the License.
13+
14+
package tikv
15+
16+
import (
17+
"context"
18+
"fmt"
19+
"time"
20+
21+
. "github.com/pingcap/check"
22+
"github.com/pingcap/failpoint"
23+
"github.com/pingcap/kvproto/pkg/tikvpb"
24+
"github.com/pingcap/tidb/config"
25+
"github.com/pingcap/tidb/store/tikv/tikvrpc"
26+
)
27+
28+
func setGrpcConnectionCount(count uint) {
29+
newConf := config.NewConfig()
30+
newConf.TiKVClient.GrpcConnectionCount = count
31+
config.StoreGlobalConfig(newConf)
32+
}
33+
34+
func (s *testClientSuite) TestPanicInRecvLoop(c *C) {
35+
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/panicInFailPendingRequests", `panic`), IsNil)
36+
c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/gotErrorInRecvLoop", `return("0")`), IsNil)
37+
38+
server, port := startMockTikvService()
39+
c.Assert(port > 0, IsTrue)
40+
41+
grpcConnectionCount := config.GetGlobalConfig().TiKVClient.GrpcConnectionCount
42+
setGrpcConnectionCount(1)
43+
addr := fmt.Sprintf("%s:%d", "127.0.0.1", port)
44+
rpcClient := newRPCClient(config.Security{})
45+
46+
// Start batchRecvLoop, and it should panic in `failPendingRequests`.
47+
_, err := rpcClient.getConnArray(addr)
48+
c.Assert(err, IsNil)
49+
50+
time.Sleep(time.Second)
51+
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/panicInFailPendingRequests"), IsNil)
52+
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/gotErrorInRecvLoop"), IsNil)
53+
54+
req := &tikvrpc.Request{
55+
Type: tikvrpc.CmdEmpty,
56+
Empty: &tikvpb.BatchCommandsEmptyRequest{},
57+
}
58+
_, err = rpcClient.SendRequest(context.Background(), addr, req, time.Second)
59+
c.Assert(err, IsNil)
60+
server.Stop()
61+
setGrpcConnectionCount(grpcConnectionCount)
62+
}

store/tikv/client_test.go

Lines changed: 9 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -35,9 +35,15 @@ type testClientSuite struct {
3535

3636
var _ = Suite(&testClientSuite{})
3737

38+
func setMaxBatchSize(size uint) {
39+
newConf := config.NewConfig()
40+
newConf.TiKVClient.MaxBatchSize = size
41+
config.StoreGlobalConfig(newConf)
42+
}
43+
3844
func (s *testClientSuite) TestConn(c *C) {
39-
globalConfig := config.GetGlobalConfig()
40-
globalConfig.TiKVClient.MaxBatchSize = 0 // Disable batch.
45+
maxBatchSize := config.GetGlobalConfig().TiKVClient.MaxBatchSize
46+
setMaxBatchSize(0)
4147

4248
client := newRPCClient(config.Security{})
4349

@@ -55,6 +61,7 @@ func (s *testClientSuite) TestConn(c *C) {
5561
conn3, err := client.getConnArray(addr)
5662
c.Assert(err, NotNil)
5763
c.Assert(conn3, IsNil)
64+
setMaxBatchSize(maxBatchSize)
5865
}
5966

6067
func (s *testClientSuite) TestRemoveCanceledRequests(c *C) {

0 commit comments

Comments
 (0)