2021-04-19 15:16:33 +08:00
|
|
|
// 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.
|
|
|
|
|
2021-01-19 11:37:16 +08:00
|
|
|
package datanode
|
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
|
|
|
"math"
|
|
|
|
"testing"
|
|
|
|
"time"
|
|
|
|
|
|
|
|
"github.com/stretchr/testify/assert"
|
|
|
|
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/msgstream"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
2021-05-25 15:35:37 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
2021-01-19 11:37:16 +08:00
|
|
|
)
|
|
|
|
|
2021-10-08 19:09:12 +08:00
|
|
|
func getVchanInfo(info *testInfo) *datapb.VchannelInfo {
|
2021-09-09 15:36:01 +08:00
|
|
|
var ufs []*datapb.SegmentInfo
|
2021-10-08 19:09:12 +08:00
|
|
|
var fs []*datapb.SegmentInfo
|
|
|
|
if info.isValidCase {
|
2021-09-09 15:36:01 +08:00
|
|
|
ufs = []*datapb.SegmentInfo{{
|
2021-10-08 19:09:12 +08:00
|
|
|
CollectionID: info.ufCollID,
|
2021-09-09 15:36:01 +08:00
|
|
|
PartitionID: 1,
|
2021-10-08 19:09:12 +08:00
|
|
|
InsertChannel: info.ufchanName,
|
|
|
|
ID: info.ufSegID,
|
|
|
|
NumOfRows: info.ufNor,
|
|
|
|
DmlPosition: &internalpb.MsgPosition{},
|
|
|
|
}}
|
|
|
|
fs = []*datapb.SegmentInfo{{
|
|
|
|
CollectionID: info.fCollID,
|
|
|
|
PartitionID: 1,
|
|
|
|
InsertChannel: info.fchanName,
|
|
|
|
ID: info.fSegID,
|
|
|
|
NumOfRows: info.fNor,
|
2021-09-09 15:36:01 +08:00
|
|
|
DmlPosition: &internalpb.MsgPosition{},
|
|
|
|
}}
|
|
|
|
} else {
|
|
|
|
ufs = []*datapb.SegmentInfo{}
|
|
|
|
}
|
|
|
|
|
|
|
|
vi := &datapb.VchannelInfo{
|
2021-10-08 19:09:12 +08:00
|
|
|
CollectionID: info.collID,
|
|
|
|
ChannelName: info.chanName,
|
2021-09-09 15:36:01 +08:00
|
|
|
SeekPosition: &internalpb.MsgPosition{},
|
|
|
|
UnflushedSegments: ufs,
|
2021-10-08 19:09:12 +08:00
|
|
|
FlushedSegments: fs,
|
2021-09-09 15:36:01 +08:00
|
|
|
}
|
|
|
|
return vi
|
|
|
|
}
|
|
|
|
|
2021-10-08 19:09:12 +08:00
|
|
|
type testInfo struct {
|
|
|
|
isValidCase bool
|
|
|
|
replicaNil bool
|
|
|
|
inMsgFactory msgstream.Factory
|
2021-09-09 15:36:01 +08:00
|
|
|
|
2021-10-08 19:09:12 +08:00
|
|
|
collID UniqueID
|
|
|
|
chanName string
|
|
|
|
|
|
|
|
ufCollID UniqueID
|
|
|
|
ufSegID UniqueID
|
|
|
|
ufchanName string
|
|
|
|
ufNor int64
|
|
|
|
|
|
|
|
fCollID UniqueID
|
|
|
|
fSegID UniqueID
|
|
|
|
fchanName string
|
|
|
|
fNor int64
|
2021-09-09 15:36:01 +08:00
|
|
|
|
2021-10-08 19:09:12 +08:00
|
|
|
description string
|
|
|
|
}
|
|
|
|
|
|
|
|
func TestDataSyncService_newDataSyncService(te *testing.T) {
|
2021-09-09 15:36:01 +08:00
|
|
|
|
2021-10-08 19:09:12 +08:00
|
|
|
ctx := context.Background()
|
2021-09-09 15:36:01 +08:00
|
|
|
|
2021-10-08 19:09:12 +08:00
|
|
|
tests := []*testInfo{
|
2021-09-09 15:36:01 +08:00
|
|
|
{false, false, &mockMsgStreamFactory{false, true},
|
2021-10-08 19:09:12 +08:00
|
|
|
0, "",
|
|
|
|
0, 0, "", 0,
|
|
|
|
0, 0, "", 0,
|
2021-09-09 15:36:01 +08:00
|
|
|
"SetParamsReturnError"},
|
|
|
|
{true, false, &mockMsgStreamFactory{true, true},
|
2021-10-08 19:09:12 +08:00
|
|
|
0, "",
|
|
|
|
1, 0, "", 0,
|
|
|
|
1, 1, "", 0,
|
2021-09-09 15:36:01 +08:00
|
|
|
"CollID 0 mismach with seginfo collID 1"},
|
|
|
|
{true, false, &mockMsgStreamFactory{true, true},
|
2021-10-08 19:09:12 +08:00
|
|
|
1, "c1",
|
|
|
|
1, 0, "c2", 0,
|
|
|
|
1, 1, "c3", 0,
|
2021-09-09 15:36:01 +08:00
|
|
|
"chanName c1 mismach with seginfo chanName c2"},
|
|
|
|
{true, false, &mockMsgStreamFactory{true, true},
|
2021-10-08 19:09:12 +08:00
|
|
|
1, "c1",
|
|
|
|
1, 0, "c1", 0,
|
|
|
|
1, 1, "c2", 0,
|
2021-09-09 15:36:01 +08:00
|
|
|
"add normal segments"},
|
|
|
|
{false, false, &mockMsgStreamFactory{true, false},
|
2021-10-08 19:09:12 +08:00
|
|
|
0, "",
|
|
|
|
0, 0, "", 0,
|
|
|
|
0, 0, "", 0,
|
2021-09-09 15:36:01 +08:00
|
|
|
"error when newinsertbufernode"},
|
|
|
|
{false, true, &mockMsgStreamFactory{true, false},
|
2021-10-08 19:09:12 +08:00
|
|
|
0, "",
|
|
|
|
0, 0, "", 0,
|
|
|
|
0, 0, "", 0,
|
2021-09-09 15:36:01 +08:00
|
|
|
"replica nil"},
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, test := range tests {
|
|
|
|
te.Run(test.description, func(t *testing.T) {
|
|
|
|
df := &DataCoordFactory{}
|
|
|
|
|
|
|
|
replica := newReplica(&RootCoordFactory{}, test.collID)
|
|
|
|
if test.replicaNil {
|
|
|
|
replica = nil
|
|
|
|
}
|
|
|
|
|
|
|
|
ds, err := newDataSyncService(ctx,
|
2021-09-28 18:22:16 +08:00
|
|
|
&flushChans{make(chan *flushMsg), make(chan *flushMsg)},
|
2021-09-09 15:36:01 +08:00
|
|
|
replica,
|
|
|
|
NewAllocatorFactory(),
|
|
|
|
test.inMsgFactory,
|
2021-10-08 19:09:12 +08:00
|
|
|
getVchanInfo(test),
|
2021-09-09 15:36:01 +08:00
|
|
|
make(chan UniqueID),
|
|
|
|
df,
|
2021-09-23 16:03:54 +08:00
|
|
|
newCache(),
|
2021-09-09 15:36:01 +08:00
|
|
|
)
|
|
|
|
|
|
|
|
if !test.isValidCase {
|
|
|
|
assert.Error(t, err)
|
|
|
|
assert.Nil(t, ds)
|
|
|
|
} else {
|
|
|
|
assert.NoError(t, err)
|
|
|
|
assert.NotNil(t, ds)
|
|
|
|
|
|
|
|
// save binlog
|
|
|
|
fu := &segmentFlushUnit{
|
|
|
|
collID: 1,
|
|
|
|
segID: 100,
|
|
|
|
field2Path: map[UniqueID]string{100: "path1"},
|
|
|
|
checkPoint: map[UniqueID]segmentCheckPoint{100: {100, internalpb.MsgPosition{}}},
|
|
|
|
}
|
|
|
|
|
|
|
|
df.SaveBinlogPathError = true
|
|
|
|
err := ds.saveBinlog(fu)
|
|
|
|
assert.Error(t, err)
|
|
|
|
|
|
|
|
df.SaveBinlogPathError = false
|
|
|
|
df.SaveBinlogPathNotSucess = true
|
|
|
|
err = ds.saveBinlog(fu)
|
|
|
|
assert.Error(t, err)
|
|
|
|
|
|
|
|
df.SaveBinlogPathError = false
|
|
|
|
df.SaveBinlogPathNotSucess = false
|
|
|
|
err = ds.saveBinlog(fu)
|
|
|
|
assert.NoError(t, err)
|
|
|
|
|
|
|
|
// start
|
|
|
|
ds.fg = nil
|
|
|
|
ds.start()
|
|
|
|
}
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
2021-01-19 11:37:16 +08:00
|
|
|
// NOTE: start pulsar before test
|
|
|
|
func TestDataSyncService_Start(t *testing.T) {
|
|
|
|
const ctxTimeInMillisecond = 2000
|
2021-05-25 15:35:37 +08:00
|
|
|
|
|
|
|
delay := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
|
|
|
|
ctx, cancel := context.WithDeadline(context.Background(), delay)
|
|
|
|
defer cancel()
|
2021-01-19 11:37:16 +08:00
|
|
|
|
|
|
|
// init data node
|
|
|
|
pulsarURL := Params.PulsarAddress
|
2021-01-22 19:36:09 +08:00
|
|
|
|
2021-01-24 21:20:11 +08:00
|
|
|
Factory := &MetaFactory{}
|
2021-01-22 19:36:09 +08:00
|
|
|
collMeta := Factory.CollectionMetaFactory(UniqueID(0), "coll1")
|
2021-06-21 17:28:03 +08:00
|
|
|
mockRootCoord := &RootCoordFactory{}
|
2021-06-08 19:25:37 +08:00
|
|
|
collectionID := UniqueID(1)
|
2021-01-24 21:20:11 +08:00
|
|
|
|
2021-09-28 18:22:16 +08:00
|
|
|
flushChan := &flushChans{make(chan *flushMsg, 100), make(chan *flushMsg, 100)}
|
2021-06-21 17:28:03 +08:00
|
|
|
replica := newReplica(mockRootCoord, collectionID)
|
2021-06-08 19:25:37 +08:00
|
|
|
|
2021-05-25 15:35:37 +08:00
|
|
|
allocFactory := NewAllocatorFactory(1)
|
2021-04-02 13:48:25 +08:00
|
|
|
msFactory := msgstream.NewPmsFactory()
|
2021-02-08 14:30:54 +08:00
|
|
|
m := map[string]interface{}{
|
|
|
|
"pulsarAddress": pulsarURL,
|
|
|
|
"receiveBufSize": 1024,
|
|
|
|
"pulsarBufSize": 1024}
|
2021-06-16 19:03:57 +08:00
|
|
|
err := msFactory.SetParams(m)
|
2021-06-19 15:18:06 +08:00
|
|
|
assert.Nil(t, err)
|
2021-05-25 15:35:37 +08:00
|
|
|
|
|
|
|
insertChannelName := "data_sync_service_test_dml"
|
|
|
|
ddlChannelName := "data_sync_service_test_ddl"
|
|
|
|
Params.FlushInsertBufferSize = 1
|
|
|
|
|
2021-10-08 19:09:12 +08:00
|
|
|
ufs := []*datapb.SegmentInfo{{
|
|
|
|
CollectionID: collMeta.ID,
|
|
|
|
InsertChannel: insertChannelName,
|
|
|
|
ID: 0,
|
|
|
|
NumOfRows: 0,
|
|
|
|
DmlPosition: &internalpb.MsgPosition{},
|
|
|
|
}}
|
|
|
|
fs := []*datapb.SegmentInfo{{
|
|
|
|
CollectionID: collMeta.ID,
|
|
|
|
PartitionID: 1,
|
|
|
|
InsertChannel: insertChannelName,
|
|
|
|
ID: 1,
|
|
|
|
NumOfRows: 0,
|
|
|
|
DmlPosition: &internalpb.MsgPosition{},
|
|
|
|
}}
|
|
|
|
|
2021-06-04 09:57:54 +08:00
|
|
|
vchan := &datapb.VchannelInfo{
|
2021-10-08 19:09:12 +08:00
|
|
|
CollectionID: collMeta.ID,
|
2021-06-08 19:25:37 +08:00
|
|
|
ChannelName: insertChannelName,
|
2021-10-08 19:09:12 +08:00
|
|
|
UnflushedSegments: ufs,
|
|
|
|
FlushedSegments: fs,
|
2021-05-25 15:35:37 +08:00
|
|
|
}
|
|
|
|
|
2021-06-07 11:25:37 +08:00
|
|
|
signalCh := make(chan UniqueID, 100)
|
2021-09-23 16:03:54 +08:00
|
|
|
sync, err := newDataSyncService(ctx, flushChan, replica, allocFactory, msFactory, vchan, signalCh, &DataCoordFactory{}, newCache())
|
2021-05-25 15:35:37 +08:00
|
|
|
|
2021-06-19 15:18:06 +08:00
|
|
|
assert.Nil(t, err)
|
2021-06-08 19:25:37 +08:00
|
|
|
// sync.replica.addCollection(collMeta.ID, collMeta.Schema)
|
2021-09-29 20:19:59 +08:00
|
|
|
sync.start()
|
2021-01-19 11:37:16 +08:00
|
|
|
|
|
|
|
timeRange := TimeRange{
|
|
|
|
timestampMin: 0,
|
|
|
|
timestampMax: math.MaxUint64,
|
|
|
|
}
|
2021-02-03 17:30:10 +08:00
|
|
|
dataFactory := NewDataFactory()
|
2021-05-25 15:35:37 +08:00
|
|
|
insertMessages := dataFactory.GetMsgStreamTsInsertMsgs(2, insertChannelName)
|
2021-01-19 11:37:16 +08:00
|
|
|
|
|
|
|
msgPack := msgstream.MsgPack{
|
|
|
|
BeginTs: timeRange.timestampMin,
|
|
|
|
EndTs: timeRange.timestampMax,
|
|
|
|
Msgs: insertMessages,
|
2021-05-25 15:35:37 +08:00
|
|
|
StartPositions: []*internalpb.MsgPosition{{
|
|
|
|
ChannelName: insertChannelName,
|
|
|
|
}},
|
|
|
|
EndPositions: []*internalpb.MsgPosition{{
|
|
|
|
ChannelName: insertChannelName,
|
|
|
|
}},
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// generate timeTick
|
|
|
|
timeTickMsgPack := msgstream.MsgPack{}
|
|
|
|
|
|
|
|
timeTickMsg := &msgstream.TimeTickMsg{
|
|
|
|
BaseMsg: msgstream.BaseMsg{
|
|
|
|
BeginTimestamp: Timestamp(0),
|
|
|
|
EndTimestamp: Timestamp(0),
|
|
|
|
HashValues: []uint32{0},
|
|
|
|
},
|
2021-03-12 14:22:09 +08:00
|
|
|
TimeTickMsg: internalpb.TimeTickMsg{
|
2021-01-19 11:37:16 +08:00
|
|
|
Base: &commonpb.MsgBase{
|
2021-03-10 14:45:35 +08:00
|
|
|
MsgType: commonpb.MsgType_TimeTick,
|
2021-01-19 11:37:16 +08:00
|
|
|
MsgID: UniqueID(0),
|
|
|
|
Timestamp: math.MaxUint64,
|
|
|
|
SourceID: 0,
|
|
|
|
},
|
|
|
|
},
|
|
|
|
}
|
|
|
|
timeTickMsgPack.Msgs = append(timeTickMsgPack.Msgs, timeTickMsg)
|
|
|
|
|
|
|
|
// pulsar produce
|
2021-02-08 14:30:54 +08:00
|
|
|
assert.NoError(t, err)
|
|
|
|
insertStream, _ := msFactory.NewMsgStream(ctx)
|
2021-05-25 15:35:37 +08:00
|
|
|
insertStream.AsProducer([]string{insertChannelName})
|
2021-02-04 14:37:12 +08:00
|
|
|
|
2021-02-08 14:30:54 +08:00
|
|
|
ddStream, _ := msFactory.NewMsgStream(ctx)
|
2021-05-25 15:35:37 +08:00
|
|
|
ddStream.AsProducer([]string{ddlChannelName})
|
2021-01-19 11:37:16 +08:00
|
|
|
|
|
|
|
var insertMsgStream msgstream.MsgStream = insertStream
|
|
|
|
insertMsgStream.Start()
|
|
|
|
|
|
|
|
var ddMsgStream msgstream.MsgStream = ddStream
|
|
|
|
ddMsgStream.Start()
|
|
|
|
|
2021-03-25 14:41:46 +08:00
|
|
|
err = insertMsgStream.Produce(&msgPack)
|
2021-01-19 11:37:16 +08:00
|
|
|
assert.NoError(t, err)
|
|
|
|
|
2021-03-25 14:41:46 +08:00
|
|
|
err = insertMsgStream.Broadcast(&timeTickMsgPack)
|
2021-01-19 11:37:16 +08:00
|
|
|
assert.NoError(t, err)
|
2021-03-25 14:41:46 +08:00
|
|
|
err = ddMsgStream.Broadcast(&timeTickMsgPack)
|
2021-01-19 11:37:16 +08:00
|
|
|
assert.NoError(t, err)
|
|
|
|
|
|
|
|
// dataSync
|
2021-02-03 17:30:10 +08:00
|
|
|
<-sync.ctx.Done()
|
2021-01-19 11:37:16 +08:00
|
|
|
|
2021-01-24 21:20:11 +08:00
|
|
|
sync.close()
|
2021-01-19 11:37:16 +08:00
|
|
|
}
|