mirror of https://github.com/milvus-io/milvus.git
Migrate softTimeTickBarrier from proxyservice to timesync (#5146)
Signed-off-by: dragondriver <jiquan.long@zilliz.com>pull/5157/head
parent
8752cae5ee
commit
df71f4c77c
|
@ -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 ...")
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue