Remove old time tick logic (#8596)

Signed-off-by: dragondriver <jiquan.long@zilliz.com>
pull/8617/head
dragondriver 2021-09-26 19:23:56 +08:00 committed by GitHub
parent 0fa38d5d93
commit 8091e76666
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 64 additions and 163 deletions

View File

@ -17,6 +17,8 @@ import (
"sync"
"time"
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/milvus-io/milvus/internal/log"
"go.uber.org/zap"
)
@ -31,6 +33,7 @@ type channelsTimeTicker interface {
removePChan(pchan pChan) error
getLastTick(pchan pChan) (Timestamp, error)
getMinTsStatistics() (map[pChan]Timestamp, error)
getMinTick() Timestamp
}
type channelsTimeTickerImpl struct {
@ -210,6 +213,21 @@ func (ticker *channelsTimeTickerImpl) getLastTick(pchan pChan) (Timestamp, error
return ts, nil
}
func (ticker *channelsTimeTickerImpl) getMinTick() Timestamp {
ticker.statisticsMtx.RLock()
defer ticker.statisticsMtx.RUnlock()
minTs := typeutil.ZeroTimestamp
for _, ts := range ticker.minTsStatistics {
if ts < minTs {
minTs = ts
}
}
return minTs
}
func newChannelsTimeTicker(
ctx context.Context,
interval time.Duration,

View File

@ -18,6 +18,8 @@ import (
"testing"
"time"
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/milvus-io/milvus/internal/util/funcutil"
"go.uber.org/zap"
@ -214,3 +216,46 @@ func TestChannelsTimeTickerImpl_getMinTsStatistics(t *testing.T) {
time.Sleep(100 * time.Millisecond)
}
func TestChannelsTimeTickerImpl_getMinTick(t *testing.T) {
interval := time.Millisecond * 10
pchanNum := rand.Uint64()%10 + 1
pchans := make([]pChan, 0, pchanNum)
for i := 0; uint64(i) < pchanNum; i++ {
pchans = append(pchans, funcutil.GenRandomStr())
}
tso := newMockTsoAllocator()
ctx := context.Background()
ticker := newChannelsTimeTicker(ctx, interval, pchans, newGetStatisticsFunc(pchans), tso)
err := ticker.start()
assert.Equal(t, nil, err)
var wg sync.WaitGroup
wg.Add(1)
b := make(chan struct{}, 1)
ts := typeutil.ZeroTimestamp
go func() {
defer wg.Done()
timer := time.NewTicker(interval * 40)
for {
select {
case <-b:
return
case <-timer.C:
minTs := ticker.getMinTick()
assert.GreaterOrEqual(t, minTs, ts)
}
}
}()
time.Sleep(100 * time.Millisecond)
b <- struct{}{}
wg.Wait()
defer func() {
err := ticker.close()
assert.Equal(t, nil, err)
}()
time.Sleep(100 * time.Millisecond)
}

View File

@ -62,7 +62,6 @@ type Proxy struct {
chMgr channelsMgr
sched *taskScheduler
tick *timeTick
chTicker channelsTimeTicker
@ -197,8 +196,6 @@ func (node *Proxy) Init() error {
return err
}
node.tick = newTimeTick(node.ctx, node.tsoAllocator, time.Millisecond*200, node.sched.TaskDoneTest, node.msFactory)
node.chTicker = newChannelsTimeTicker(node.ctx, channelMgrTickerInterval, []string{}, node.sched.getPChanStatistics, tsoAllocator)
node.metricsCacheManager = metricsinfo.NewMetricsCacheManager()
@ -299,11 +296,6 @@ func (node *Proxy) Start() error {
}
log.Debug("start seg assigner ...")
if err := node.tick.Start(); err != nil {
return err
}
log.Debug("start time tick ...")
err = node.chTicker.start()
if err != nil {
return err
@ -335,9 +327,6 @@ func (node *Proxy) Stop() error {
if node.sched != nil {
node.sched.Close()
}
if node.tick != nil {
node.tick.Close()
}
if node.chTicker != nil {
err := node.chTicker.close()
if err != nil {
@ -360,7 +349,7 @@ func (node *Proxy) AddStartCallback(callbacks ...func()) {
}
func (node *Proxy) lastTick() Timestamp {
return node.tick.LastTick()
return node.chTicker.getMinTick()
}
// AddCloseCallback adds a callback in the Close phase.

View File

@ -1,151 +0,0 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// 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 proxy
import (
"context"
"sync"
"time"
"go.uber.org/zap"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/msgstream"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
)
type tickCheckFunc = func(Timestamp) bool
type timeTick struct {
lastTick Timestamp
currentTick Timestamp
interval time.Duration
pulsarProducer pulsar.Producer
tsoAllocator *TimestampAllocator
tickMsgStream msgstream.MsgStream
msFactory msgstream.Factory
peerID UniqueID
wg sync.WaitGroup
ctx context.Context
cancel func()
timer *time.Ticker
tickLock sync.RWMutex
checkFunc tickCheckFunc
}
func newTimeTick(ctx context.Context,
tsoAllocator *TimestampAllocator,
interval time.Duration,
checkFunc tickCheckFunc,
factory msgstream.Factory) *timeTick {
ctx1, cancel := context.WithCancel(ctx)
t := &timeTick{
ctx: ctx1,
cancel: cancel,
tsoAllocator: tsoAllocator,
interval: interval,
peerID: Params.ProxyID,
checkFunc: checkFunc,
msFactory: factory,
}
t.tickMsgStream, _ = t.msFactory.NewMsgStream(t.ctx)
t.tickMsgStream.AsProducer(Params.ProxyTimeTickChannelNames)
log.Debug("proxy", zap.Strings("proxy AsProducer", Params.ProxyTimeTickChannelNames))
return t
}
func (tt *timeTick) tick() error {
if tt.lastTick == tt.currentTick {
ts, err := tt.tsoAllocator.AllocOne()
if err != nil {
return err
}
tt.currentTick = ts
}
if !tt.checkFunc(tt.currentTick) {
return nil
}
msgPack := msgstream.MsgPack{}
timeTickMsg := &msgstream.TimeTickMsg{
BaseMsg: msgstream.BaseMsg{
HashValues: []uint32{uint32(Params.ProxyID)},
},
TimeTickMsg: internalpb.TimeTickMsg{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_TimeTick,
MsgID: 0,
Timestamp: tt.currentTick,
SourceID: tt.peerID,
},
},
}
msgPack.Msgs = append(msgPack.Msgs, timeTickMsg)
err := tt.tickMsgStream.Produce(&msgPack)
if err != nil {
log.Warn("proxy", zap.String("error", err.Error()))
}
tt.tickLock.Lock()
defer tt.tickLock.Unlock()
tt.lastTick = tt.currentTick
return nil
}
func (tt *timeTick) tickLoop() {
defer tt.wg.Done()
tt.timer = time.NewTicker(tt.interval)
for {
select {
case <-tt.timer.C:
if err := tt.tick(); err != nil {
log.Warn("timeTick error")
}
case <-tt.ctx.Done():
return
}
}
}
func (tt *timeTick) LastTick() Timestamp {
tt.tickLock.RLock()
defer tt.tickLock.RUnlock()
return tt.lastTick
}
func (tt *timeTick) Start() error {
tt.lastTick = 0
ts, err := tt.tsoAllocator.AllocOne()
if err != nil {
return err
}
tt.currentTick = ts
tt.tickMsgStream.Start()
tt.wg.Add(1)
go tt.tickLoop()
return nil
}
func (tt *timeTick) Close() {
if tt.timer != nil {
tt.timer.Stop()
}
tt.cancel()
tt.tickMsgStream.Close()
tt.wg.Wait()
}