2021-12-28 15:43:51 +08:00
|
|
|
// Licensed to the LF AI & Data foundation under one
|
|
|
|
// or more contributor license agreements. See the NOTICE file
|
|
|
|
// distributed with this work for additional information
|
|
|
|
// regarding copyright ownership. The ASF licenses this file
|
|
|
|
// to you 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-11-05 22:25:00 +08:00
|
|
|
package datacoord
|
|
|
|
|
|
|
|
import (
|
2022-08-31 12:10:57 +08:00
|
|
|
"context"
|
2021-11-05 22:25:00 +08:00
|
|
|
"sync"
|
|
|
|
"testing"
|
|
|
|
"time"
|
|
|
|
|
2023-02-26 11:31:49 +08:00
|
|
|
"github.com/cockroachdb/errors"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/tsoutil"
|
2023-01-06 14:33:36 +08:00
|
|
|
"github.com/stretchr/testify/assert"
|
|
|
|
"github.com/stretchr/testify/mock"
|
|
|
|
"github.com/stretchr/testify/require"
|
|
|
|
"go.uber.org/zap"
|
2022-11-18 15:35:09 +08:00
|
|
|
|
2023-06-09 01:28:37 +08:00
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
2022-08-20 10:24:51 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/metastore/kv/datacoord"
|
2022-08-31 12:10:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/mocks"
|
2021-11-05 22:25:00 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/log"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/metautil"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
2023-07-07 19:38:26 +08:00
|
|
|
|
|
|
|
mockkv "github.com/milvus-io/milvus/internal/kv/mocks"
|
2021-11-05 22:25:00 +08:00
|
|
|
)
|
|
|
|
|
|
|
|
func Test_compactionPlanHandler_execCompactionPlan(t *testing.T) {
|
|
|
|
type fields struct {
|
2023-02-07 19:02:31 +08:00
|
|
|
plans map[int64]*compactionTask
|
|
|
|
sessions *SessionManager
|
|
|
|
chManager *ChannelManager
|
|
|
|
allocatorFactory func() allocator
|
2021-11-05 22:25:00 +08:00
|
|
|
}
|
|
|
|
type args struct {
|
2021-11-11 15:54:42 +08:00
|
|
|
signal *compactionSignal
|
|
|
|
plan *datapb.CompactionPlan
|
2021-11-05 22:25:00 +08:00
|
|
|
}
|
|
|
|
tests := []struct {
|
|
|
|
name string
|
|
|
|
fields fields
|
|
|
|
args args
|
|
|
|
wantErr bool
|
|
|
|
err error
|
|
|
|
}{
|
|
|
|
{
|
|
|
|
"test exec compaction",
|
|
|
|
fields{
|
|
|
|
plans: map[int64]*compactionTask{},
|
|
|
|
sessions: &SessionManager{
|
|
|
|
sessions: struct {
|
|
|
|
sync.RWMutex
|
|
|
|
data map[int64]*Session
|
|
|
|
}{
|
|
|
|
data: map[int64]*Session{
|
2023-02-15 16:00:33 +08:00
|
|
|
1: {client: &mockDataNodeClient{ch: make(chan interface{}, 1)}},
|
2021-11-05 22:25:00 +08:00
|
|
|
},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
chManager: &ChannelManager{
|
|
|
|
store: &ChannelStore{
|
|
|
|
channelsInfo: map[int64]*NodeChannelInfo{
|
|
|
|
1: {NodeID: 1, Channels: []*channel{{Name: "ch1"}}},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
},
|
2023-02-07 19:02:31 +08:00
|
|
|
allocatorFactory: func() allocator { return newMockAllocator() },
|
2021-11-05 22:25:00 +08:00
|
|
|
},
|
|
|
|
args{
|
2021-11-11 15:54:42 +08:00
|
|
|
signal: &compactionSignal{id: 100},
|
|
|
|
plan: &datapb.CompactionPlan{PlanID: 1, Channel: "ch1", Type: datapb.CompactionType_MergeCompaction},
|
2021-11-05 22:25:00 +08:00
|
|
|
},
|
|
|
|
false,
|
|
|
|
nil,
|
|
|
|
},
|
2022-08-26 14:22:55 +08:00
|
|
|
{
|
|
|
|
"test exec compaction failed",
|
|
|
|
fields{
|
|
|
|
plans: map[int64]*compactionTask{},
|
|
|
|
sessions: &SessionManager{
|
|
|
|
sessions: struct {
|
|
|
|
sync.RWMutex
|
|
|
|
data map[int64]*Session
|
|
|
|
}{
|
|
|
|
data: map[int64]*Session{
|
2023-02-15 16:00:33 +08:00
|
|
|
1: {client: &mockDataNodeClient{ch: make(chan interface{}, 1), compactionResp: &commonpb.Status{ErrorCode: commonpb.ErrorCode_CacheFailed}}},
|
2022-08-26 14:22:55 +08:00
|
|
|
},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
chManager: &ChannelManager{
|
|
|
|
store: &ChannelStore{
|
|
|
|
channelsInfo: map[int64]*NodeChannelInfo{
|
|
|
|
1: {NodeID: 1, Channels: []*channel{{Name: "ch1"}}},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
},
|
2023-02-07 19:02:31 +08:00
|
|
|
allocatorFactory: func() allocator { return newMockAllocator() },
|
|
|
|
},
|
|
|
|
args{
|
|
|
|
signal: &compactionSignal{id: 100},
|
|
|
|
plan: &datapb.CompactionPlan{PlanID: 1, Channel: "ch1", Type: datapb.CompactionType_MergeCompaction},
|
|
|
|
},
|
|
|
|
true,
|
|
|
|
nil,
|
|
|
|
},
|
|
|
|
{
|
|
|
|
"test_allocate_ts_failed",
|
|
|
|
fields{
|
|
|
|
plans: map[int64]*compactionTask{},
|
|
|
|
sessions: &SessionManager{
|
|
|
|
sessions: struct {
|
|
|
|
sync.RWMutex
|
|
|
|
data map[int64]*Session
|
|
|
|
}{
|
|
|
|
data: map[int64]*Session{
|
2023-02-15 16:00:33 +08:00
|
|
|
1: {client: &mockDataNodeClient{ch: make(chan interface{}, 1), compactionResp: &commonpb.Status{ErrorCode: commonpb.ErrorCode_CacheFailed}}},
|
2023-02-07 19:02:31 +08:00
|
|
|
},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
chManager: &ChannelManager{
|
|
|
|
store: &ChannelStore{
|
|
|
|
channelsInfo: map[int64]*NodeChannelInfo{
|
|
|
|
1: {NodeID: 1, Channels: []*channel{{Name: "ch1"}}},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
allocatorFactory: func() allocator {
|
|
|
|
a := &NMockAllocator{}
|
|
|
|
start := time.Now()
|
|
|
|
a.EXPECT().allocTimestamp(mock.Anything).Call.Return(func(_ context.Context) Timestamp {
|
|
|
|
return tsoutil.ComposeTSByTime(time.Now(), 0)
|
|
|
|
}, func(_ context.Context) error {
|
|
|
|
if time.Since(start) > time.Second*2 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
return errors.New("mocked")
|
|
|
|
})
|
|
|
|
return a
|
|
|
|
},
|
2022-08-26 14:22:55 +08:00
|
|
|
},
|
|
|
|
args{
|
|
|
|
signal: &compactionSignal{id: 100},
|
|
|
|
plan: &datapb.CompactionPlan{PlanID: 1, Channel: "ch1", Type: datapb.CompactionType_MergeCompaction},
|
|
|
|
},
|
|
|
|
true,
|
|
|
|
nil,
|
|
|
|
},
|
2021-11-05 22:25:00 +08:00
|
|
|
}
|
|
|
|
for _, tt := range tests {
|
|
|
|
t.Run(tt.name, func(t *testing.T) {
|
|
|
|
c := &compactionPlanHandler{
|
2022-08-23 15:50:52 +08:00
|
|
|
plans: tt.fields.plans,
|
|
|
|
sessions: tt.fields.sessions,
|
|
|
|
chManager: tt.fields.chManager,
|
|
|
|
parallelCh: make(map[int64]chan struct{}),
|
2023-02-07 19:02:31 +08:00
|
|
|
allocator: tt.fields.allocatorFactory(),
|
2021-11-05 22:25:00 +08:00
|
|
|
}
|
2022-12-07 18:01:19 +08:00
|
|
|
Params.Save(Params.DataCoordCfg.CompactionCheckIntervalInSeconds.Key, "1")
|
2022-12-06 20:09:18 +08:00
|
|
|
c.start()
|
2021-11-11 15:54:42 +08:00
|
|
|
err := c.execCompactionPlan(tt.args.signal, tt.args.plan)
|
2021-11-05 22:25:00 +08:00
|
|
|
assert.Equal(t, tt.err, err)
|
|
|
|
if err == nil {
|
|
|
|
task := c.getCompaction(tt.args.plan.PlanID)
|
2022-08-26 14:22:55 +08:00
|
|
|
if !tt.wantErr {
|
|
|
|
assert.Equal(t, tt.args.plan, task.plan)
|
|
|
|
assert.Equal(t, tt.args.signal, task.triggerInfo)
|
|
|
|
assert.Equal(t, 1, c.executingTaskNum)
|
|
|
|
} else {
|
2022-12-06 20:09:18 +08:00
|
|
|
|
2022-08-26 14:22:55 +08:00
|
|
|
assert.Eventually(t,
|
|
|
|
func() bool {
|
|
|
|
c.mu.RLock()
|
|
|
|
defer c.mu.RUnlock()
|
|
|
|
return c.executingTaskNum == 0 && len(c.parallelCh[1]) == 0
|
|
|
|
},
|
|
|
|
5*time.Second, 100*time.Millisecond)
|
|
|
|
}
|
2021-11-05 22:25:00 +08:00
|
|
|
}
|
2022-12-06 20:09:18 +08:00
|
|
|
c.stop()
|
2021-11-05 22:25:00 +08:00
|
|
|
})
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-08-23 15:50:52 +08:00
|
|
|
func Test_compactionPlanHandler_execWithParallels(t *testing.T) {
|
2022-08-31 12:10:57 +08:00
|
|
|
|
2023-07-14 10:12:31 +08:00
|
|
|
mockDataNode := &mocks.MockDataNode{}
|
2022-12-07 18:01:19 +08:00
|
|
|
paramtable.Get().Save(Params.DataCoordCfg.CompactionCheckIntervalInSeconds.Key, "1")
|
|
|
|
defer paramtable.Get().Reset(Params.DataCoordCfg.CompactionCheckIntervalInSeconds.Key)
|
2022-08-23 15:50:52 +08:00
|
|
|
c := &compactionPlanHandler{
|
|
|
|
plans: map[int64]*compactionTask{},
|
|
|
|
sessions: &SessionManager{
|
|
|
|
sessions: struct {
|
|
|
|
sync.RWMutex
|
|
|
|
data map[int64]*Session
|
|
|
|
}{
|
|
|
|
data: map[int64]*Session{
|
2022-08-31 12:10:57 +08:00
|
|
|
1: {client: mockDataNode},
|
2022-08-23 15:50:52 +08:00
|
|
|
},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
chManager: &ChannelManager{
|
|
|
|
store: &ChannelStore{
|
|
|
|
channelsInfo: map[int64]*NodeChannelInfo{
|
|
|
|
1: {NodeID: 1, Channels: []*channel{{Name: "ch1"}}},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
parallelCh: make(map[int64]chan struct{}),
|
|
|
|
allocator: newMockAllocator(),
|
|
|
|
}
|
|
|
|
|
|
|
|
signal := &compactionSignal{id: 100}
|
|
|
|
plan1 := &datapb.CompactionPlan{PlanID: 1, Channel: "ch1", Type: datapb.CompactionType_MergeCompaction}
|
|
|
|
plan2 := &datapb.CompactionPlan{PlanID: 2, Channel: "ch1", Type: datapb.CompactionType_MergeCompaction}
|
|
|
|
plan3 := &datapb.CompactionPlan{PlanID: 3, Channel: "ch1", Type: datapb.CompactionType_MergeCompaction}
|
|
|
|
|
|
|
|
c.parallelCh[1] = make(chan struct{}, 2)
|
|
|
|
|
2022-08-31 12:10:57 +08:00
|
|
|
var mut sync.RWMutex
|
|
|
|
called := 0
|
|
|
|
|
|
|
|
mockDataNode.EXPECT().Compaction(mock.Anything, mock.Anything).Run(func(ctx context.Context, req *datapb.CompactionPlan) {
|
|
|
|
mut.Lock()
|
|
|
|
defer mut.Unlock()
|
|
|
|
called++
|
|
|
|
}).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil).Times(3)
|
2022-08-23 15:50:52 +08:00
|
|
|
go func() {
|
|
|
|
c.execCompactionPlan(signal, plan1)
|
|
|
|
c.execCompactionPlan(signal, plan2)
|
|
|
|
c.execCompactionPlan(signal, plan3)
|
|
|
|
}()
|
|
|
|
|
2022-08-31 12:10:57 +08:00
|
|
|
// wait for dispatch signal
|
2022-08-23 15:50:52 +08:00
|
|
|
<-c.parallelCh[1]
|
|
|
|
<-c.parallelCh[1]
|
|
|
|
<-c.parallelCh[1]
|
|
|
|
|
2022-08-31 12:10:57 +08:00
|
|
|
// wait for compaction called
|
|
|
|
assert.Eventually(t, func() bool {
|
|
|
|
mut.RLock()
|
|
|
|
defer mut.RUnlock()
|
|
|
|
return called == 3
|
|
|
|
}, time.Second, time.Millisecond*10)
|
|
|
|
|
2022-08-23 15:50:52 +08:00
|
|
|
tasks := c.getCompactionTasksBySignalID(0)
|
|
|
|
max, min := uint64(0), uint64(0)
|
|
|
|
for _, v := range tasks {
|
|
|
|
if max < v.plan.GetStartTime() {
|
|
|
|
max = v.plan.GetStartTime()
|
|
|
|
}
|
|
|
|
if min > v.plan.GetStartTime() {
|
|
|
|
min = v.plan.GetStartTime()
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Debug("start time", zap.Uint64("min", min), zap.Uint64("max", max))
|
|
|
|
assert.Less(t, uint64(2), max-min)
|
|
|
|
}
|
|
|
|
|
2022-11-18 15:35:09 +08:00
|
|
|
func getInsertLogPath(rootPath string, segmentID typeutil.UniqueID) string {
|
|
|
|
return metautil.BuildInsertLogPath(rootPath, 10, 100, segmentID, 1000, 10000)
|
|
|
|
}
|
|
|
|
|
|
|
|
func getStatsLogPath(rootPath string, segmentID typeutil.UniqueID) string {
|
|
|
|
return metautil.BuildStatsLogPath(rootPath, 10, 100, segmentID, 1000, 10000)
|
|
|
|
}
|
|
|
|
|
|
|
|
func getDeltaLogPath(rootPath string, segmentID typeutil.UniqueID) string {
|
|
|
|
return metautil.BuildDeltaLogPath(rootPath, 10, 100, segmentID, 10000)
|
|
|
|
}
|
|
|
|
|
2022-09-27 16:02:53 +08:00
|
|
|
func TestCompactionPlanHandler_handleMergeCompactionResult(t *testing.T) {
|
2023-07-14 10:12:31 +08:00
|
|
|
mockDataNode := &mocks.MockDataNode{}
|
2023-03-17 17:27:56 +08:00
|
|
|
call := mockDataNode.EXPECT().SyncSegments(mock.Anything, mock.Anything).Run(func(ctx context.Context, req *datapb.SyncSegmentsRequest) {}).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil)
|
2022-09-27 16:02:53 +08:00
|
|
|
|
|
|
|
dataNodeID := UniqueID(111)
|
|
|
|
|
|
|
|
seg1 := &datapb.SegmentInfo{
|
|
|
|
ID: 1,
|
2022-11-18 15:35:09 +08:00
|
|
|
Binlogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log1", 1))},
|
|
|
|
Statslogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log2", 1))},
|
|
|
|
Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log3", 1))},
|
2021-11-05 22:25:00 +08:00
|
|
|
}
|
2022-09-27 16:02:53 +08:00
|
|
|
|
|
|
|
seg2 := &datapb.SegmentInfo{
|
|
|
|
ID: 2,
|
2022-11-18 15:35:09 +08:00
|
|
|
Binlogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log4", 2))},
|
|
|
|
Statslogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log5", 2))},
|
|
|
|
Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log6", 2))},
|
2021-11-05 22:25:00 +08:00
|
|
|
}
|
2022-09-27 16:02:53 +08:00
|
|
|
|
|
|
|
plan := &datapb.CompactionPlan{
|
|
|
|
PlanID: 1,
|
|
|
|
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
|
|
|
|
{
|
|
|
|
SegmentID: seg1.ID,
|
|
|
|
FieldBinlogs: seg1.GetBinlogs(),
|
|
|
|
Field2StatslogPaths: seg1.GetStatslogs(),
|
|
|
|
Deltalogs: seg1.GetDeltalogs(),
|
2021-11-05 22:25:00 +08:00
|
|
|
},
|
2022-09-27 16:02:53 +08:00
|
|
|
{
|
|
|
|
SegmentID: seg2.ID,
|
|
|
|
FieldBinlogs: seg2.GetBinlogs(),
|
|
|
|
Field2StatslogPaths: seg2.GetStatslogs(),
|
|
|
|
Deltalogs: seg2.GetDeltalogs(),
|
2021-11-05 22:25:00 +08:00
|
|
|
},
|
|
|
|
},
|
2022-09-27 16:02:53 +08:00
|
|
|
Type: datapb.CompactionType_MergeCompaction,
|
|
|
|
}
|
|
|
|
|
|
|
|
sessions := &SessionManager{
|
|
|
|
sessions: struct {
|
|
|
|
sync.RWMutex
|
|
|
|
data map[int64]*Session
|
|
|
|
}{
|
|
|
|
data: map[int64]*Session{
|
|
|
|
dataNodeID: {client: mockDataNode}},
|
2021-11-05 22:25:00 +08:00
|
|
|
},
|
2022-09-27 16:02:53 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
task := &compactionTask{
|
|
|
|
triggerInfo: &compactionSignal{id: 1},
|
|
|
|
state: executing,
|
|
|
|
plan: plan,
|
|
|
|
dataNodeID: dataNodeID,
|
|
|
|
}
|
|
|
|
|
|
|
|
plans := map[int64]*compactionTask{1: task}
|
|
|
|
|
2023-07-07 19:38:26 +08:00
|
|
|
metakv := mockkv.NewMetaKv(t)
|
|
|
|
metakv.EXPECT().Save(mock.Anything, mock.Anything).Return(errors.New("failed")).Maybe()
|
|
|
|
metakv.EXPECT().MultiSave(mock.Anything).Return(errors.New("failed")).Maybe()
|
2023-07-11 19:38:28 +08:00
|
|
|
metakv.EXPECT().HasPrefix(mock.Anything).Return(false, nil).Maybe()
|
2022-10-26 16:49:31 +08:00
|
|
|
errMeta := &meta{
|
2023-07-07 19:38:26 +08:00
|
|
|
catalog: &datacoord.Catalog{MetaKv: metakv},
|
2022-10-26 16:49:31 +08:00
|
|
|
segments: &SegmentsInfo{
|
|
|
|
map[int64]*SegmentInfo{
|
|
|
|
seg1.ID: {SegmentInfo: seg1},
|
|
|
|
seg2.ID: {SegmentInfo: seg2},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
}
|
|
|
|
|
2022-09-27 16:02:53 +08:00
|
|
|
meta := &meta{
|
2023-01-06 14:33:36 +08:00
|
|
|
catalog: &datacoord.Catalog{MetaKv: NewMetaMemoryKV()},
|
2022-09-27 16:02:53 +08:00
|
|
|
segments: &SegmentsInfo{
|
|
|
|
map[int64]*SegmentInfo{
|
|
|
|
seg1.ID: {SegmentInfo: seg1},
|
|
|
|
seg2.ID: {SegmentInfo: seg2},
|
2021-11-11 15:54:42 +08:00
|
|
|
},
|
2021-11-05 22:25:00 +08:00
|
|
|
},
|
|
|
|
}
|
2022-06-20 21:56:12 +08:00
|
|
|
|
2022-09-27 16:02:53 +08:00
|
|
|
c := &compactionPlanHandler{
|
|
|
|
plans: plans,
|
|
|
|
sessions: sessions,
|
|
|
|
meta: meta,
|
2022-06-20 21:56:12 +08:00
|
|
|
}
|
2022-09-27 16:02:53 +08:00
|
|
|
|
2022-10-26 16:49:31 +08:00
|
|
|
c2 := &compactionPlanHandler{
|
|
|
|
plans: plans,
|
|
|
|
sessions: sessions,
|
|
|
|
meta: errMeta,
|
|
|
|
}
|
|
|
|
|
2022-09-27 16:02:53 +08:00
|
|
|
compactionResult := &datapb.CompactionResult{
|
|
|
|
PlanID: 1,
|
|
|
|
SegmentID: 3,
|
|
|
|
NumOfRows: 15,
|
2022-11-18 15:35:09 +08:00
|
|
|
InsertLogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log301", 3))},
|
|
|
|
Field2StatslogPaths: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log302", 3))},
|
|
|
|
Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log303", 3))},
|
2022-06-20 21:56:12 +08:00
|
|
|
}
|
2022-09-27 16:02:53 +08:00
|
|
|
|
2022-10-26 16:49:31 +08:00
|
|
|
compactionResult2 := &datapb.CompactionResult{
|
|
|
|
PlanID: 1,
|
|
|
|
SegmentID: 3,
|
|
|
|
NumOfRows: 0,
|
2022-11-18 15:35:09 +08:00
|
|
|
InsertLogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log301", 3))},
|
|
|
|
Field2StatslogPaths: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log302", 3))},
|
|
|
|
Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log303", 3))},
|
2022-10-26 16:49:31 +08:00
|
|
|
}
|
|
|
|
|
2022-09-27 16:02:53 +08:00
|
|
|
has, err := c.meta.HasSegments([]UniqueID{1, 2})
|
|
|
|
require.NoError(t, err)
|
|
|
|
require.True(t, has)
|
|
|
|
|
|
|
|
has, err = c.meta.HasSegments([]UniqueID{3})
|
|
|
|
require.Error(t, err)
|
|
|
|
require.False(t, has)
|
|
|
|
|
|
|
|
err = c.handleMergeCompactionResult(plan, compactionResult)
|
|
|
|
assert.NoError(t, err)
|
|
|
|
|
2022-10-26 16:49:31 +08:00
|
|
|
err = c.handleMergeCompactionResult(plan, compactionResult2)
|
|
|
|
assert.NoError(t, err)
|
|
|
|
|
|
|
|
err = c2.handleMergeCompactionResult(plan, compactionResult2)
|
|
|
|
assert.Error(t, err)
|
|
|
|
|
2022-09-27 16:02:53 +08:00
|
|
|
has, err = c.meta.HasSegments([]UniqueID{1, 2, 3})
|
|
|
|
require.NoError(t, err)
|
|
|
|
require.True(t, has)
|
2023-03-17 17:27:56 +08:00
|
|
|
|
|
|
|
call.Unset()
|
|
|
|
call = mockDataNode.EXPECT().SyncSegments(mock.Anything, mock.Anything).Run(func(ctx context.Context, req *datapb.SyncSegmentsRequest) {}).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_UnexpectedError}, nil)
|
|
|
|
err = c.handleMergeCompactionResult(plan, compactionResult2)
|
|
|
|
assert.Error(t, err)
|
2022-09-27 16:02:53 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func TestCompactionPlanHandler_completeCompaction(t *testing.T) {
|
|
|
|
t.Run("test not exists compaction task", func(t *testing.T) {
|
|
|
|
c := &compactionPlanHandler{
|
|
|
|
plans: map[int64]*compactionTask{1: {}},
|
|
|
|
}
|
|
|
|
err := c.completeCompaction(&datapb.CompactionResult{PlanID: 2})
|
|
|
|
assert.Error(t, err)
|
|
|
|
})
|
|
|
|
t.Run("test completed compaction task", func(t *testing.T) {
|
|
|
|
c := &compactionPlanHandler{
|
|
|
|
plans: map[int64]*compactionTask{1: {state: completed}},
|
|
|
|
}
|
|
|
|
err := c.completeCompaction(&datapb.CompactionResult{PlanID: 1})
|
|
|
|
assert.Error(t, err)
|
|
|
|
})
|
|
|
|
|
|
|
|
t.Run("test complete merge compaction task", func(t *testing.T) {
|
2023-07-14 10:12:31 +08:00
|
|
|
mockDataNode := &mocks.MockDataNode{}
|
2022-09-27 16:02:53 +08:00
|
|
|
mockDataNode.EXPECT().SyncSegments(mock.Anything, mock.Anything).Run(func(ctx context.Context, req *datapb.SyncSegmentsRequest) {}).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil)
|
|
|
|
|
|
|
|
dataNodeID := UniqueID(111)
|
|
|
|
|
|
|
|
seg1 := &datapb.SegmentInfo{
|
|
|
|
ID: 1,
|
2022-11-18 15:35:09 +08:00
|
|
|
Binlogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log1", 1))},
|
|
|
|
Statslogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log2", 1))},
|
|
|
|
Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log3", 1))},
|
2022-09-27 16:02:53 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
seg2 := &datapb.SegmentInfo{
|
|
|
|
ID: 2,
|
2022-11-18 15:35:09 +08:00
|
|
|
Binlogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log4", 2))},
|
|
|
|
Statslogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log5", 2))},
|
|
|
|
Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log6", 2))},
|
2022-09-27 16:02:53 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
plan := &datapb.CompactionPlan{
|
|
|
|
PlanID: 1,
|
|
|
|
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
|
|
|
|
{
|
|
|
|
SegmentID: seg1.ID,
|
|
|
|
FieldBinlogs: seg1.GetBinlogs(),
|
|
|
|
Field2StatslogPaths: seg1.GetStatslogs(),
|
|
|
|
Deltalogs: seg1.GetDeltalogs(),
|
2022-06-20 21:56:12 +08:00
|
|
|
},
|
2022-09-27 16:02:53 +08:00
|
|
|
{
|
|
|
|
SegmentID: seg2.ID,
|
|
|
|
FieldBinlogs: seg2.GetBinlogs(),
|
|
|
|
Field2StatslogPaths: seg2.GetStatslogs(),
|
|
|
|
Deltalogs: seg2.GetDeltalogs(),
|
2022-06-20 21:56:12 +08:00
|
|
|
},
|
|
|
|
},
|
2022-09-27 16:02:53 +08:00
|
|
|
Type: datapb.CompactionType_MergeCompaction,
|
|
|
|
}
|
|
|
|
|
|
|
|
sessions := &SessionManager{
|
|
|
|
sessions: struct {
|
|
|
|
sync.RWMutex
|
|
|
|
data map[int64]*Session
|
|
|
|
}{
|
|
|
|
data: map[int64]*Session{
|
|
|
|
dataNodeID: {client: mockDataNode}},
|
2022-06-20 21:56:12 +08:00
|
|
|
},
|
2022-09-27 16:02:53 +08:00
|
|
|
}
|
2022-06-20 21:56:12 +08:00
|
|
|
|
2022-09-27 16:02:53 +08:00
|
|
|
task := &compactionTask{
|
|
|
|
triggerInfo: &compactionSignal{id: 1},
|
|
|
|
state: executing,
|
|
|
|
plan: plan,
|
|
|
|
dataNodeID: dataNodeID,
|
|
|
|
}
|
|
|
|
|
|
|
|
plans := map[int64]*compactionTask{1: task}
|
|
|
|
|
|
|
|
meta := &meta{
|
2023-01-06 14:33:36 +08:00
|
|
|
catalog: &datacoord.Catalog{MetaKv: NewMetaMemoryKV()},
|
2022-09-27 16:02:53 +08:00
|
|
|
segments: &SegmentsInfo{
|
|
|
|
map[int64]*SegmentInfo{
|
|
|
|
seg1.ID: {SegmentInfo: seg1},
|
|
|
|
seg2.ID: {SegmentInfo: seg2},
|
2022-06-20 21:56:12 +08:00
|
|
|
},
|
2022-09-27 16:02:53 +08:00
|
|
|
},
|
|
|
|
}
|
|
|
|
compactionResult := datapb.CompactionResult{
|
|
|
|
PlanID: 1,
|
|
|
|
SegmentID: 3,
|
|
|
|
NumOfRows: 15,
|
2022-11-18 15:35:09 +08:00
|
|
|
InsertLogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log301", 3))},
|
|
|
|
Field2StatslogPaths: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log302", 3))},
|
|
|
|
Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log303", 3))},
|
2022-09-27 16:02:53 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
flushCh := make(chan UniqueID, 1)
|
|
|
|
c := &compactionPlanHandler{
|
|
|
|
plans: plans,
|
|
|
|
sessions: sessions,
|
|
|
|
meta: meta,
|
|
|
|
flushCh: flushCh,
|
|
|
|
}
|
|
|
|
|
|
|
|
err := c.completeCompaction(&compactionResult)
|
|
|
|
|
|
|
|
segID, ok := <-flushCh
|
|
|
|
assert.True(t, ok)
|
|
|
|
assert.Equal(t, compactionResult.GetSegmentID(), segID)
|
|
|
|
assert.NoError(t, err)
|
|
|
|
})
|
2022-10-27 17:15:32 +08:00
|
|
|
|
|
|
|
t.Run("test empty result merge compaction task", func(t *testing.T) {
|
2023-07-14 10:12:31 +08:00
|
|
|
mockDataNode := &mocks.MockDataNode{}
|
2022-10-27 17:15:32 +08:00
|
|
|
mockDataNode.EXPECT().SyncSegments(mock.Anything, mock.Anything).Run(func(ctx context.Context, req *datapb.SyncSegmentsRequest) {}).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil)
|
|
|
|
|
|
|
|
dataNodeID := UniqueID(111)
|
|
|
|
|
|
|
|
seg1 := &datapb.SegmentInfo{
|
|
|
|
ID: 1,
|
2022-11-18 15:35:09 +08:00
|
|
|
Binlogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log1", 1))},
|
|
|
|
Statslogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log2", 1))},
|
|
|
|
Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log3", 1))},
|
2022-10-27 17:15:32 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
seg2 := &datapb.SegmentInfo{
|
|
|
|
ID: 2,
|
2022-11-18 15:35:09 +08:00
|
|
|
Binlogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log4", 2))},
|
|
|
|
Statslogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log5", 2))},
|
|
|
|
Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log6", 2))},
|
2022-10-27 17:15:32 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
plan := &datapb.CompactionPlan{
|
|
|
|
PlanID: 1,
|
|
|
|
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
|
|
|
|
{
|
|
|
|
SegmentID: seg1.ID,
|
|
|
|
FieldBinlogs: seg1.GetBinlogs(),
|
|
|
|
Field2StatslogPaths: seg1.GetStatslogs(),
|
|
|
|
Deltalogs: seg1.GetDeltalogs(),
|
|
|
|
},
|
|
|
|
{
|
|
|
|
SegmentID: seg2.ID,
|
|
|
|
FieldBinlogs: seg2.GetBinlogs(),
|
|
|
|
Field2StatslogPaths: seg2.GetStatslogs(),
|
|
|
|
Deltalogs: seg2.GetDeltalogs(),
|
|
|
|
},
|
|
|
|
},
|
|
|
|
Type: datapb.CompactionType_MergeCompaction,
|
|
|
|
}
|
|
|
|
|
|
|
|
sessions := &SessionManager{
|
|
|
|
sessions: struct {
|
|
|
|
sync.RWMutex
|
|
|
|
data map[int64]*Session
|
|
|
|
}{
|
|
|
|
data: map[int64]*Session{
|
|
|
|
dataNodeID: {client: mockDataNode}},
|
|
|
|
},
|
|
|
|
}
|
|
|
|
|
|
|
|
task := &compactionTask{
|
|
|
|
triggerInfo: &compactionSignal{id: 1},
|
|
|
|
state: executing,
|
|
|
|
plan: plan,
|
|
|
|
dataNodeID: dataNodeID,
|
|
|
|
}
|
|
|
|
|
|
|
|
plans := map[int64]*compactionTask{1: task}
|
|
|
|
|
|
|
|
meta := &meta{
|
2023-01-06 14:33:36 +08:00
|
|
|
catalog: &datacoord.Catalog{MetaKv: NewMetaMemoryKV()},
|
2022-10-27 17:15:32 +08:00
|
|
|
segments: &SegmentsInfo{
|
|
|
|
map[int64]*SegmentInfo{
|
|
|
|
seg1.ID: {SegmentInfo: seg1},
|
|
|
|
seg2.ID: {SegmentInfo: seg2},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
}
|
|
|
|
|
|
|
|
meta.AddSegment(NewSegmentInfo(seg1))
|
|
|
|
meta.AddSegment(NewSegmentInfo(seg2))
|
|
|
|
|
|
|
|
segments := meta.GetAllSegmentsUnsafe()
|
|
|
|
assert.Equal(t, len(segments), 2)
|
|
|
|
compactionResult := datapb.CompactionResult{
|
|
|
|
PlanID: 1,
|
|
|
|
SegmentID: 3,
|
|
|
|
NumOfRows: 0,
|
2022-11-18 15:35:09 +08:00
|
|
|
InsertLogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getInsertLogPath("log301", 3))},
|
|
|
|
Field2StatslogPaths: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getStatsLogPath("log302", 3))},
|
|
|
|
Deltalogs: []*datapb.FieldBinlog{getFieldBinlogPaths(101, getDeltaLogPath("log303", 3))},
|
2022-10-27 17:15:32 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
flushCh := make(chan UniqueID, 1)
|
|
|
|
c := &compactionPlanHandler{
|
|
|
|
plans: plans,
|
|
|
|
sessions: sessions,
|
|
|
|
meta: meta,
|
|
|
|
flushCh: flushCh,
|
|
|
|
}
|
|
|
|
|
|
|
|
err := c.completeCompaction(&compactionResult)
|
2023-07-21 14:21:05 +08:00
|
|
|
assert.NoError(t, err)
|
2022-10-27 17:15:32 +08:00
|
|
|
|
|
|
|
segID, ok := <-flushCh
|
|
|
|
assert.True(t, ok)
|
|
|
|
assert.Equal(t, compactionResult.GetSegmentID(), segID)
|
|
|
|
assert.NoError(t, err)
|
|
|
|
|
|
|
|
segments = meta.GetAllSegmentsUnsafe()
|
2023-03-03 14:13:49 +08:00
|
|
|
assert.Equal(t, len(segments), 3)
|
2022-10-27 17:15:32 +08:00
|
|
|
|
|
|
|
for _, segment := range segments {
|
|
|
|
assert.True(t, segment.State == commonpb.SegmentState_Dropped)
|
|
|
|
}
|
|
|
|
})
|
2021-11-05 22:25:00 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func Test_compactionPlanHandler_getCompaction(t *testing.T) {
|
|
|
|
type fields struct {
|
|
|
|
plans map[int64]*compactionTask
|
|
|
|
sessions *SessionManager
|
|
|
|
}
|
|
|
|
type args struct {
|
|
|
|
planID int64
|
|
|
|
}
|
|
|
|
tests := []struct {
|
|
|
|
name string
|
|
|
|
fields fields
|
|
|
|
args args
|
|
|
|
want *compactionTask
|
|
|
|
}{
|
|
|
|
{
|
|
|
|
"test get non existed task",
|
|
|
|
fields{plans: map[int64]*compactionTask{}},
|
|
|
|
args{planID: 1},
|
|
|
|
nil,
|
|
|
|
},
|
|
|
|
{
|
|
|
|
"test get existed task",
|
|
|
|
fields{
|
|
|
|
plans: map[int64]*compactionTask{1: {
|
|
|
|
state: executing,
|
|
|
|
}},
|
|
|
|
},
|
|
|
|
args{planID: 1},
|
|
|
|
&compactionTask{
|
|
|
|
state: executing,
|
|
|
|
},
|
|
|
|
},
|
|
|
|
}
|
|
|
|
for _, tt := range tests {
|
|
|
|
t.Run(tt.name, func(t *testing.T) {
|
|
|
|
c := &compactionPlanHandler{
|
|
|
|
plans: tt.fields.plans,
|
|
|
|
sessions: tt.fields.sessions,
|
|
|
|
}
|
|
|
|
got := c.getCompaction(tt.args.planID)
|
|
|
|
assert.EqualValues(t, tt.want, got)
|
|
|
|
})
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-08-23 15:50:52 +08:00
|
|
|
func Test_compactionPlanHandler_updateCompaction(t *testing.T) {
|
2021-11-05 22:25:00 +08:00
|
|
|
type fields struct {
|
|
|
|
plans map[int64]*compactionTask
|
|
|
|
sessions *SessionManager
|
|
|
|
meta *meta
|
|
|
|
}
|
|
|
|
type args struct {
|
|
|
|
ts Timestamp
|
|
|
|
}
|
|
|
|
|
|
|
|
ts := time.Now()
|
|
|
|
tests := []struct {
|
|
|
|
name string
|
|
|
|
fields fields
|
|
|
|
args args
|
|
|
|
wantErr bool
|
2022-08-26 14:22:55 +08:00
|
|
|
timeout []int64
|
|
|
|
failed []int64
|
2021-11-05 22:25:00 +08:00
|
|
|
unexpired []int64
|
|
|
|
}{
|
|
|
|
{
|
2022-08-23 15:50:52 +08:00
|
|
|
"test update compaction task",
|
2021-11-05 22:25:00 +08:00
|
|
|
fields{
|
|
|
|
plans: map[int64]*compactionTask{
|
|
|
|
1: {
|
2022-08-23 15:50:52 +08:00
|
|
|
state: executing,
|
|
|
|
dataNodeID: 1,
|
2021-11-05 22:25:00 +08:00
|
|
|
plan: &datapb.CompactionPlan{
|
|
|
|
PlanID: 1,
|
|
|
|
StartTime: tsoutil.ComposeTS(ts.UnixNano()/int64(time.Millisecond), 0),
|
|
|
|
TimeoutInSeconds: 10,
|
|
|
|
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
|
|
|
|
{SegmentID: 1},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
2: {
|
2022-08-23 15:50:52 +08:00
|
|
|
state: executing,
|
|
|
|
dataNodeID: 2,
|
2021-11-05 22:25:00 +08:00
|
|
|
plan: &datapb.CompactionPlan{
|
|
|
|
PlanID: 2,
|
|
|
|
StartTime: tsoutil.ComposeTS(ts.UnixNano()/int64(time.Millisecond), 0),
|
|
|
|
TimeoutInSeconds: 1,
|
|
|
|
},
|
|
|
|
},
|
2022-08-23 15:50:52 +08:00
|
|
|
3: {
|
2022-08-26 14:22:55 +08:00
|
|
|
state: executing,
|
2022-08-23 15:50:52 +08:00
|
|
|
dataNodeID: 2,
|
|
|
|
plan: &datapb.CompactionPlan{
|
|
|
|
PlanID: 3,
|
|
|
|
StartTime: tsoutil.ComposeTS(ts.UnixNano()/int64(time.Millisecond), 0),
|
|
|
|
TimeoutInSeconds: 1,
|
|
|
|
},
|
|
|
|
},
|
|
|
|
4: {
|
|
|
|
state: executing,
|
|
|
|
dataNodeID: 2,
|
|
|
|
plan: &datapb.CompactionPlan{
|
|
|
|
PlanID: 4,
|
|
|
|
StartTime: tsoutil.ComposeTS(ts.UnixNano()/int64(time.Millisecond), 0) - 200*1000,
|
|
|
|
TimeoutInSeconds: 1,
|
|
|
|
},
|
|
|
|
},
|
2021-11-05 22:25:00 +08:00
|
|
|
},
|
|
|
|
meta: &meta{
|
|
|
|
segments: &SegmentsInfo{
|
|
|
|
map[int64]*SegmentInfo{
|
|
|
|
1: {SegmentInfo: &datapb.SegmentInfo{ID: 1}},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
},
|
2022-08-23 15:50:52 +08:00
|
|
|
sessions: &SessionManager{
|
|
|
|
sessions: struct {
|
|
|
|
sync.RWMutex
|
|
|
|
data map[int64]*Session
|
|
|
|
}{
|
|
|
|
data: map[int64]*Session{
|
|
|
|
1: {client: &mockDataNodeClient{
|
|
|
|
compactionStateResp: &datapb.CompactionStateResponse{
|
|
|
|
Results: []*datapb.CompactionStateResult{
|
|
|
|
{PlanID: 1, State: commonpb.CompactionState_Executing},
|
2022-08-26 14:22:55 +08:00
|
|
|
{PlanID: 3, State: commonpb.CompactionState_Completed, Result: &datapb.CompactionResult{PlanID: 3}},
|
|
|
|
{PlanID: 4, State: commonpb.CompactionState_Executing},
|
2022-08-23 15:50:52 +08:00
|
|
|
},
|
|
|
|
},
|
|
|
|
}},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
},
|
2021-11-05 22:25:00 +08:00
|
|
|
},
|
|
|
|
args{ts: tsoutil.ComposeTS(ts.Add(5*time.Second).UnixNano()/int64(time.Millisecond), 0)},
|
|
|
|
false,
|
2022-08-26 14:22:55 +08:00
|
|
|
[]int64{4},
|
|
|
|
[]int64{2},
|
2022-08-23 15:50:52 +08:00
|
|
|
[]int64{1, 3},
|
2021-11-05 22:25:00 +08:00
|
|
|
},
|
|
|
|
}
|
|
|
|
for _, tt := range tests {
|
|
|
|
t.Run(tt.name, func(t *testing.T) {
|
|
|
|
c := &compactionPlanHandler{
|
|
|
|
plans: tt.fields.plans,
|
|
|
|
sessions: tt.fields.sessions,
|
|
|
|
meta: tt.fields.meta,
|
|
|
|
}
|
|
|
|
|
2022-08-23 15:50:52 +08:00
|
|
|
err := c.updateCompaction(tt.args.ts)
|
2021-11-05 22:25:00 +08:00
|
|
|
assert.Equal(t, tt.wantErr, err != nil)
|
|
|
|
|
2022-08-26 14:22:55 +08:00
|
|
|
for _, id := range tt.timeout {
|
|
|
|
task := c.getCompaction(id)
|
|
|
|
assert.Equal(t, timeout, task.state)
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, id := range tt.failed {
|
2021-11-05 22:25:00 +08:00
|
|
|
task := c.getCompaction(id)
|
2022-08-23 15:50:52 +08:00
|
|
|
assert.Equal(t, failed, task.state)
|
2021-11-05 22:25:00 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
for _, id := range tt.unexpired {
|
|
|
|
task := c.getCompaction(id)
|
2022-08-23 15:50:52 +08:00
|
|
|
assert.NotEqual(t, failed, task.state)
|
2021-11-05 22:25:00 +08:00
|
|
|
}
|
|
|
|
})
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func Test_newCompactionPlanHandler(t *testing.T) {
|
|
|
|
type args struct {
|
|
|
|
sessions *SessionManager
|
|
|
|
cm *ChannelManager
|
|
|
|
meta *meta
|
|
|
|
allocator allocator
|
|
|
|
flush chan UniqueID
|
|
|
|
}
|
|
|
|
tests := []struct {
|
|
|
|
name string
|
|
|
|
args args
|
|
|
|
want *compactionPlanHandler
|
|
|
|
}{
|
|
|
|
{
|
|
|
|
"test new handler",
|
|
|
|
args{
|
|
|
|
&SessionManager{},
|
|
|
|
&ChannelManager{},
|
|
|
|
&meta{},
|
|
|
|
newMockAllocator(),
|
|
|
|
nil,
|
|
|
|
},
|
|
|
|
&compactionPlanHandler{
|
2022-08-23 15:50:52 +08:00
|
|
|
plans: map[int64]*compactionTask{},
|
|
|
|
sessions: &SessionManager{},
|
|
|
|
chManager: &ChannelManager{},
|
|
|
|
meta: &meta{},
|
|
|
|
allocator: newMockAllocator(),
|
|
|
|
flushCh: nil,
|
|
|
|
parallelCh: make(map[int64]chan struct{}),
|
2021-11-05 22:25:00 +08:00
|
|
|
},
|
|
|
|
},
|
|
|
|
}
|
|
|
|
for _, tt := range tests {
|
|
|
|
t.Run(tt.name, func(t *testing.T) {
|
2023-01-04 19:37:36 +08:00
|
|
|
got := newCompactionPlanHandler(tt.args.sessions, tt.args.cm, tt.args.meta, tt.args.allocator, tt.args.flush)
|
2021-11-05 22:25:00 +08:00
|
|
|
assert.EqualValues(t, tt.want, got)
|
|
|
|
})
|
|
|
|
}
|
|
|
|
}
|
2021-11-09 14:47:02 +08:00
|
|
|
|
|
|
|
func Test_getCompactionTasksBySignalID(t *testing.T) {
|
|
|
|
type fields struct {
|
|
|
|
plans map[int64]*compactionTask
|
|
|
|
}
|
|
|
|
type args struct {
|
|
|
|
signalID int64
|
|
|
|
}
|
|
|
|
tests := []struct {
|
|
|
|
name string
|
|
|
|
fields fields
|
|
|
|
args args
|
|
|
|
want []*compactionTask
|
|
|
|
}{
|
|
|
|
{
|
|
|
|
"test get compaction tasks",
|
|
|
|
fields{
|
|
|
|
plans: map[int64]*compactionTask{
|
|
|
|
1: {
|
|
|
|
triggerInfo: &compactionSignal{id: 1},
|
|
|
|
state: executing,
|
|
|
|
},
|
|
|
|
2: {
|
|
|
|
triggerInfo: &compactionSignal{id: 1},
|
|
|
|
state: completed,
|
|
|
|
},
|
|
|
|
3: {
|
|
|
|
triggerInfo: &compactionSignal{id: 1},
|
2022-08-23 15:50:52 +08:00
|
|
|
state: failed,
|
2021-11-09 14:47:02 +08:00
|
|
|
},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
args{1},
|
|
|
|
[]*compactionTask{
|
|
|
|
{
|
|
|
|
triggerInfo: &compactionSignal{id: 1},
|
|
|
|
state: executing,
|
|
|
|
},
|
|
|
|
{
|
|
|
|
triggerInfo: &compactionSignal{id: 1},
|
|
|
|
state: completed,
|
|
|
|
},
|
|
|
|
{
|
|
|
|
triggerInfo: &compactionSignal{id: 1},
|
2022-08-23 15:50:52 +08:00
|
|
|
state: failed,
|
2021-11-09 14:47:02 +08:00
|
|
|
},
|
|
|
|
},
|
|
|
|
},
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, tt := range tests {
|
|
|
|
t.Run(tt.name, func(t *testing.T) {
|
|
|
|
h := &compactionPlanHandler{
|
|
|
|
plans: tt.fields.plans,
|
|
|
|
}
|
|
|
|
got := h.getCompactionTasksBySignalID(tt.args.signalID)
|
|
|
|
assert.ElementsMatch(t, tt.want, got)
|
|
|
|
})
|
|
|
|
}
|
|
|
|
}
|
2022-06-15 23:14:10 +08:00
|
|
|
|
|
|
|
func getFieldBinlogPaths(id int64, paths ...string) *datapb.FieldBinlog {
|
|
|
|
l := &datapb.FieldBinlog{
|
|
|
|
FieldID: id,
|
|
|
|
Binlogs: make([]*datapb.Binlog, 0, len(paths)),
|
|
|
|
}
|
|
|
|
for _, path := range paths {
|
|
|
|
l.Binlogs = append(l.Binlogs, &datapb.Binlog{LogPath: path})
|
|
|
|
}
|
|
|
|
return l
|
|
|
|
}
|
2023-02-27 10:41:46 +08:00
|
|
|
|
|
|
|
func getFieldBinlogPathsWithEntry(id int64, entry int64, paths ...string) *datapb.FieldBinlog {
|
|
|
|
l := &datapb.FieldBinlog{
|
|
|
|
FieldID: id,
|
|
|
|
Binlogs: make([]*datapb.Binlog, 0, len(paths)),
|
|
|
|
}
|
|
|
|
for _, path := range paths {
|
|
|
|
l.Binlogs = append(l.Binlogs, &datapb.Binlog{LogPath: path, EntriesNum: entry})
|
|
|
|
}
|
|
|
|
return l
|
|
|
|
}
|