Migrate softTimeTickBarrier from proxyservice to timesync (#5146)

Signed-off-by: dragondriver <jiquan.long@zilliz.com>
pull/5157/head
dragondriver 2021-05-10 09:28:59 +08:00 committed by GitHub
parent 8752cae5ee
commit df71f4c77c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 139 additions and 257 deletions

View File

@ -20,6 +20,8 @@ import (
"strconv"
"time"
"github.com/milvus-io/milvus/internal/timesync"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/log"
@ -138,7 +140,7 @@ func (s *ProxyService) Init() error {
"proxyservicesub") // TODO: add config
log.Debug("proxyservice", zap.Strings("create node time tick consumer channel", Params.NodeTimeTickChannel))
ttBarrier := newSoftTimeTickBarrier(s.ctx, nodeTimeTickMsgStream, []UniqueID{1}, 10)
ttBarrier := timesync.NewSoftTimeTickBarrier(s.ctx, nodeTimeTickMsgStream, []UniqueID{1}, 10)
log.Debug("create soft time tick barrier ...")
s.tick = newTimeTick(s.ctx, ttBarrier, serviceTimeTickMsgStream, insertTickMsgStream)
log.Debug("create time tick ...")

View File

@ -1,167 +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 proxyservice
import (
"context"
"errors"
"math"
"sync"
"sync/atomic"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/log"
ms "github.com/milvus-io/milvus/internal/msgstream"
)
type (
TimeTickBarrier interface {
GetTimeTick() (Timestamp, error)
Start() error
Close()
AddPeer(peerID UniqueID) error
TickChan() <-chan Timestamp
}
softTimeTickBarrier struct {
peer2LastTt map[UniqueID]Timestamp
peerMtx sync.RWMutex
minTtInterval Timestamp
lastTt int64
outTt chan Timestamp
ttStream ms.MsgStream
ctx context.Context
cancel context.CancelFunc
}
)
func (ttBarrier *softTimeTickBarrier) TickChan() <-chan Timestamp {
return ttBarrier.outTt
}
func (ttBarrier *softTimeTickBarrier) AddPeer(peerID UniqueID) error {
ttBarrier.peerMtx.Lock()
defer ttBarrier.peerMtx.Unlock()
_, ok := ttBarrier.peer2LastTt[peerID]
if ok {
log.Debug("proxyservice", zap.Int64("no need to add duplicated peer", peerID))
return nil
}
ttBarrier.peer2LastTt[peerID] = 0
return nil
}
func (ttBarrier *softTimeTickBarrier) GetTimeTick() (Timestamp, error) {
select {
case <-ttBarrier.ctx.Done():
return 0, errors.New("getTimeTick closed")
case ts, ok := <-ttBarrier.outTt:
if !ok {
return 0, errors.New("getTimeTick closed")
}
num := len(ttBarrier.outTt)
for i := 0; i < num; i++ {
ts, ok = <-ttBarrier.outTt
if !ok {
return 0, errors.New("getTimeTick closed")
}
}
atomic.StoreInt64(&(ttBarrier.lastTt), int64(ts))
return ts, ttBarrier.ctx.Err()
}
}
func (ttBarrier *softTimeTickBarrier) Start() error {
go func() {
for {
select {
case <-ttBarrier.ctx.Done():
log.Warn("TtBarrierStart", zap.Error(ttBarrier.ctx.Err()))
return
case ttmsgs := <-ttBarrier.ttStream.Chan():
ttBarrier.peerMtx.RLock()
if len(ttmsgs.Msgs) > 0 {
for _, timetickmsg := range ttmsgs.Msgs {
ttmsg := timetickmsg.(*ms.TimeTickMsg)
oldT, ok := ttBarrier.peer2LastTt[ttmsg.Base.SourceID]
if !ok {
log.Warn("softTimeTickBarrier", zap.Int64("peerID %d not exist", ttmsg.Base.SourceID))
continue
}
if ttmsg.Base.Timestamp > oldT {
ttBarrier.peer2LastTt[ttmsg.Base.SourceID] = ttmsg.Base.Timestamp
// get a legal Timestamp
ts := ttBarrier.minTimestamp()
lastTt := atomic.LoadInt64(&(ttBarrier.lastTt))
if lastTt != 0 && ttBarrier.minTtInterval > ts-Timestamp(lastTt) {
continue
}
ttBarrier.outTt <- ts
}
}
}
ttBarrier.peerMtx.RUnlock()
}
}
}()
ttBarrier.ttStream.Start()
return nil
}
func newSoftTimeTickBarrier(ctx context.Context,
ttStream ms.MsgStream,
peerIds []UniqueID,
minTtInterval Timestamp) TimeTickBarrier {
if len(peerIds) <= 0 {
log.Warn("[newSoftTimeTickBarrier] Warning: peerIds is empty!")
//return nil
}
sttbarrier := softTimeTickBarrier{}
sttbarrier.minTtInterval = minTtInterval
sttbarrier.ttStream = ttStream
sttbarrier.outTt = make(chan Timestamp, 1024)
sttbarrier.ctx, sttbarrier.cancel = context.WithCancel(ctx)
sttbarrier.peer2LastTt = make(map[UniqueID]Timestamp)
for _, id := range peerIds {
sttbarrier.peer2LastTt[id] = Timestamp(0)
}
if len(peerIds) != len(sttbarrier.peer2LastTt) {
log.Warn("[newSoftTimeTickBarrier] Warning: there are duplicate peerIds!")
}
return &sttbarrier
}
func (ttBarrier *softTimeTickBarrier) Close() {
ttBarrier.cancel()
}
func (ttBarrier *softTimeTickBarrier) minTimestamp() Timestamp {
tempMin := Timestamp(math.MaxUint64)
for _, tt := range ttBarrier.peer2LastTt {
if tt < tempMin {
tempMin = tt
}
}
return tempMin
}

View File

@ -15,6 +15,8 @@ import (
"context"
"sync"
"github.com/milvus-io/milvus/internal/timesync"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/log"
@ -24,7 +26,7 @@ import (
)
type TimeTick struct {
ttBarrier TimeTickBarrier
ttBarrier timesync.TimeTickBarrier
channels []msgstream.MsgStream
wg sync.WaitGroup
ctx context.Context
@ -79,10 +81,7 @@ func (tt *TimeTick) Start() error {
channel.Start()
}
err := tt.ttBarrier.Start()
if err != nil {
return err
}
tt.ttBarrier.Start()
return nil
}
@ -96,7 +95,7 @@ func (tt *TimeTick) Close() {
tt.wg.Wait()
}
func newTimeTick(ctx context.Context, ttBarrier TimeTickBarrier, channels ...msgstream.MsgStream) *TimeTick {
func newTimeTick(ctx context.Context, ttBarrier timesync.TimeTickBarrier, channels ...msgstream.MsgStream) *TimeTick {
ctx1, cancel := context.WithCancel(ctx)
return &TimeTick{ctx: ctx1, cancel: cancel, ttBarrier: ttBarrier, channels: channels}
}

View File

@ -17,10 +17,63 @@ import (
"testing"
"time"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/commonpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/timesync"
"github.com/milvus-io/milvus/internal/msgstream"
"github.com/stretchr/testify/assert"
)
func ttStreamProduceLoop(ctx context.Context, ttStream msgstream.MsgStream, durationInterval time.Duration, sourceID int64) {
log.Debug("ttStreamProduceLoop", zap.Any("durationInterval", durationInterval))
timer := time.NewTicker(durationInterval)
go func() {
for {
select {
case <-ctx.Done():
return
case <-timer.C:
ttMsgs := &msgstream.MsgPack{
BeginTs: 0,
EndTs: 0,
Msgs: nil,
StartPositions: nil,
EndPositions: nil,
}
currentT := uint64(time.Now().Nanosecond())
msg := &msgstream.TimeTickMsg{
BaseMsg: msgstream.BaseMsg{
Ctx: ctx,
BeginTimestamp: 0,
EndTimestamp: 0,
HashValues: nil,
MsgPosition: nil,
},
TimeTickMsg: internalpb.TimeTickMsg{
Base: &commonpb.MsgBase{
MsgType: 0,
MsgID: 0,
Timestamp: currentT,
SourceID: sourceID,
},
},
}
ttMsgs.Msgs = append(ttMsgs.Msgs, msg)
_ = ttStream.Produce(ttMsgs)
//log.Debug("ttStreamProduceLoop", zap.Any("Send", currentT))
}
}
}()
}
func TestTimeTick_Start(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
@ -32,13 +85,13 @@ func TestTimeTick_Start(t *testing.T) {
minTtInterval := Timestamp(interval)
durationInterval := time.Duration(interval*int(math.Pow10(6))) >> 18
ttStreamProduceLoop(ctx, ttStream, durationInterval, int64(sourceID))
ttBarrier := newSoftTimeTickBarrier(ctx, ttStream, peerIds, minTtInterval)
ttBarrier := timesync.NewSoftTimeTickBarrier(ctx, ttStream, peerIds, minTtInterval)
channels := msgstream.NewSimpleMsgStream()
tick := newTimeTick(ctx, ttBarrier, channels)
err := tick.Start()
ttStreamProduceLoop(ctx, ttStream, durationInterval, int64(sourceID))
assert.Equal(t, nil, err)
defer tick.Close()
}
@ -54,13 +107,13 @@ func TestTimeTick_Close(t *testing.T) {
minTtInterval := Timestamp(interval)
durationInterval := time.Duration(interval*int(math.Pow10(6))) >> 18
ttStreamProduceLoop(ctx, ttStream, durationInterval, int64(sourceID))
ttBarrier := newSoftTimeTickBarrier(ctx, ttStream, peerIds, minTtInterval)
ttBarrier := timesync.NewSoftTimeTickBarrier(ctx, ttStream, peerIds, minTtInterval)
channels := msgstream.NewSimpleMsgStream()
tick := newTimeTick(ctx, ttBarrier, channels)
err := tick.Start()
ttStreamProduceLoop(ctx, ttStream, durationInterval, int64(sourceID))
assert.Equal(t, nil, err)
defer tick.Close()
}

View File

@ -42,11 +42,14 @@ type (
softTimeTickBarrier struct {
peer2LastTt map[UniqueID]Timestamp
peerMtx sync.RWMutex
minTtInterval Timestamp
lastTt int64
outTt chan Timestamp
ttStream ms.MsgStream
ctx context.Context
cancel context.CancelFunc
wg sync.WaitGroup
}
hardTimeTickBarrier struct {
@ -62,21 +65,21 @@ type (
func NewSoftTimeTickBarrier(ctx context.Context, ttStream ms.MsgStream, peerIds []UniqueID, minTtInterval Timestamp) *softTimeTickBarrier {
if len(peerIds) <= 0 {
log.Debug("[newSoftTimeTickBarrier] Error: peerIds is empty!")
return nil
log.Warn("[newSoftTimeTickBarrier] Warning: peerIds is empty!")
//return nil
}
sttbarrier := softTimeTickBarrier{}
sttbarrier.minTtInterval = minTtInterval
sttbarrier.ttStream = ttStream
sttbarrier.outTt = make(chan Timestamp, 1024)
sttbarrier.ctx, sttbarrier.cancel = context.WithCancel(ctx)
sttbarrier.peer2LastTt = make(map[UniqueID]Timestamp)
sttbarrier.ctx = ctx
for _, id := range peerIds {
sttbarrier.peer2LastTt[id] = Timestamp(0)
}
if len(peerIds) != len(sttbarrier.peer2LastTt) {
log.Debug("[newSoftTimeTickBarrier] Warning: there are duplicate peerIds!")
log.Warn("[newSoftTimeTickBarrier] Warning: there are duplicate peerIds!")
}
return &sttbarrier
@ -103,38 +106,66 @@ func (ttBarrier *softTimeTickBarrier) GetTimeTick() (Timestamp, error) {
}
func (ttBarrier *softTimeTickBarrier) Start() {
for {
select {
case <-ttBarrier.ctx.Done():
log.Debug("[TtBarrierStart] shut down", zap.Error(ttBarrier.ctx.Err()))
return
default:
}
ttmsgs := ttBarrier.ttStream.Consume()
if len(ttmsgs.Msgs) > 0 {
for _, timetickmsg := range ttmsgs.Msgs {
ttmsg := timetickmsg.(*ms.TimeTickMsg)
oldT, ok := ttBarrier.peer2LastTt[ttmsg.Base.SourceID]
// log.Printf("[softTimeTickBarrier] peer(%d)=%d\n", ttmsg.PeerID, ttmsg.Timestamp)
ttBarrier.wg.Add(1)
go func() {
defer ttBarrier.wg.Done()
for {
select {
case <-ttBarrier.ctx.Done():
log.Warn("TtBarrierStart", zap.Error(ttBarrier.ctx.Err()))
return
if !ok {
log.Warn("[softTimeTickBarrier] peerID not exist", zap.Int64("peerID", ttmsg.Base.SourceID))
continue
}
if ttmsg.Base.Timestamp > oldT {
ttBarrier.peer2LastTt[ttmsg.Base.SourceID] = ttmsg.Base.Timestamp
case ttmsgs := <-ttBarrier.ttStream.Chan():
ttBarrier.peerMtx.RLock()
if len(ttmsgs.Msgs) > 0 {
for _, timetickmsg := range ttmsgs.Msgs {
ttmsg := timetickmsg.(*ms.TimeTickMsg)
oldT, ok := ttBarrier.peer2LastTt[ttmsg.Base.SourceID]
// get a legal Timestamp
ts := ttBarrier.minTimestamp()
lastTt := atomic.LoadInt64(&(ttBarrier.lastTt))
if lastTt != 0 && ttBarrier.minTtInterval > ts-Timestamp(lastTt) {
continue
if !ok {
log.Warn("softTimeTickBarrier", zap.Int64("peerID %d not exist", ttmsg.Base.SourceID))
continue
}
if ttmsg.Base.Timestamp > oldT {
ttBarrier.peer2LastTt[ttmsg.Base.SourceID] = ttmsg.Base.Timestamp
// get a legal Timestamp
ts := ttBarrier.minTimestamp()
lastTt := atomic.LoadInt64(&(ttBarrier.lastTt))
if lastTt != 0 && ttBarrier.minTtInterval > ts-Timestamp(lastTt) {
continue
}
ttBarrier.outTt <- ts
}
}
ttBarrier.outTt <- ts
}
ttBarrier.peerMtx.RUnlock()
}
}
}()
ttBarrier.ttStream.Start()
}
func (ttBarrier *softTimeTickBarrier) Close() {
ttBarrier.cancel()
ttBarrier.wg.Wait()
ttBarrier.ttStream.Close()
}
func (ttBarrier *softTimeTickBarrier) AddPeer(peerID UniqueID) error {
ttBarrier.peerMtx.Lock()
defer ttBarrier.peerMtx.Unlock()
_, ok := ttBarrier.peer2LastTt[peerID]
if ok {
log.Debug("softTimeTickBarrier.AddPeer", zap.Int64("no need to add duplicated peer", peerID))
return nil
}
ttBarrier.peer2LastTt[peerID] = 0
return nil
}
func (ttBarrier *softTimeTickBarrier) minTimestamp() Timestamp {

View File

@ -9,7 +9,7 @@
// 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 proxyservice
package timesync
import (
"context"
@ -86,11 +86,10 @@ func TestSoftTimeTickBarrier_Start(t *testing.T) {
minTtInterval := Timestamp(interval)
durationInterval := time.Duration(interval*int(math.Pow10(6))) >> 18
ttStreamProduceLoop(ctx, ttStream, durationInterval, int64(sourceID))
ttBarrier := newSoftTimeTickBarrier(ctx, ttStream, peerIds, minTtInterval)
err := ttBarrier.Start()
assert.Equal(t, nil, err)
ttBarrier := NewSoftTimeTickBarrier(ctx, ttStream, peerIds, minTtInterval)
ttBarrier.Start()
ttStreamProduceLoop(ctx, ttStream, durationInterval, int64(sourceID))
defer ttBarrier.Close()
}
@ -105,11 +104,10 @@ func TestSoftTimeTickBarrier_Close(t *testing.T) {
minTtInterval := Timestamp(interval)
durationInterval := time.Duration(interval*int(math.Pow10(6))) >> 18
ttStreamProduceLoop(ctx, ttStream, durationInterval, int64(sourceID))
ttBarrier := newSoftTimeTickBarrier(ctx, ttStream, peerIds, minTtInterval)
err := ttBarrier.Start()
assert.Equal(t, nil, err)
ttBarrier := NewSoftTimeTickBarrier(ctx, ttStream, peerIds, minTtInterval)
ttBarrier.Start()
ttStreamProduceLoop(ctx, ttStream, durationInterval, int64(sourceID))
defer ttBarrier.Close()
}
@ -124,11 +122,10 @@ func TestSoftTimeTickBarrier_GetTimeTick(t *testing.T) {
minTtInterval := Timestamp(interval)
durationInterval := time.Duration(interval*int(math.Pow10(6))) >> 18
ttStreamProduceLoop(ctx, ttStream, durationInterval, int64(sourceID))
ttBarrier := newSoftTimeTickBarrier(ctx, ttStream, peerIds, minTtInterval)
err := ttBarrier.Start()
assert.Equal(t, nil, err)
ttBarrier := NewSoftTimeTickBarrier(ctx, ttStream, peerIds, minTtInterval)
ttBarrier.Start()
ttStreamProduceLoop(ctx, ttStream, durationInterval, int64(sourceID))
defer ttBarrier.Close()
num := 10
@ -150,15 +147,14 @@ func TestSoftTimeTickBarrier_AddPeer(t *testing.T) {
minTtInterval := Timestamp(interval)
durationInterval := time.Duration(interval*int(math.Pow10(6))) >> 18
ttStreamProduceLoop(ctx, ttStream, durationInterval, int64(sourceID))
ttBarrier := newSoftTimeTickBarrier(ctx, ttStream, peerIds, minTtInterval)
err := ttBarrier.Start()
assert.Equal(t, nil, err)
ttBarrier := NewSoftTimeTickBarrier(ctx, ttStream, peerIds, minTtInterval)
ttBarrier.Start()
ttStreamProduceLoop(ctx, ttStream, durationInterval, int64(sourceID))
defer ttBarrier.Close()
newSourceID := UniqueID(2)
err = ttBarrier.AddPeer(newSourceID)
err := ttBarrier.AddPeer(newSourceID)
assert.Equal(t, nil, err)
ttStreamProduceLoop(ctx, ttStream, durationInterval, newSourceID)
@ -169,35 +165,3 @@ func TestSoftTimeTickBarrier_AddPeer(t *testing.T) {
log.Debug("TestSoftTimeTickBarrier", zap.Any("GetTimeTick", tick))
}
}
func TestSoftTimeTickBarrier_TickChan(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ttStream := msgstream.NewSimpleMsgStream()
sourceID := 1
peerIds := []UniqueID{UniqueID(sourceID)}
interval := 100
minTtInterval := Timestamp(interval)
durationInterval := time.Duration(interval*int(math.Pow10(6))) >> 18
ttStreamProduceLoop(ctx, ttStream, durationInterval, int64(sourceID))
ttBarrier := newSoftTimeTickBarrier(ctx, ttStream, peerIds, minTtInterval)
err := ttBarrier.Start()
assert.Equal(t, nil, err)
defer ttBarrier.Close()
duration := time.Second
timer := time.NewTimer(duration)
for {
select {
case <-ctx.Done():
return
case <-timer.C:
return
case ts := <-ttBarrier.TickChan():
log.Debug("TestSoftTimeTickBarrier", zap.Any("GetTimeTick", ts))
}
}
}