mirror of https://github.com/milvus-io/milvus.git
fix: watch channel stuck due to misuse of timer.Reset (#37433)
issue: #37166 cause the misuse of timer.Reset, which cause dispatcher failed to send msg to virtual channel buffer, and dispatcher do splitting again and again, which hold the dispatcher manager's lock, block watching channel progress. This PR fix the misuse of timer.Reset Signed-off-by: Wei Liu <wei.liu@zilliz.com>pull/37501/head
parent
86fd3200be
commit
00f6d0ec51
|
@ -75,6 +75,12 @@ func (t *CreateIndexTask) Await(ctx context.Context) error {
|
|||
if finished {
|
||||
return nil
|
||||
}
|
||||
if !timer.Stop() {
|
||||
select {
|
||||
case <-timer.C:
|
||||
default:
|
||||
}
|
||||
}
|
||||
timer.Reset(t.interval)
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
|
|
|
@ -57,6 +57,12 @@ func (t *LoadTask) Await(ctx context.Context) error {
|
|||
if loaded {
|
||||
return nil
|
||||
}
|
||||
if !timer.Stop() {
|
||||
select {
|
||||
case <-timer.C:
|
||||
default:
|
||||
}
|
||||
}
|
||||
timer.Reset(t.interval)
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
|
@ -163,6 +169,12 @@ func (t *FlushTask) Await(ctx context.Context) error {
|
|||
if flushed {
|
||||
return nil
|
||||
}
|
||||
if !timer.Stop() {
|
||||
select {
|
||||
case <-timer.C:
|
||||
default:
|
||||
}
|
||||
}
|
||||
timer.Reset(t.interval)
|
||||
case <-ctx.Done():
|
||||
return ctx.Err()
|
||||
|
|
|
@ -402,6 +402,12 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState {
|
|||
|
||||
case <-tickler.GetProgressSig():
|
||||
log.Info("Reset timer for tickler updated", zap.Int32("current progress", tickler.Progress()))
|
||||
if !timer.Stop() {
|
||||
select {
|
||||
case <-timer.C:
|
||||
default:
|
||||
}
|
||||
}
|
||||
timer.Reset(watchTimeout)
|
||||
|
||||
case <-successSig:
|
||||
|
|
|
@ -73,6 +73,13 @@ func (t *target) send(pack *MsgPack) error {
|
|||
if t.closed {
|
||||
return nil
|
||||
}
|
||||
|
||||
if !t.timer.Stop() {
|
||||
select {
|
||||
case <-t.timer.C:
|
||||
default:
|
||||
}
|
||||
}
|
||||
t.timer.Reset(t.maxLag)
|
||||
select {
|
||||
case <-t.cancelCh.CloseCh():
|
||||
|
|
|
@ -0,0 +1,30 @@
|
|||
package msgdispatcher
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
)
|
||||
|
||||
func TestSendTimeout(t *testing.T) {
|
||||
target := newTarget("test1", &msgpb.MsgPosition{})
|
||||
|
||||
time.Sleep(paramtable.Get().MQCfg.MaxTolerantLag.GetAsDuration(time.Second))
|
||||
|
||||
counter := 0
|
||||
for i := 0; i < 10; i++ {
|
||||
err := target.send(&msgstream.MsgPack{})
|
||||
if err != nil {
|
||||
log.Error("send failed", zap.Int("idx", i), zap.Error(err))
|
||||
counter++
|
||||
}
|
||||
}
|
||||
assert.Equal(t, counter, 0)
|
||||
}
|
Loading…
Reference in New Issue