mirror of https://github.com/milvus-io/milvus.git
Remove old time tick logic (#8596)
Signed-off-by: dragondriver <jiquan.long@zilliz.com>pull/8617/head
parent
0fa38d5d93
commit
8091e76666
|
@ -17,6 +17,8 @@ import (
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/log"
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
|
@ -31,6 +33,7 @@ type channelsTimeTicker interface {
|
||||||
removePChan(pchan pChan) error
|
removePChan(pchan pChan) error
|
||||||
getLastTick(pchan pChan) (Timestamp, error)
|
getLastTick(pchan pChan) (Timestamp, error)
|
||||||
getMinTsStatistics() (map[pChan]Timestamp, error)
|
getMinTsStatistics() (map[pChan]Timestamp, error)
|
||||||
|
getMinTick() Timestamp
|
||||||
}
|
}
|
||||||
|
|
||||||
type channelsTimeTickerImpl struct {
|
type channelsTimeTickerImpl struct {
|
||||||
|
@ -210,6 +213,21 @@ func (ticker *channelsTimeTickerImpl) getLastTick(pchan pChan) (Timestamp, error
|
||||||
return ts, nil
|
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(
|
func newChannelsTimeTicker(
|
||||||
ctx context.Context,
|
ctx context.Context,
|
||||||
interval time.Duration,
|
interval time.Duration,
|
||||||
|
|
|
@ -18,6 +18,8 @@ import (
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||||
|
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
@ -214,3 +216,46 @@ func TestChannelsTimeTickerImpl_getMinTsStatistics(t *testing.T) {
|
||||||
|
|
||||||
time.Sleep(100 * time.Millisecond)
|
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)
|
||||||
|
}
|
||||||
|
|
|
@ -62,7 +62,6 @@ type Proxy struct {
|
||||||
chMgr channelsMgr
|
chMgr channelsMgr
|
||||||
|
|
||||||
sched *taskScheduler
|
sched *taskScheduler
|
||||||
tick *timeTick
|
|
||||||
|
|
||||||
chTicker channelsTimeTicker
|
chTicker channelsTimeTicker
|
||||||
|
|
||||||
|
@ -197,8 +196,6 @@ func (node *Proxy) Init() error {
|
||||||
return err
|
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.chTicker = newChannelsTimeTicker(node.ctx, channelMgrTickerInterval, []string{}, node.sched.getPChanStatistics, tsoAllocator)
|
||||||
|
|
||||||
node.metricsCacheManager = metricsinfo.NewMetricsCacheManager()
|
node.metricsCacheManager = metricsinfo.NewMetricsCacheManager()
|
||||||
|
@ -299,11 +296,6 @@ func (node *Proxy) Start() error {
|
||||||
}
|
}
|
||||||
log.Debug("start seg assigner ...")
|
log.Debug("start seg assigner ...")
|
||||||
|
|
||||||
if err := node.tick.Start(); err != nil {
|
|
||||||
return err
|
|
||||||
}
|
|
||||||
log.Debug("start time tick ...")
|
|
||||||
|
|
||||||
err = node.chTicker.start()
|
err = node.chTicker.start()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
|
@ -335,9 +327,6 @@ func (node *Proxy) Stop() error {
|
||||||
if node.sched != nil {
|
if node.sched != nil {
|
||||||
node.sched.Close()
|
node.sched.Close()
|
||||||
}
|
}
|
||||||
if node.tick != nil {
|
|
||||||
node.tick.Close()
|
|
||||||
}
|
|
||||||
if node.chTicker != nil {
|
if node.chTicker != nil {
|
||||||
err := node.chTicker.close()
|
err := node.chTicker.close()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -360,7 +349,7 @@ func (node *Proxy) AddStartCallback(callbacks ...func()) {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (node *Proxy) lastTick() Timestamp {
|
func (node *Proxy) lastTick() Timestamp {
|
||||||
return node.tick.LastTick()
|
return node.chTicker.getMinTick()
|
||||||
}
|
}
|
||||||
|
|
||||||
// AddCloseCallback adds a callback in the Close phase.
|
// AddCloseCallback adds a callback in the Close phase.
|
||||||
|
|
|
@ -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()
|
|
||||||
}
|
|
Loading…
Reference in New Issue