mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 17:18:35 +08:00
enhance: Move sort stats task to sort compaction (#42562)
issue: #42560 --------- Signed-off-by: Cai Zhang <cai.zhang@zilliz.com>
This commit is contained in:
parent
f1e75085e8
commit
6989e18599
@ -706,7 +706,6 @@ dataCoord:
|
||||
analyzeTaskSlotUsage: 65535 # slot usage of analyze task
|
||||
jsonStatsTriggerCount: 10 # jsonkey stats task count per trigger
|
||||
jsonStatsTriggerInterval: 10 # jsonkey task interval per trigger
|
||||
enabledJSONKeyStatsInSort: false # Indicates whether to enable JSON key stats task with sort
|
||||
jsonKeyStatsMemoryBudgetInTantivy: 16777216 # the memory budget for the JSON index In Tantivy, the unit is bytes
|
||||
ip: # TCP/IP address of dataCoord. If not specified, use the first unicastable address
|
||||
port: 13333 # TCP port of dataCoord
|
||||
|
||||
@ -18,12 +18,13 @@ package compaction
|
||||
|
||||
import (
|
||||
"github.com/milvus-io/milvus/internal/json"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
)
|
||||
|
||||
type Params struct {
|
||||
EnableStorageV2 bool `json:"enable_storage_v2,omitempty"`
|
||||
StorageVersion int64 `json:"storage_version,omitempty"`
|
||||
BinLogMaxSize uint64 `json:"binlog_max_size,omitempty"`
|
||||
UseMergeSort bool `json:"use_merge_sort,omitempty"`
|
||||
MaxSegmentMergeSort int `json:"max_segment_merge_sort,omitempty"`
|
||||
@ -33,8 +34,12 @@ type Params struct {
|
||||
}
|
||||
|
||||
func GenParams() Params {
|
||||
storageVersion := storage.StorageV1
|
||||
if paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
storageVersion = storage.StorageV2
|
||||
}
|
||||
return Params{
|
||||
EnableStorageV2: paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool(),
|
||||
StorageVersion: storageVersion,
|
||||
BinLogMaxSize: paramtable.Get().DataNodeCfg.BinLogMaxSize.GetAsUint64(),
|
||||
UseMergeSort: paramtable.Get().DataNodeCfg.UseMergeSort.GetAsBool(),
|
||||
MaxSegmentMergeSort: paramtable.Get().DataNodeCfg.MaxSegmentMergeSort.GetAsInt(),
|
||||
|
||||
@ -22,6 +22,7 @@ import (
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/json"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
)
|
||||
|
||||
@ -33,8 +34,12 @@ func TestGetJSONParams(t *testing.T) {
|
||||
var result Params
|
||||
err = json.Unmarshal([]byte(jsonStr), &result)
|
||||
assert.NoError(t, err)
|
||||
storageVersion := storage.StorageV1
|
||||
if paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
storageVersion = storage.StorageV2
|
||||
}
|
||||
assert.Equal(t, Params{
|
||||
EnableStorageV2: paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool(),
|
||||
StorageVersion: storageVersion,
|
||||
BinLogMaxSize: paramtable.Get().DataNodeCfg.BinLogMaxSize.GetAsUint64(),
|
||||
UseMergeSort: paramtable.Get().DataNodeCfg.UseMergeSort.GetAsBool(),
|
||||
MaxSegmentMergeSort: paramtable.Get().DataNodeCfg.MaxSegmentMergeSort.GetAsInt(),
|
||||
@ -46,7 +51,7 @@ func TestGetJSONParams(t *testing.T) {
|
||||
|
||||
func TestGetParamsFromJSON(t *testing.T) {
|
||||
input := `{
|
||||
"enable_storage_v2": false,
|
||||
"storage_version": 0,
|
||||
"binlog_max_size": 4096,
|
||||
"use_merge_sort": false,
|
||||
"max_segment_merge_sort": 2,
|
||||
@ -55,7 +60,7 @@ func TestGetParamsFromJSON(t *testing.T) {
|
||||
}`
|
||||
|
||||
expected := Params{
|
||||
EnableStorageV2: false,
|
||||
StorageVersion: storage.StorageV1,
|
||||
BinLogMaxSize: 4096,
|
||||
UseMergeSort: false,
|
||||
MaxSegmentMergeSort: 2,
|
||||
@ -79,8 +84,12 @@ func TestGetParamsFromJSON_EmptyJSON(t *testing.T) {
|
||||
emptyJSON := ``
|
||||
result, err := ParseParamsFromJSON(emptyJSON)
|
||||
assert.NoError(t, err)
|
||||
storageVersion := storage.StorageV1
|
||||
if paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
storageVersion = storage.StorageV2
|
||||
}
|
||||
assert.Equal(t, Params{
|
||||
EnableStorageV2: paramtable.Get().CommonCfg.EnableStorageV2.GetAsBool(),
|
||||
StorageVersion: storageVersion,
|
||||
BinLogMaxSize: paramtable.Get().DataNodeCfg.BinLogMaxSize.GetAsUint64(),
|
||||
UseMergeSort: paramtable.Get().DataNodeCfg.UseMergeSort.GetAsBool(),
|
||||
MaxSegmentMergeSort: paramtable.Get().DataNodeCfg.MaxSegmentMergeSort.GetAsInt(),
|
||||
|
||||
@ -46,6 +46,7 @@ var maxCompactionTaskExecutionDuration = map[datapb.CompactionType]time.Duration
|
||||
datapb.CompactionType_MixCompaction: 30 * time.Minute,
|
||||
datapb.CompactionType_Level0DeleteCompaction: 30 * time.Minute,
|
||||
datapb.CompactionType_ClusteringCompaction: 60 * time.Minute,
|
||||
datapb.CompactionType_SortCompaction: 20 * time.Minute,
|
||||
}
|
||||
|
||||
type CompactionInspector interface {
|
||||
@ -59,7 +60,6 @@ type CompactionInspector interface {
|
||||
getCompactionTasksNumBySignalID(signalID int64) int
|
||||
getCompactionInfo(ctx context.Context, signalID int64) *compactionInfo
|
||||
removeTasksByChannel(channel string)
|
||||
checkAndSetSegmentStating(channel string, segmentID int64) bool
|
||||
getCompactionTasksNum(filters ...compactionTaskFilter) int
|
||||
}
|
||||
|
||||
@ -94,6 +94,7 @@ type compactionInspector struct {
|
||||
analyzeScheduler task.GlobalScheduler
|
||||
handler Handler
|
||||
scheduler task.GlobalScheduler
|
||||
ievm IndexEngineVersionManager
|
||||
|
||||
stopCh chan struct{}
|
||||
stopOnce sync.Once
|
||||
@ -167,21 +168,6 @@ func summaryCompactionState(triggerID int64, tasks []*datapb.CompactionTask) *co
|
||||
return ret
|
||||
}
|
||||
|
||||
func (c *compactionInspector) checkAndSetSegmentStating(channel string, segmentID int64) bool {
|
||||
c.executingGuard.Lock()
|
||||
defer c.executingGuard.Unlock()
|
||||
|
||||
for _, t := range c.executingTasks {
|
||||
if t.GetTaskProto().GetType() == datapb.CompactionType_Level0DeleteCompaction {
|
||||
if t.GetTaskProto().GetChannel() == channel && t.CheckCompactionContainsSegment(segmentID) {
|
||||
return false
|
||||
}
|
||||
}
|
||||
}
|
||||
c.meta.SetSegmentStating(segmentID, true)
|
||||
return true
|
||||
}
|
||||
|
||||
func (c *compactionInspector) getCompactionTasksNumBySignalID(triggerID int64) int {
|
||||
cnt := 0
|
||||
c.queueTasks.ForEach(func(ct CompactionTask) {
|
||||
@ -200,7 +186,7 @@ func (c *compactionInspector) getCompactionTasksNumBySignalID(triggerID int64) i
|
||||
}
|
||||
|
||||
func newCompactionInspector(meta CompactionMeta,
|
||||
allocator allocator.Allocator, handler Handler, scheduler task.GlobalScheduler,
|
||||
allocator allocator.Allocator, handler Handler, scheduler task.GlobalScheduler, ievm IndexEngineVersionManager,
|
||||
) *compactionInspector {
|
||||
// Higher capacity will have better ordering in priority, but consumes more memory.
|
||||
// TODO[GOOSE]: Higher capacity makes tasks waiting longer, which need to be get rid of.
|
||||
@ -214,6 +200,7 @@ func newCompactionInspector(meta CompactionMeta,
|
||||
cleaningTasks: make(map[int64]CompactionTask),
|
||||
handler: handler,
|
||||
scheduler: scheduler,
|
||||
ievm: ievm,
|
||||
}
|
||||
}
|
||||
|
||||
@ -243,7 +230,7 @@ func (c *compactionInspector) schedule() []CompactionTask {
|
||||
switch t.GetTaskProto().GetType() {
|
||||
case datapb.CompactionType_Level0DeleteCompaction:
|
||||
l0ChannelExcludes.Insert(t.GetTaskProto().GetChannel())
|
||||
case datapb.CompactionType_MixCompaction:
|
||||
case datapb.CompactionType_MixCompaction, datapb.CompactionType_SortCompaction:
|
||||
mixChannelExcludes.Insert(t.GetTaskProto().GetChannel())
|
||||
mixLabelExcludes.Insert(t.GetLabel())
|
||||
case datapb.CompactionType_ClusteringCompaction:
|
||||
@ -305,15 +292,6 @@ func (c *compactionInspector) schedule() []CompactionTask {
|
||||
}
|
||||
|
||||
c.executingGuard.Lock()
|
||||
// Do not move this check logic outside the lock; it needs to remain mutually exclusive with the stats task.
|
||||
if t.GetTaskProto().GetType() == datapb.CompactionType_Level0DeleteCompaction {
|
||||
if !t.PreparePlan() {
|
||||
selected = selected[:len(selected)-1]
|
||||
excluded = append(excluded, t)
|
||||
c.executingGuard.Unlock()
|
||||
continue
|
||||
}
|
||||
}
|
||||
c.executingTasks[t.GetTaskProto().GetPlanID()] = t
|
||||
c.scheduler.Enqueue(t)
|
||||
c.executingGuard.Unlock()
|
||||
@ -609,8 +587,8 @@ func (c *compactionInspector) enqueueCompaction(task *datapb.CompactionTask) err
|
||||
func (c *compactionInspector) createCompactTask(t *datapb.CompactionTask) (CompactionTask, error) {
|
||||
var task CompactionTask
|
||||
switch t.GetType() {
|
||||
case datapb.CompactionType_MixCompaction:
|
||||
task = newMixCompactionTask(t, c.allocator, c.meta)
|
||||
case datapb.CompactionType_MixCompaction, datapb.CompactionType_SortCompaction:
|
||||
task = newMixCompactionTask(t, c.allocator, c.meta, c.ievm)
|
||||
case datapb.CompactionType_Level0DeleteCompaction:
|
||||
task = newL0CompactionTask(t, c.allocator, c.meta)
|
||||
case datapb.CompactionType_ClusteringCompaction:
|
||||
|
||||
@ -62,7 +62,7 @@ func (s *CompactionPlanHandlerSuite) SetupTest() {
|
||||
s.mockCm = NewMockChannelManager(s.T())
|
||||
s.cluster = NewMockCluster(s.T())
|
||||
mockScheduler := task.NewMockGlobalScheduler(s.T())
|
||||
s.handler = newCompactionInspector(s.mockMeta, s.mockAlloc, nil, mockScheduler)
|
||||
s.handler = newCompactionInspector(s.mockMeta, s.mockAlloc, nil, mockScheduler, newMockVersionManager())
|
||||
s.mockHandler = NewNMockHandler(s.T())
|
||||
s.mockHandler.EXPECT().GetCollection(mock.Anything, mock.Anything).Return(&collectionInfo{}, nil).Maybe()
|
||||
}
|
||||
@ -127,22 +127,6 @@ func (s *CompactionPlanHandlerSuite) generateInitTasksForSchedule() {
|
||||
}
|
||||
|
||||
func (s *CompactionPlanHandlerSuite) TestScheduleNodeWith1ParallelTask() {
|
||||
// dataNode 101's paralleTasks has 1 task running, not L0 task
|
||||
s.mockMeta.EXPECT().SelectSegments(mock.Anything, mock.Anything, mock.Anything).Return([]*SegmentInfo{
|
||||
{
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
ID: 1,
|
||||
CollectionID: 2,
|
||||
PartitionID: 3,
|
||||
},
|
||||
allocations: nil,
|
||||
lastFlushTime: time.Time{},
|
||||
isCompacting: false,
|
||||
lastWrittenTime: time.Time{},
|
||||
isStating: false,
|
||||
},
|
||||
})
|
||||
s.mockMeta.EXPECT().CheckSegmentsStating(mock.Anything, mock.Anything).Return(true, false)
|
||||
tests := []struct {
|
||||
description string
|
||||
tasks []CompactionTask
|
||||
@ -182,7 +166,7 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWith1ParallelTask() {
|
||||
State: datapb.CompactionTaskState_pipelining,
|
||||
Channel: "ch-11",
|
||||
NodeID: 101,
|
||||
}, nil, s.mockMeta),
|
||||
}, nil, s.mockMeta, newMockVersionManager()),
|
||||
newL0CompactionTask(&datapb.CompactionTask{
|
||||
PlanID: 10,
|
||||
Type: datapb.CompactionType_Level0DeleteCompaction,
|
||||
@ -206,14 +190,14 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWith1ParallelTask() {
|
||||
State: datapb.CompactionTaskState_pipelining,
|
||||
Channel: "ch-2",
|
||||
NodeID: 101,
|
||||
}, nil, s.mockMeta),
|
||||
}, nil, s.mockMeta, newMockVersionManager()),
|
||||
newMixCompactionTask(&datapb.CompactionTask{
|
||||
PlanID: 13,
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
State: datapb.CompactionTaskState_pipelining,
|
||||
Channel: "ch-11",
|
||||
NodeID: 101,
|
||||
}, nil, s.mockMeta),
|
||||
}, nil, s.mockMeta, newMockVersionManager()),
|
||||
},
|
||||
[]*datapb.CompactionPlan{
|
||||
{PlanID: 14, Channel: "ch-2", Type: datapb.CompactionType_MixCompaction},
|
||||
@ -251,24 +235,6 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWith1ParallelTask() {
|
||||
}
|
||||
|
||||
func (s *CompactionPlanHandlerSuite) TestScheduleNodeWithL0Executing() {
|
||||
// dataNode 102's paralleTasks has running L0 tasks
|
||||
// nothing of the same channel will be able to schedule
|
||||
|
||||
s.mockMeta.EXPECT().SelectSegments(mock.Anything, mock.Anything, mock.Anything).Return([]*SegmentInfo{
|
||||
{
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
ID: 1,
|
||||
CollectionID: 2,
|
||||
PartitionID: 3,
|
||||
},
|
||||
allocations: nil,
|
||||
lastFlushTime: time.Time{},
|
||||
isCompacting: false,
|
||||
lastWrittenTime: time.Time{},
|
||||
isStating: false,
|
||||
},
|
||||
})
|
||||
s.mockMeta.EXPECT().CheckSegmentsStating(mock.Anything, mock.Anything).Return(true, false)
|
||||
tests := []struct {
|
||||
description string
|
||||
tasks []CompactionTask
|
||||
@ -291,7 +257,7 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWithL0Executing() {
|
||||
State: datapb.CompactionTaskState_pipelining,
|
||||
Channel: "ch-11",
|
||||
NodeID: 102,
|
||||
}, nil, s.mockMeta),
|
||||
}, nil, s.mockMeta, newMockVersionManager()),
|
||||
},
|
||||
[]*datapb.CompactionPlan{{}, {}},
|
||||
[]UniqueID{10, 11},
|
||||
@ -312,14 +278,14 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWithL0Executing() {
|
||||
State: datapb.CompactionTaskState_pipelining,
|
||||
Channel: "ch-11",
|
||||
NodeID: 102,
|
||||
}, nil, s.mockMeta),
|
||||
}, nil, s.mockMeta, newMockVersionManager()),
|
||||
newMixCompactionTask(&datapb.CompactionTask{
|
||||
PlanID: 13,
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
State: datapb.CompactionTaskState_pipelining,
|
||||
Channel: "ch-3",
|
||||
NodeID: 102,
|
||||
}, nil, s.mockMeta),
|
||||
}, nil, s.mockMeta, newMockVersionManager()),
|
||||
},
|
||||
[]*datapb.CompactionPlan{
|
||||
{PlanID: 10, Channel: "ch-3", Type: datapb.CompactionType_Level0DeleteCompaction},
|
||||
@ -368,13 +334,13 @@ func (s *CompactionPlanHandlerSuite) TestScheduleNodeWithL0Executing() {
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
Channel: "ch-3",
|
||||
NodeID: 102,
|
||||
}, nil, s.mockMeta),
|
||||
}, nil, s.mockMeta, newMockVersionManager()),
|
||||
newMixCompactionTask(&datapb.CompactionTask{
|
||||
PlanID: 13,
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
Channel: "ch-11",
|
||||
NodeID: 102,
|
||||
}, nil, s.mockMeta),
|
||||
}, nil, s.mockMeta, newMockVersionManager()),
|
||||
},
|
||||
[]*datapb.CompactionPlan{
|
||||
{PlanID: 14, Channel: "ch-3", Type: datapb.CompactionType_MixCompaction},
|
||||
@ -415,14 +381,14 @@ func (s *CompactionPlanHandlerSuite) TestRemoveTasksByChannel() {
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
Channel: ch,
|
||||
NodeID: 1,
|
||||
}, nil, s.mockMeta)
|
||||
}, nil, s.mockMeta, newMockVersionManager())
|
||||
|
||||
t2 := newMixCompactionTask(&datapb.CompactionTask{
|
||||
PlanID: 19531,
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
Channel: ch,
|
||||
NodeID: 1,
|
||||
}, nil, s.mockMeta)
|
||||
}, nil, s.mockMeta, newMockVersionManager())
|
||||
|
||||
s.handler.submitTask(t1)
|
||||
s.handler.restoreTask(t2)
|
||||
@ -440,7 +406,7 @@ func (s *CompactionPlanHandlerSuite) TestGetCompactionTask() {
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
Channel: "ch-01",
|
||||
State: datapb.CompactionTaskState_executing,
|
||||
}, nil, s.mockMeta)
|
||||
}, nil, s.mockMeta, newMockVersionManager())
|
||||
|
||||
t2 := newMixCompactionTask(&datapb.CompactionTask{
|
||||
TriggerID: 1,
|
||||
@ -448,7 +414,7 @@ func (s *CompactionPlanHandlerSuite) TestGetCompactionTask() {
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
Channel: "ch-01",
|
||||
State: datapb.CompactionTaskState_completed,
|
||||
}, nil, s.mockMeta)
|
||||
}, nil, s.mockMeta, newMockVersionManager())
|
||||
|
||||
t3 := newL0CompactionTask(&datapb.CompactionTask{
|
||||
TriggerID: 1,
|
||||
@ -473,21 +439,6 @@ func (s *CompactionPlanHandlerSuite) TestGetCompactionTask() {
|
||||
}
|
||||
return ret
|
||||
})
|
||||
s.mockMeta.EXPECT().SelectSegments(mock.Anything, mock.Anything, mock.Anything).Return([]*SegmentInfo{
|
||||
{
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
ID: 1,
|
||||
CollectionID: 2,
|
||||
PartitionID: 3,
|
||||
},
|
||||
allocations: nil,
|
||||
lastFlushTime: time.Time{},
|
||||
isCompacting: false,
|
||||
lastWrittenTime: time.Time{},
|
||||
isStating: false,
|
||||
},
|
||||
})
|
||||
s.mockMeta.EXPECT().CheckSegmentsStating(mock.Anything, mock.Anything).Return(true, false)
|
||||
|
||||
for _, t := range inTasks {
|
||||
s.handler.submitTask(t)
|
||||
@ -513,7 +464,7 @@ func (s *CompactionPlanHandlerSuite) TestCompactionQueueFull() {
|
||||
t.QueryTaskOnWorker(cluster)
|
||||
}
|
||||
}).Maybe()
|
||||
s.handler = newCompactionInspector(s.mockMeta, s.mockAlloc, nil, mockScheduler)
|
||||
s.handler = newCompactionInspector(s.mockMeta, s.mockAlloc, nil, mockScheduler, newMockVersionManager())
|
||||
|
||||
t1 := newMixCompactionTask(&datapb.CompactionTask{
|
||||
TriggerID: 1,
|
||||
@ -521,7 +472,7 @@ func (s *CompactionPlanHandlerSuite) TestCompactionQueueFull() {
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
Channel: "ch-01",
|
||||
State: datapb.CompactionTaskState_executing,
|
||||
}, nil, s.mockMeta)
|
||||
}, nil, s.mockMeta, newMockVersionManager())
|
||||
|
||||
s.NoError(s.handler.submitTask(t1))
|
||||
|
||||
@ -531,7 +482,7 @@ func (s *CompactionPlanHandlerSuite) TestCompactionQueueFull() {
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
Channel: "ch-01",
|
||||
State: datapb.CompactionTaskState_completed,
|
||||
}, nil, s.mockMeta)
|
||||
}, nil, s.mockMeta, newMockVersionManager())
|
||||
|
||||
s.Error(s.handler.submitTask(t2))
|
||||
}
|
||||
@ -547,7 +498,7 @@ func (s *CompactionPlanHandlerSuite) TestExecCompactionPlan() {
|
||||
t.QueryTaskOnWorker(cluster)
|
||||
}
|
||||
}).Maybe()
|
||||
handler := newCompactionInspector(s.mockMeta, s.mockAlloc, nil, mockScheduler)
|
||||
handler := newCompactionInspector(s.mockMeta, s.mockAlloc, nil, mockScheduler, newMockVersionManager())
|
||||
|
||||
task := &datapb.CompactionTask{
|
||||
TriggerID: 1,
|
||||
@ -605,7 +556,7 @@ func (s *CompactionPlanHandlerSuite) TestCheckCompaction() {
|
||||
Channel: "ch-1",
|
||||
State: datapb.CompactionTaskState_executing,
|
||||
NodeID: 111,
|
||||
}, s.mockAlloc, s.mockMeta)
|
||||
}, s.mockAlloc, s.mockMeta, newMockVersionManager())
|
||||
|
||||
t2 := newMixCompactionTask(&datapb.CompactionTask{
|
||||
PlanID: 2,
|
||||
@ -613,7 +564,7 @@ func (s *CompactionPlanHandlerSuite) TestCheckCompaction() {
|
||||
Channel: "ch-1",
|
||||
State: datapb.CompactionTaskState_executing,
|
||||
NodeID: 111,
|
||||
}, s.mockAlloc, s.mockMeta)
|
||||
}, s.mockAlloc, s.mockMeta, newMockVersionManager())
|
||||
|
||||
t3 := newMixCompactionTask(&datapb.CompactionTask{
|
||||
PlanID: 3,
|
||||
@ -621,7 +572,7 @@ func (s *CompactionPlanHandlerSuite) TestCheckCompaction() {
|
||||
Channel: "ch-1",
|
||||
State: datapb.CompactionTaskState_timeout,
|
||||
NodeID: 111,
|
||||
}, s.mockAlloc, s.mockMeta)
|
||||
}, s.mockAlloc, s.mockMeta, newMockVersionManager())
|
||||
|
||||
t4 := newMixCompactionTask(&datapb.CompactionTask{
|
||||
PlanID: 4,
|
||||
@ -629,7 +580,7 @@ func (s *CompactionPlanHandlerSuite) TestCheckCompaction() {
|
||||
Channel: "ch-1",
|
||||
State: datapb.CompactionTaskState_timeout,
|
||||
NodeID: 111,
|
||||
}, s.mockAlloc, s.mockMeta)
|
||||
}, s.mockAlloc, s.mockMeta, newMockVersionManager())
|
||||
|
||||
t6 := newMixCompactionTask(&datapb.CompactionTask{
|
||||
PlanID: 6,
|
||||
@ -637,7 +588,7 @@ func (s *CompactionPlanHandlerSuite) TestCheckCompaction() {
|
||||
Channel: "ch-2",
|
||||
State: datapb.CompactionTaskState_executing,
|
||||
NodeID: 111,
|
||||
}, s.mockAlloc, s.mockMeta)
|
||||
}, s.mockAlloc, s.mockMeta, newMockVersionManager())
|
||||
|
||||
inTasks := map[int64]CompactionTask{
|
||||
1: t1,
|
||||
@ -788,7 +739,7 @@ func (s *CompactionPlanHandlerSuite) TestProcessCompleteCompaction() {
|
||||
State: datapb.CompactionTaskState_executing,
|
||||
NodeID: dataNodeID,
|
||||
InputSegments: []UniqueID{1, 2},
|
||||
}, nil, s.mockMeta)
|
||||
}, nil, s.mockMeta, newMockVersionManager())
|
||||
|
||||
compactionResult := datapb.CompactionPlanResult{
|
||||
PlanID: 1,
|
||||
@ -830,7 +781,7 @@ func (s *CompactionPlanHandlerSuite) TestCleanCompaction() {
|
||||
NodeID: 1,
|
||||
InputSegments: []UniqueID{1, 2},
|
||||
},
|
||||
nil, s.mockMeta),
|
||||
nil, s.mockMeta, newMockVersionManager()),
|
||||
},
|
||||
{
|
||||
newL0CompactionTask(&datapb.CompactionTask{
|
||||
@ -1056,7 +1007,7 @@ func TestCheckDelay(t *testing.T) {
|
||||
handler := &compactionInspector{}
|
||||
t1 := newMixCompactionTask(&datapb.CompactionTask{
|
||||
StartTime: time.Now().Add(-100 * time.Minute).Unix(),
|
||||
}, nil, nil)
|
||||
}, nil, nil, newMockVersionManager())
|
||||
handler.checkDelay(t1)
|
||||
t2 := newL0CompactionTask(&datapb.CompactionTask{
|
||||
StartTime: time.Now().Add(-100 * time.Minute).Unix(),
|
||||
@ -1075,7 +1026,7 @@ func TestGetCompactionTasksNum(t *testing.T) {
|
||||
StartTime: time.Now().Add(-100 * time.Minute).Unix(),
|
||||
CollectionID: 1,
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
}, nil, nil),
|
||||
}, nil, nil, newMockVersionManager()),
|
||||
)
|
||||
queueTasks.Enqueue(
|
||||
newL0CompactionTask(&datapb.CompactionTask{
|
||||
@ -1096,7 +1047,7 @@ func TestGetCompactionTasksNum(t *testing.T) {
|
||||
StartTime: time.Now().Add(-100 * time.Minute).Unix(),
|
||||
CollectionID: 1,
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
}, nil, nil)
|
||||
}, nil, nil, newMockVersionManager())
|
||||
executingTasks[2] = newL0CompactionTask(&datapb.CompactionTask{
|
||||
StartTime: time.Now().Add(-100 * time.Minute).Unix(),
|
||||
CollectionID: 10,
|
||||
|
||||
@ -51,41 +51,179 @@ func (policy *singleCompactionPolicy) Trigger(ctx context.Context) (map[Compacti
|
||||
|
||||
events := make(map[CompactionTriggerType][]CompactionView, 0)
|
||||
views := make([]CompactionView, 0)
|
||||
sortViews := make([]CompactionView, 0)
|
||||
for _, collection := range collections {
|
||||
collectionViews, _, err := policy.triggerOneCollection(ctx, collection.ID, false)
|
||||
collectionViews, collectionSortViews, _, err := policy.triggerOneCollection(ctx, collection.ID, false)
|
||||
if err != nil {
|
||||
// not throw this error because no need to fail because of one collection
|
||||
log.Warn("fail to trigger single compaction", zap.Int64("collectionID", collection.ID), zap.Error(err))
|
||||
}
|
||||
views = append(views, collectionViews...)
|
||||
sortViews = append(sortViews, collectionSortViews...)
|
||||
}
|
||||
events[TriggerTypeSingle] = views
|
||||
events[TriggerTypeSort] = sortViews
|
||||
return events, nil
|
||||
}
|
||||
|
||||
func (policy *singleCompactionPolicy) triggerOneCollection(ctx context.Context, collectionID int64, manual bool) ([]CompactionView, int64, error) {
|
||||
func (policy *singleCompactionPolicy) triggerSegmentSortCompaction(
|
||||
ctx context.Context,
|
||||
segmentID int64,
|
||||
) CompactionView {
|
||||
log := log.With(zap.Int64("segmentID", segmentID))
|
||||
if !Params.DataCoordCfg.EnableSortCompaction.GetAsBool() {
|
||||
log.RatedInfo(20, "stats task disabled, skip sort compaction")
|
||||
return nil
|
||||
}
|
||||
segment := policy.meta.GetHealthySegment(ctx, segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("fail to apply triggerSegmentSortCompaction, segment not healthy")
|
||||
return nil
|
||||
}
|
||||
if !canTriggerSortCompaction(segment) {
|
||||
log.Warn("fail to apply triggerSegmentSortCompaction",
|
||||
zap.String("state", segment.GetState().String()),
|
||||
zap.String("level", segment.GetLevel().String()),
|
||||
zap.Bool("isSorted", segment.GetIsSorted()),
|
||||
zap.Bool("isImporting", segment.GetIsImporting()),
|
||||
zap.Bool("isCompacting", segment.isCompacting),
|
||||
zap.Bool("isInvisible", segment.GetIsInvisible()))
|
||||
return nil
|
||||
}
|
||||
|
||||
collection, err := policy.handler.GetCollection(ctx, segment.GetCollectionID())
|
||||
if err != nil {
|
||||
log.Warn("fail to apply triggerSegmentSortCompaction, unable to get collection from handler",
|
||||
zap.Error(err))
|
||||
return nil
|
||||
}
|
||||
if collection == nil {
|
||||
log.Warn("fail to apply triggerSegmentSortCompaction, collection not exist")
|
||||
return nil
|
||||
}
|
||||
collectionTTL, err := getCollectionTTL(collection.Properties)
|
||||
if err != nil {
|
||||
log.Warn("failed to apply triggerSegmentSortCompaction, get collection ttl failed")
|
||||
return nil
|
||||
}
|
||||
|
||||
newTriggerID, err := policy.allocator.AllocID(ctx)
|
||||
if err != nil {
|
||||
log.Warn("fail to apply triggerSegmentSortCompaction, unable to allocate triggerID", zap.Error(err))
|
||||
return nil
|
||||
}
|
||||
|
||||
segmentViews := GetViewsByInfo(segment)
|
||||
view := &MixSegmentView{
|
||||
label: segmentViews[0].label,
|
||||
segments: segmentViews,
|
||||
collectionTTL: collectionTTL,
|
||||
triggerID: newTriggerID,
|
||||
}
|
||||
|
||||
log.Info("succeeded to apply triggerSegmentSortCompaction",
|
||||
zap.Int64("triggerID", newTriggerID))
|
||||
return view
|
||||
}
|
||||
|
||||
func (policy *singleCompactionPolicy) triggerSortCompaction(
|
||||
ctx context.Context,
|
||||
triggerID int64,
|
||||
collectionID int64,
|
||||
collectionTTL time.Duration,
|
||||
) ([]CompactionView, error) {
|
||||
log := log.With(zap.Int64("collectionID", collectionID))
|
||||
if !Params.DataCoordCfg.EnableSortCompaction.GetAsBool() {
|
||||
log.RatedInfo(20, "stats task disabled, skip sort compaction")
|
||||
return nil, nil
|
||||
}
|
||||
views := make([]CompactionView, 0)
|
||||
|
||||
triggerableSegments := policy.meta.SelectSegments(ctx, WithCollection(collectionID),
|
||||
SegmentFilterFunc(func(seg *SegmentInfo) bool {
|
||||
return canTriggerSortCompaction(seg)
|
||||
}))
|
||||
if len(triggerableSegments) == 0 {
|
||||
log.RatedInfo(20, "no triggerable segments")
|
||||
return views, nil
|
||||
}
|
||||
|
||||
gbSegments := lo.GroupBy(triggerableSegments, func(seg *SegmentInfo) bool {
|
||||
return seg.GetIsInvisible()
|
||||
})
|
||||
invisibleSegments, ok := gbSegments[true]
|
||||
if ok {
|
||||
for _, segment := range invisibleSegments {
|
||||
segmentViews := GetViewsByInfo(segment)
|
||||
view := &MixSegmentView{
|
||||
label: segmentViews[0].label,
|
||||
segments: segmentViews,
|
||||
collectionTTL: collectionTTL,
|
||||
triggerID: triggerID,
|
||||
}
|
||||
views = append(views, view)
|
||||
}
|
||||
}
|
||||
|
||||
visibleSegments, ok := gbSegments[false]
|
||||
if ok {
|
||||
for i, segment := range visibleSegments {
|
||||
if i > Params.DataCoordCfg.SortCompactionTriggerCount.GetAsInt() {
|
||||
break
|
||||
}
|
||||
segmentViews := GetViewsByInfo(segment)
|
||||
view := &MixSegmentView{
|
||||
label: segmentViews[0].label,
|
||||
segments: segmentViews,
|
||||
collectionTTL: collectionTTL,
|
||||
triggerID: triggerID,
|
||||
}
|
||||
views = append(views, view)
|
||||
}
|
||||
}
|
||||
|
||||
log.Info("succeeded to apply triggerSortCompaction",
|
||||
zap.Int64("triggerID", triggerID),
|
||||
zap.Int("triggered view num", len(views)))
|
||||
return views, nil
|
||||
}
|
||||
|
||||
func (policy *singleCompactionPolicy) triggerOneCollection(ctx context.Context, collectionID int64, manual bool) ([]CompactionView, []CompactionView, int64, error) {
|
||||
log := log.With(zap.Int64("collectionID", collectionID))
|
||||
collection, err := policy.handler.GetCollection(ctx, collectionID)
|
||||
if err != nil {
|
||||
log.Warn("fail to apply singleCompactionPolicy, unable to get collection from handler",
|
||||
zap.Error(err))
|
||||
return nil, 0, err
|
||||
return nil, nil, 0, err
|
||||
}
|
||||
if collection == nil {
|
||||
log.Warn("fail to apply singleCompactionPolicy, collection not exist")
|
||||
return nil, 0, nil
|
||||
return nil, nil, 0, nil
|
||||
}
|
||||
if !isCollectionAutoCompactionEnabled(collection) {
|
||||
log.RatedInfo(20, "collection auto compaction disabled")
|
||||
return nil, 0, nil
|
||||
|
||||
collectionTTL, err := getCollectionTTL(collection.Properties)
|
||||
if err != nil {
|
||||
log.Warn("failed to apply singleCompactionPolicy, get collection ttl failed")
|
||||
return nil, nil, 0, err
|
||||
}
|
||||
|
||||
newTriggerID, err := policy.allocator.AllocID(ctx)
|
||||
if err != nil {
|
||||
log.Warn("fail to apply singleCompactionPolicy, unable to allocate triggerID", zap.Error(err))
|
||||
return nil, 0, err
|
||||
return nil, nil, 0, err
|
||||
}
|
||||
|
||||
sortViews, err := policy.triggerSortCompaction(ctx, newTriggerID, collectionID, collectionTTL)
|
||||
if err != nil {
|
||||
log.Warn("failed to apply singleCompactionPolicy, trigger sort compaction failed", zap.Error(err))
|
||||
return nil, nil, 0, err
|
||||
}
|
||||
if !isCollectionAutoCompactionEnabled(collection) {
|
||||
log.RatedInfo(20, "collection auto compaction disabled")
|
||||
return nil, sortViews, 0, nil
|
||||
}
|
||||
|
||||
views := make([]CompactionView, 0)
|
||||
partSegments := GetSegmentsChanPart(policy.meta, collectionID, SegmentFilterFunc(func(segment *SegmentInfo) bool {
|
||||
return isSegmentHealthy(segment) &&
|
||||
isFlushed(segment) &&
|
||||
@ -95,18 +233,11 @@ func (policy *singleCompactionPolicy) triggerOneCollection(ctx context.Context,
|
||||
!segment.GetIsInvisible()
|
||||
}))
|
||||
|
||||
views := make([]CompactionView, 0)
|
||||
for _, group := range partSegments {
|
||||
if Params.DataCoordCfg.IndexBasedCompaction.GetAsBool() {
|
||||
group.segments = FilterInIndexedSegments(ctx, policy.handler, policy.meta, false, group.segments...)
|
||||
}
|
||||
|
||||
collectionTTL, err := getCollectionTTL(collection.Properties)
|
||||
if err != nil {
|
||||
log.Warn("failed to apply singleCompactionPolicy, get collection ttl failed")
|
||||
return make([]CompactionView, 0), 0, err
|
||||
}
|
||||
|
||||
for _, segment := range group.segments {
|
||||
if hasTooManyDeletions(segment) {
|
||||
segmentViews := GetViewsByInfo(segment)
|
||||
@ -126,7 +257,7 @@ func (policy *singleCompactionPolicy) triggerOneCollection(ctx context.Context,
|
||||
zap.Int64("triggerID", newTriggerID),
|
||||
zap.Int("triggered view num", len(views)))
|
||||
}
|
||||
return views, newTriggerID, nil
|
||||
return views, sortViews, newTriggerID, nil
|
||||
}
|
||||
|
||||
var _ CompactionView = (*MixSegmentView)(nil)
|
||||
|
||||
@ -74,7 +74,15 @@ func (s *SingleCompactionPolicySuite) TestTrigger() {
|
||||
s.Equal(0, len(gotViews))
|
||||
}
|
||||
|
||||
func buildTestSegment(id int64, collId int64, level datapb.SegmentLevel, deleteRows int64, totalRows int64, deltaLogNum int) *SegmentInfo {
|
||||
func buildTestSegment(id int64,
|
||||
collId int64,
|
||||
level datapb.SegmentLevel,
|
||||
deleteRows int64,
|
||||
totalRows int64,
|
||||
deltaLogNum int,
|
||||
isSorted bool,
|
||||
isInvisible bool,
|
||||
) *SegmentInfo {
|
||||
deltaBinlogs := make([]*datapb.Binlog, 0)
|
||||
for i := 0; i < deltaLogNum; i++ {
|
||||
deltaBinlogs = append(deltaBinlogs, &datapb.Binlog{
|
||||
@ -94,18 +102,20 @@ func buildTestSegment(id int64, collId int64, level datapb.SegmentLevel, deleteR
|
||||
Binlogs: deltaBinlogs,
|
||||
},
|
||||
},
|
||||
IsSorted: isSorted,
|
||||
IsInvisible: isInvisible,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (s *SingleCompactionPolicySuite) TestIsDeleteRowsTooManySegment() {
|
||||
segment0 := buildTestSegment(101, collID, datapb.SegmentLevel_L2, 0, 10000, 201)
|
||||
segment0 := buildTestSegment(101, collID, datapb.SegmentLevel_L2, 0, 10000, 201, true, true)
|
||||
s.Equal(true, hasTooManyDeletions(segment0))
|
||||
|
||||
segment1 := buildTestSegment(101, collID, datapb.SegmentLevel_L2, 3000, 10000, 1)
|
||||
segment1 := buildTestSegment(101, collID, datapb.SegmentLevel_L2, 3000, 10000, 1, true, true)
|
||||
s.Equal(true, hasTooManyDeletions(segment1))
|
||||
|
||||
segment2 := buildTestSegment(101, collID, datapb.SegmentLevel_L2, 300, 10000, 10)
|
||||
segment2 := buildTestSegment(101, collID, datapb.SegmentLevel_L2, 300, 10000, 10, true, true)
|
||||
s.Equal(true, hasTooManyDeletions(segment2))
|
||||
}
|
||||
|
||||
@ -122,9 +132,9 @@ func (s *SingleCompactionPolicySuite) TestL2SingleCompaction() {
|
||||
s.handler.EXPECT().GetCollection(mock.Anything, mock.Anything).Return(coll, nil)
|
||||
|
||||
segments := make(map[UniqueID]*SegmentInfo, 0)
|
||||
segments[101] = buildTestSegment(101, collID, datapb.SegmentLevel_L2, 0, 10000, 201)
|
||||
segments[102] = buildTestSegment(101, collID, datapb.SegmentLevel_L2, 500, 10000, 10)
|
||||
segments[103] = buildTestSegment(101, collID, datapb.SegmentLevel_L2, 100, 10000, 1)
|
||||
segments[101] = buildTestSegment(101, collID, datapb.SegmentLevel_L2, 0, 10000, 201, true, false)
|
||||
segments[102] = buildTestSegment(101, collID, datapb.SegmentLevel_L2, 500, 10000, 10, true, false)
|
||||
segments[103] = buildTestSegment(101, collID, datapb.SegmentLevel_L2, 100, 10000, 1, true, false)
|
||||
segmentsInfo := &SegmentsInfo{
|
||||
segments: segments,
|
||||
secondaryIndexes: segmentInfoIndexes{
|
||||
@ -150,7 +160,107 @@ func (s *SingleCompactionPolicySuite) TestL2SingleCompaction() {
|
||||
State: datapb.CompactionTaskState_executing,
|
||||
})
|
||||
|
||||
views, _, err := s.singlePolicy.triggerOneCollection(context.TODO(), collID, false)
|
||||
views, _, _, err := s.singlePolicy.triggerOneCollection(context.TODO(), collID, false)
|
||||
s.NoError(err)
|
||||
s.Equal(2, len(views))
|
||||
}
|
||||
|
||||
func (s *SingleCompactionPolicySuite) TestSortCompaction() {
|
||||
ctx := context.Background()
|
||||
paramtable.Get().Save(paramtable.Get().DataCoordCfg.IndexBasedCompaction.Key, "false")
|
||||
defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.IndexBasedCompaction.Key)
|
||||
|
||||
collID := int64(100)
|
||||
coll := &collectionInfo{
|
||||
ID: collID,
|
||||
Schema: newTestSchema(),
|
||||
}
|
||||
s.handler.EXPECT().GetCollection(mock.Anything, mock.Anything).Return(coll, nil)
|
||||
|
||||
segments := make(map[UniqueID]*SegmentInfo, 0)
|
||||
segments[101] = buildTestSegment(101, collID, datapb.SegmentLevel_L1, 0, 10000, 201, false, true)
|
||||
segments[102] = buildTestSegment(101, collID, datapb.SegmentLevel_L2, 500, 10000, 10, false, true)
|
||||
segments[103] = buildTestSegment(101, collID, datapb.SegmentLevel_L1, 100, 10000, 1, false, false)
|
||||
segmentsInfo := &SegmentsInfo{
|
||||
segments: segments,
|
||||
secondaryIndexes: segmentInfoIndexes{
|
||||
coll2Segments: map[UniqueID]map[UniqueID]*SegmentInfo{
|
||||
collID: {
|
||||
101: segments[101],
|
||||
102: segments[102],
|
||||
103: segments[103],
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
compactionTaskMeta := newTestCompactionTaskMeta(s.T())
|
||||
s.singlePolicy.meta = &meta{
|
||||
compactionTaskMeta: compactionTaskMeta,
|
||||
segments: segmentsInfo,
|
||||
}
|
||||
compactionTaskMeta.SaveCompactionTask(ctx, &datapb.CompactionTask{
|
||||
TriggerID: 1,
|
||||
PlanID: 10,
|
||||
CollectionID: collID,
|
||||
State: datapb.CompactionTaskState_executing,
|
||||
Type: datapb.CompactionType_SortCompaction,
|
||||
})
|
||||
|
||||
_, sortViews, _, err := s.singlePolicy.triggerOneCollection(context.TODO(), collID, false)
|
||||
s.NoError(err)
|
||||
s.Equal(3, len(sortViews))
|
||||
}
|
||||
|
||||
func (s *SingleCompactionPolicySuite) TestSegmentSortCompaction() {
|
||||
ctx := context.Background()
|
||||
paramtable.Get().Save(paramtable.Get().DataCoordCfg.IndexBasedCompaction.Key, "false")
|
||||
defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.IndexBasedCompaction.Key)
|
||||
|
||||
collID := int64(100)
|
||||
coll := &collectionInfo{
|
||||
ID: collID,
|
||||
Schema: newTestSchema(),
|
||||
}
|
||||
s.handler.EXPECT().GetCollection(mock.Anything, mock.Anything).Return(coll, nil)
|
||||
|
||||
segments := make(map[UniqueID]*SegmentInfo, 0)
|
||||
segments[101] = buildTestSegment(101, collID, datapb.SegmentLevel_L1, 0, 10000, 201, false, true)
|
||||
segments[102] = buildTestSegment(102, collID, datapb.SegmentLevel_L1, 0, 10000, 201, true, true)
|
||||
segments[103] = buildTestSegment(103, collID, datapb.SegmentLevel_L1, 0, 10000, 201, true, true)
|
||||
segments[103].SegmentInfo.State = commonpb.SegmentState_Dropped
|
||||
segmentsInfo := &SegmentsInfo{
|
||||
segments: segments,
|
||||
secondaryIndexes: segmentInfoIndexes{
|
||||
coll2Segments: map[UniqueID]map[UniqueID]*SegmentInfo{
|
||||
collID: {
|
||||
101: segments[101],
|
||||
102: segments[102],
|
||||
103: segments[103],
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
compactionTaskMeta := newTestCompactionTaskMeta(s.T())
|
||||
s.singlePolicy.meta = &meta{
|
||||
compactionTaskMeta: compactionTaskMeta,
|
||||
segments: segmentsInfo,
|
||||
}
|
||||
compactionTaskMeta.SaveCompactionTask(ctx, &datapb.CompactionTask{
|
||||
TriggerID: 1,
|
||||
PlanID: 10,
|
||||
CollectionID: collID,
|
||||
State: datapb.CompactionTaskState_executing,
|
||||
Type: datapb.CompactionType_SortCompaction,
|
||||
})
|
||||
|
||||
sortView := s.singlePolicy.triggerSegmentSortCompaction(context.TODO(), 101)
|
||||
s.NotNil(sortView)
|
||||
|
||||
sortView = s.singlePolicy.triggerSegmentSortCompaction(context.TODO(), 102)
|
||||
s.Nil(sortView)
|
||||
|
||||
sortView = s.singlePolicy.triggerSegmentSortCompaction(context.TODO(), 103)
|
||||
s.Nil(sortView)
|
||||
}
|
||||
|
||||
@ -46,7 +46,6 @@ type CompactionTask interface {
|
||||
NeedReAssignNodeID() bool
|
||||
SaveTaskMeta() error
|
||||
|
||||
PreparePlan() bool
|
||||
CheckCompactionContainsSegment(segmentID int64) bool
|
||||
}
|
||||
|
||||
|
||||
@ -326,10 +326,6 @@ func (t *clusteringCompactionTask) Clean() bool {
|
||||
return t.doClean() == nil
|
||||
}
|
||||
|
||||
func (t *clusteringCompactionTask) PreparePlan() bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func (t *clusteringCompactionTask) CheckCompactionContainsSegment(segmentID int64) bool {
|
||||
return false
|
||||
}
|
||||
@ -403,7 +399,7 @@ func (t *clusteringCompactionTask) processStats() error {
|
||||
log := log.Ctx(context.TODO()).With(zap.Int64("planID", t.GetTaskProto().GetPlanID()))
|
||||
// just the memory step, if it crashes at this step, the state after recovery is CompactionTaskState_statistic.
|
||||
resultSegments := make([]int64, 0, len(t.GetTaskProto().GetTmpSegments()))
|
||||
if Params.DataCoordCfg.EnableStatsTask.GetAsBool() {
|
||||
if Params.DataCoordCfg.EnableSortCompaction.GetAsBool() {
|
||||
existNonStats := false
|
||||
tmpToResultSegments := make(map[int64][]int64, len(t.GetTaskProto().GetTmpSegments()))
|
||||
for _, segmentID := range t.GetTaskProto().GetTmpSegments() {
|
||||
|
||||
@ -995,8 +995,8 @@ func (s *ClusteringCompactionTaskSuite) TestProcessStatsState() {
|
||||
})
|
||||
|
||||
s.Run("not enable stats task", func() {
|
||||
Params.Save(Params.DataCoordCfg.EnableStatsTask.Key, "false")
|
||||
defer Params.Reset(Params.DataCoordCfg.EnableStatsTask.Key)
|
||||
Params.Save(Params.DataCoordCfg.EnableSortCompaction.Key, "false")
|
||||
defer Params.Reset(Params.DataCoordCfg.EnableSortCompaction.Key)
|
||||
task := s.generateBasicTask(false)
|
||||
task.updateAndSaveTaskMeta(setState(datapb.CompactionTaskState_statistic), setTmpSegments(task.GetTaskProto().GetResultSegments()), setResultSegments(nil))
|
||||
task.maxRetryTimes = 3
|
||||
|
||||
@ -291,15 +291,6 @@ func (t *l0CompactionTask) CheckCompactionContainsSegment(segmentID int64) bool
|
||||
return false
|
||||
}
|
||||
|
||||
func (t *l0CompactionTask) PreparePlan() bool {
|
||||
sealedSegments, _ := t.selectSealedSegment()
|
||||
sealedSegmentIDs := lo.Map(sealedSegments, func(info *SegmentInfo, _ int) int64 {
|
||||
return info.GetID()
|
||||
})
|
||||
exist, hasStating := t.meta.CheckSegmentsStating(context.TODO(), sealedSegmentIDs)
|
||||
return exist && !hasStating
|
||||
}
|
||||
|
||||
func (t *l0CompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, error) {
|
||||
compactionParams, err := compaction.GenerateJSONParams()
|
||||
if err != nil {
|
||||
|
||||
@ -181,6 +181,7 @@ func (s *L0CompactionTaskSuite) TestBuildCompactionRequestFailed_AllocFailed() {
|
||||
task = &mixCompactionTask{
|
||||
allocator: s.mockAlloc,
|
||||
meta: meta,
|
||||
ievm: newMockVersionManager(),
|
||||
}
|
||||
task.SetTask(&datapb.CompactionTask{})
|
||||
_, err = task.BuildCompactionRequest()
|
||||
|
||||
@ -30,7 +30,11 @@ type mixCompactionTask struct {
|
||||
allocator allocator.Allocator
|
||||
meta CompactionMeta
|
||||
|
||||
ievm IndexEngineVersionManager
|
||||
|
||||
times *taskcommon.Times
|
||||
|
||||
slotUsage atomic.Int64
|
||||
}
|
||||
|
||||
func (t *mixCompactionTask) GetTaskID() int64 {
|
||||
@ -46,7 +50,18 @@ func (t *mixCompactionTask) GetTaskState() taskcommon.State {
|
||||
}
|
||||
|
||||
func (t *mixCompactionTask) GetTaskSlot() int64 {
|
||||
return paramtable.Get().DataCoordCfg.MixCompactionSlotUsage.GetAsInt64()
|
||||
slotUsage := t.slotUsage.Load()
|
||||
if slotUsage == 0 {
|
||||
slotUsage = paramtable.Get().DataCoordCfg.MixCompactionSlotUsage.GetAsInt64()
|
||||
if t.GetTaskProto().GetType() == datapb.CompactionType_SortCompaction {
|
||||
segment := t.meta.GetHealthySegment(context.Background(), t.GetTaskProto().GetInputSegments()[0])
|
||||
if segment != nil {
|
||||
slotUsage = calculateStatsTaskSlot(segment.getSegmentSize())
|
||||
}
|
||||
}
|
||||
t.slotUsage.Store(slotUsage)
|
||||
}
|
||||
return slotUsage
|
||||
}
|
||||
|
||||
func (t *mixCompactionTask) SetTaskTime(timeType taskcommon.TimeType, time time.Time) {
|
||||
@ -167,10 +182,14 @@ func (t *mixCompactionTask) GetTaskProto() *datapb.CompactionTask {
|
||||
return task.(*datapb.CompactionTask)
|
||||
}
|
||||
|
||||
func newMixCompactionTask(t *datapb.CompactionTask, allocator allocator.Allocator, meta CompactionMeta) *mixCompactionTask {
|
||||
func newMixCompactionTask(t *datapb.CompactionTask,
|
||||
allocator allocator.Allocator,
|
||||
meta CompactionMeta,
|
||||
ievm IndexEngineVersionManager) *mixCompactionTask {
|
||||
task := &mixCompactionTask{
|
||||
allocator: allocator,
|
||||
meta: meta,
|
||||
ievm: ievm,
|
||||
times: taskcommon.NewTimes(),
|
||||
}
|
||||
task.taskProto.Store(t)
|
||||
@ -205,6 +224,13 @@ func (t *mixCompactionTask) saveSegmentMeta(result *datapb.CompactionPlanResult)
|
||||
// Apply metrics after successful meta update.
|
||||
newSegmentIDs := lo.Map(newSegments, func(s *SegmentInfo, _ int) UniqueID { return s.GetID() })
|
||||
metricMutation.commit()
|
||||
for _, newSegID := range newSegmentIDs {
|
||||
select {
|
||||
case getBuildIndexChSingleton() <- newSegID:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
err = t.updateAndSaveTaskMeta(setState(datapb.CompactionTaskState_meta_saved), setResultSegments(newSegmentIDs))
|
||||
if err != nil {
|
||||
log.Warn("mixCompaction failed to setState meta saved", zap.Error(err))
|
||||
@ -318,10 +344,6 @@ func (t *mixCompactionTask) SetTask(task *datapb.CompactionTask) {
|
||||
t.taskProto.Store(task)
|
||||
}
|
||||
|
||||
func (t *mixCompactionTask) PreparePlan() bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func (t *mixCompactionTask) CheckCompactionContainsSegment(segmentID int64) bool {
|
||||
return false
|
||||
}
|
||||
@ -346,6 +368,7 @@ func (t *mixCompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, er
|
||||
SlotUsage: t.GetSlotUsage(),
|
||||
MaxSize: taskProto.GetMaxSize(),
|
||||
JsonParams: compactionParams,
|
||||
CurrentScalarIndexVersion: t.ievm.GetCurrentScalarIndexEngineVersion(),
|
||||
}
|
||||
|
||||
segIDMap := make(map[int64][]*datapb.FieldBinlog, len(plan.SegmentBinlogs))
|
||||
|
||||
@ -53,7 +53,7 @@ func (s *MixCompactionTaskSuite) TestProcessRefreshPlan_NormalMix() {
|
||||
State: datapb.CompactionTaskState_executing,
|
||||
InputSegments: []int64{200, 201},
|
||||
ResultSegments: []int64{100, 200},
|
||||
}, nil, s.mockMeta)
|
||||
}, nil, s.mockMeta, newMockVersionManager())
|
||||
alloc := allocator.NewMockAllocator(s.T())
|
||||
alloc.EXPECT().AllocN(mock.Anything).Return(100, 200, nil)
|
||||
task.allocator = alloc
|
||||
@ -84,7 +84,7 @@ func (s *MixCompactionTaskSuite) TestProcessRefreshPlan_MixSegmentNotFound() {
|
||||
NodeID: 1,
|
||||
InputSegments: []int64{200, 201},
|
||||
ResultSegments: []int64{100, 200},
|
||||
}, nil, s.mockMeta)
|
||||
}, nil, s.mockMeta, newMockVersionManager())
|
||||
_, err := task.BuildCompactionRequest()
|
||||
s.Error(err)
|
||||
s.ErrorIs(err, merr.ErrSegmentNotFound)
|
||||
@ -108,7 +108,7 @@ func (s *MixCompactionTaskSuite) TestProcess() {
|
||||
task := newMixCompactionTask(&datapb.CompactionTask{
|
||||
PlanID: 1,
|
||||
State: tc.state,
|
||||
}, nil, s.mockMeta)
|
||||
}, nil, s.mockMeta, newMockVersionManager())
|
||||
res := task.Process()
|
||||
s.Equal(tc.processResult, res)
|
||||
}
|
||||
|
||||
@ -333,6 +333,7 @@ func (t *compactionTrigger) handleSignal(signal *compactionSignal) error {
|
||||
return merr.WrapErrServiceQuotaExceeded("compaction handler full")
|
||||
}
|
||||
|
||||
log.Info("handleSignal receive")
|
||||
groups, err := t.getCandidates(signal)
|
||||
if err != nil {
|
||||
log.Warn("handle signal failed, get candidates return error", zap.Error(err))
|
||||
@ -549,7 +550,8 @@ func (t *compactionTrigger) getCandidates(signal *compactionSignal) ([]chanPartS
|
||||
!segment.GetIsImporting() && // not importing now
|
||||
segment.GetLevel() != datapb.SegmentLevel_L0 && // ignore level zero segments
|
||||
segment.GetLevel() != datapb.SegmentLevel_L2 && // ignore l2 segment
|
||||
!segment.GetIsInvisible()
|
||||
!segment.GetIsInvisible() &&
|
||||
segment.GetIsSorted()
|
||||
}),
|
||||
}
|
||||
|
||||
@ -816,3 +818,11 @@ func (t *compactionTrigger) squeezeSmallSegmentsToBuckets(small []*SegmentInfo,
|
||||
func getExpandedSize(size int64) int64 {
|
||||
return int64(float64(size) * Params.DataCoordCfg.SegmentExpansionRate.GetAsFloat())
|
||||
}
|
||||
|
||||
func canTriggerSortCompaction(segment *SegmentInfo) bool {
|
||||
return segment.GetState() == commonpb.SegmentState_Flushed &&
|
||||
segment.GetLevel() != datapb.SegmentLevel_L0 &&
|
||||
!segment.GetIsSorted() &&
|
||||
!segment.GetIsImporting() &&
|
||||
!segment.isCompacting
|
||||
}
|
||||
|
||||
@ -73,7 +73,7 @@ func (h *spyCompactionInspector) removeTasksByChannel(channel string) {}
|
||||
|
||||
// enqueueCompaction start to execute plan and return immediately
|
||||
func (h *spyCompactionInspector) enqueueCompaction(task *datapb.CompactionTask) error {
|
||||
t := newMixCompactionTask(task, nil, h.meta)
|
||||
t := newMixCompactionTask(task, nil, h.meta, newMockVersionManager())
|
||||
alloc := newMock0Allocator(h.t)
|
||||
t.allocator = alloc
|
||||
plan, err := t.BuildCompactionRequest()
|
||||
|
||||
@ -40,6 +40,7 @@ const (
|
||||
TriggerTypeSegmentSizeViewChange
|
||||
TriggerTypeClustering
|
||||
TriggerTypeSingle
|
||||
TriggerTypeSort
|
||||
)
|
||||
|
||||
func (t CompactionTriggerType) String() string {
|
||||
@ -275,6 +276,14 @@ func (m *CompactionTriggerManager) loop(ctx context.Context) {
|
||||
m.notify(ctx, triggerType, views)
|
||||
}
|
||||
}
|
||||
case segID := <-getStatsTaskChSingleton():
|
||||
log.Info("receive new segment to trigger sort compaction", zap.Int64("segmentID", segID))
|
||||
view := m.singlePolicy.triggerSegmentSortCompaction(ctx, segID)
|
||||
if view == nil {
|
||||
log.Warn("segment no need to do sort compaction", zap.Int64("segmentID", segID))
|
||||
continue
|
||||
}
|
||||
m.notify(ctx, TriggerTypeSort, []CompactionView{view})
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -317,7 +326,9 @@ func (m *CompactionTriggerManager) notify(ctx context.Context, eventType Compact
|
||||
case TriggerTypeClustering:
|
||||
m.SubmitClusteringViewToScheduler(ctx, outView)
|
||||
case TriggerTypeSingle:
|
||||
m.SubmitSingleViewToScheduler(ctx, outView)
|
||||
m.SubmitSingleViewToScheduler(ctx, outView, datapb.CompactionType_MixCompaction)
|
||||
case TriggerTypeSort:
|
||||
m.SubmitSingleViewToScheduler(ctx, outView, datapb.CompactionType_SortCompaction)
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -528,7 +539,7 @@ func (m *CompactionTriggerManager) SubmitClusteringViewToScheduler(ctx context.C
|
||||
)
|
||||
}
|
||||
|
||||
func (m *CompactionTriggerManager) SubmitSingleViewToScheduler(ctx context.Context, view CompactionView) {
|
||||
func (m *CompactionTriggerManager) SubmitSingleViewToScheduler(ctx context.Context, view CompactionView, compactionType datapb.CompactionType) {
|
||||
log := log.Ctx(ctx).With(zap.String("view", view.String()))
|
||||
// TODO[GOOSE], 11 = 1 planID + 10 segmentID, this is a hack need to be removed.
|
||||
// Any plan that output segment number greater than 10 will be marked as invalid plan for now.
|
||||
@ -557,7 +568,7 @@ func (m *CompactionTriggerManager) SubmitSingleViewToScheduler(ctx context.Conte
|
||||
StartTime: time.Now().Unix(),
|
||||
CollectionTtl: view.(*MixSegmentView).collectionTTL.Nanoseconds(),
|
||||
TimeoutInSeconds: Params.DataCoordCfg.ClusteringCompactionTimeoutInSeconds.GetAsInt32(),
|
||||
Type: datapb.CompactionType_MixCompaction, // todo: use SingleCompaction
|
||||
Type: compactionType, // todo: use SingleCompaction
|
||||
CollectionID: view.GetGroupLabel().CollectionID,
|
||||
PartitionID: view.GetGroupLabel().PartitionID,
|
||||
Channel: view.GetGroupLabel().Channel,
|
||||
|
||||
@ -31,7 +31,6 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/tsoutil"
|
||||
@ -48,7 +47,8 @@ type importChecker struct {
|
||||
broker broker.Broker
|
||||
alloc allocator.Allocator
|
||||
importMeta ImportMeta
|
||||
si StatsInspector
|
||||
ci CompactionInspector
|
||||
handler Handler
|
||||
l0CompactionTrigger TriggerManager
|
||||
|
||||
closeOnce sync.Once
|
||||
@ -60,7 +60,8 @@ func NewImportChecker(ctx context.Context,
|
||||
broker broker.Broker,
|
||||
alloc allocator.Allocator,
|
||||
importMeta ImportMeta,
|
||||
si StatsInspector,
|
||||
ci CompactionInspector,
|
||||
handler Handler,
|
||||
l0CompactionTrigger TriggerManager,
|
||||
) ImportChecker {
|
||||
return &importChecker{
|
||||
@ -69,8 +70,9 @@ func NewImportChecker(ctx context.Context,
|
||||
broker: broker,
|
||||
alloc: alloc,
|
||||
importMeta: importMeta,
|
||||
si: si,
|
||||
ci: ci,
|
||||
l0CompactionTrigger: l0CompactionTrigger,
|
||||
handler: handler,
|
||||
closeChan: make(chan struct{}),
|
||||
}
|
||||
}
|
||||
@ -106,8 +108,8 @@ func (c *importChecker) Start() {
|
||||
c.checkPreImportingJob(job)
|
||||
case internalpb.ImportJobState_Importing:
|
||||
c.checkImportingJob(job)
|
||||
case internalpb.ImportJobState_Stats:
|
||||
c.checkStatsJob(job)
|
||||
case internalpb.ImportJobState_Sorting:
|
||||
c.checkSortingJob(job)
|
||||
case internalpb.ImportJobState_IndexBuilding:
|
||||
c.checkIndexBuildingJob(job)
|
||||
case internalpb.ImportJobState_Failed:
|
||||
@ -312,7 +314,7 @@ func (c *importChecker) checkImportingJob(job ImportJob) {
|
||||
return
|
||||
}
|
||||
}
|
||||
err := c.importMeta.UpdateJob(c.ctx, job.GetJobID(), UpdateJobState(internalpb.ImportJobState_Stats))
|
||||
err := c.importMeta.UpdateJob(c.ctx, job.GetJobID(), UpdateJobState(internalpb.ImportJobState_Sorting))
|
||||
if err != nil {
|
||||
log.Warn("failed to update job state to Stats", zap.Error(err))
|
||||
return
|
||||
@ -322,7 +324,7 @@ func (c *importChecker) checkImportingJob(job ImportJob) {
|
||||
log.Info("import job import done", zap.Duration("jobTimeCost/import", importDuration))
|
||||
}
|
||||
|
||||
func (c *importChecker) checkStatsJob(job ImportJob) {
|
||||
func (c *importChecker) checkSortingJob(job ImportJob) {
|
||||
log := log.With(zap.Int64("jobID", job.GetJobID()))
|
||||
updateJobState := func(state internalpb.ImportJobState, reason string) {
|
||||
err := c.importMeta.UpdateJob(c.ctx, job.GetJobID(), UpdateJobState(state), UpdateJobReason(reason))
|
||||
@ -336,7 +338,8 @@ func (c *importChecker) checkStatsJob(job ImportJob) {
|
||||
}
|
||||
|
||||
// Skip stats stage if not enable stats or is l0 import.
|
||||
if !Params.DataCoordCfg.EnableStatsTask.GetAsBool() || importutilv2.IsL0Import(job.GetOptions()) {
|
||||
if !enableSortCompaction() ||
|
||||
importutilv2.IsL0Import(job.GetOptions()) {
|
||||
updateJobState(internalpb.ImportJobState_IndexBuilding, "")
|
||||
return
|
||||
}
|
||||
@ -349,28 +352,37 @@ func (c *importChecker) checkStatsJob(job ImportJob) {
|
||||
tasks := c.importMeta.GetTaskBy(c.ctx, WithType(ImportTaskType), WithJob(job.GetJobID()))
|
||||
for _, task := range tasks {
|
||||
originSegmentIDs := task.(*importTask).GetSegmentIDs()
|
||||
statsSegmentIDs := task.(*importTask).GetStatsSegmentIDs()
|
||||
sortSegmentIDs := task.(*importTask).GetSortedSegmentIDs()
|
||||
taskCnt += len(originSegmentIDs)
|
||||
for i, originSegmentID := range originSegmentIDs {
|
||||
taskLogFields := WrapTaskLog(task, zap.Int64("origin", originSegmentID), zap.Int64("stats", statsSegmentIDs[i]))
|
||||
t := c.si.GetStatsTask(originSegmentID, indexpb.StatsSubJob_Sort)
|
||||
switch t.GetState() {
|
||||
case indexpb.JobState_JobStateNone:
|
||||
err := c.si.SubmitStatsTask(originSegmentID, statsSegmentIDs[i], indexpb.StatsSubJob_Sort, false)
|
||||
if err != nil {
|
||||
log.Warn("submit stats task failed", zap.Error(err))
|
||||
taskLogFields := WrapTaskLog(task, zap.Int64("origin", originSegmentID), zap.Int64("target", sortSegmentIDs[i]))
|
||||
originSegment := c.meta.GetHealthySegment(c.ctx, originSegmentID)
|
||||
targetSegment := c.meta.GetHealthySegment(c.ctx, sortSegmentIDs[i])
|
||||
if originSegment == nil {
|
||||
// import zero num rows segment
|
||||
doneCnt++
|
||||
continue
|
||||
}
|
||||
log.Info("submit stats task done", taskLogFields...)
|
||||
case indexpb.JobState_JobStateInit, indexpb.JobState_JobStateRetry, indexpb.JobState_JobStateInProgress:
|
||||
log.Debug("waiting for stats task...", taskLogFields...)
|
||||
case indexpb.JobState_JobStateFailed:
|
||||
log.Warn("import job stats failed", taskLogFields...)
|
||||
updateJobState(internalpb.ImportJobState_Failed, t.GetFailReason())
|
||||
return
|
||||
case indexpb.JobState_JobStateFinished:
|
||||
doneCnt++
|
||||
if targetSegment == nil {
|
||||
compactionTask, err := createSortCompactionTask(c.ctx, originSegment, sortSegmentIDs[i], c.meta, c.handler, c.alloc)
|
||||
if err != nil {
|
||||
log.Warn("create sort compaction task failed", zap.Int64("segmentID", originSegmentID), zap.Error(err))
|
||||
continue
|
||||
}
|
||||
if compactionTask == nil {
|
||||
log.Info("maybe it no need to create sort compaction task", zap.Int64("segmentID", originSegmentID))
|
||||
doneCnt++
|
||||
continue
|
||||
}
|
||||
log.Info("create sort compaction task success", taskLogFields...)
|
||||
err = c.ci.enqueueCompaction(compactionTask)
|
||||
if err != nil {
|
||||
log.Warn("sort compaction task enqueue failed", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
continue
|
||||
}
|
||||
doneCnt++
|
||||
}
|
||||
}
|
||||
|
||||
@ -387,11 +399,11 @@ func (c *importChecker) checkIndexBuildingJob(job ImportJob) {
|
||||
return t.(*importTask).GetSegmentIDs()
|
||||
})
|
||||
statsSegmentIDs := lo.FlatMap(tasks, func(t ImportTask, _ int) []int64 {
|
||||
return t.(*importTask).GetStatsSegmentIDs()
|
||||
return t.(*importTask).GetSortedSegmentIDs()
|
||||
})
|
||||
|
||||
targetSegmentIDs := statsSegmentIDs
|
||||
if !Params.DataCoordCfg.EnableStatsTask.GetAsBool() {
|
||||
if !enableSortCompaction() {
|
||||
targetSegmentIDs = originSegmentIDs
|
||||
}
|
||||
|
||||
@ -490,17 +502,6 @@ func (c *importChecker) updateSegmentState(originSegmentIDs, statsSegmentIDs []i
|
||||
}
|
||||
|
||||
func (c *importChecker) checkFailedJob(job ImportJob) {
|
||||
tasks := c.importMeta.GetTaskBy(c.ctx, WithType(ImportTaskType), WithJob(job.GetJobID()))
|
||||
originSegmentIDs := lo.FlatMap(tasks, func(t ImportTask, _ int) []int64 {
|
||||
return t.(*importTask).GetSegmentIDs()
|
||||
})
|
||||
for _, originSegmentID := range originSegmentIDs {
|
||||
err := c.si.DropStatsTask(originSegmentID, indexpb.StatsSubJob_Sort)
|
||||
if err != nil {
|
||||
log.Warn("Drop stats task failed", zap.Int64("jobID", job.GetJobID()))
|
||||
return
|
||||
}
|
||||
}
|
||||
c.tryFailingTasks(job)
|
||||
}
|
||||
|
||||
@ -576,7 +577,7 @@ func (c *importChecker) checkGC(job ImportJob) {
|
||||
shouldRemoveJob := true
|
||||
for _, task := range tasks {
|
||||
if job.GetState() == internalpb.ImportJobState_Failed && task.GetType() == ImportTaskType {
|
||||
if len(task.(*importTask).GetSegmentIDs()) != 0 || len(task.(*importTask).GetStatsSegmentIDs()) != 0 {
|
||||
if len(task.(*importTask).GetSegmentIDs()) != 0 || len(task.(*importTask).GetSortedSegmentIDs()) != 0 {
|
||||
shouldRemoveJob = false
|
||||
continue
|
||||
}
|
||||
|
||||
@ -36,7 +36,6 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/metastore/mocks"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/rootcoordpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
@ -78,14 +77,21 @@ func (s *ImportCheckerSuite) SetupTest() {
|
||||
s.NoError(err)
|
||||
s.importMeta = importMeta
|
||||
|
||||
sjm := NewMockStatsJobManager(s.T())
|
||||
ci := NewMockCompactionInspector(s.T())
|
||||
l0CompactionTrigger := NewMockTriggerManager(s.T())
|
||||
compactionChan := make(chan struct{}, 1)
|
||||
close(compactionChan)
|
||||
l0CompactionTrigger.EXPECT().GetPauseCompactionChan(mock.Anything, mock.Anything).Return(compactionChan).Maybe()
|
||||
l0CompactionTrigger.EXPECT().GetResumeCompactionChan(mock.Anything, mock.Anything).Return(compactionChan).Maybe()
|
||||
|
||||
checker := NewImportChecker(context.TODO(), meta, broker, s.alloc, importMeta, sjm, l0CompactionTrigger).(*importChecker)
|
||||
handler := NewNMockHandler(s.T())
|
||||
handler.EXPECT().GetCollection(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, collID int64) (*collectionInfo, error) {
|
||||
return &collectionInfo{
|
||||
ID: collID,
|
||||
}, nil
|
||||
}).Maybe()
|
||||
|
||||
checker := NewImportChecker(context.TODO(), meta, broker, s.alloc, importMeta, ci, handler, l0CompactionTrigger).(*importChecker)
|
||||
s.checker = checker
|
||||
|
||||
job := &importJob{
|
||||
@ -220,6 +226,7 @@ func (s *ImportCheckerSuite) TestCheckJob() {
|
||||
catalog.EXPECT().AddSegment(mock.Anything, mock.Anything).Return(nil)
|
||||
catalog.EXPECT().AlterSegments(mock.Anything, mock.Anything).Return(nil)
|
||||
catalog.EXPECT().SaveChannelCheckpoint(mock.Anything, mock.Anything, mock.Anything).Return(nil)
|
||||
targetSegmentIDs := make([]int64, 0)
|
||||
for _, t := range importTasks {
|
||||
segment := &SegmentInfo{
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
@ -227,41 +234,45 @@ func (s *ImportCheckerSuite) TestCheckJob() {
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
IsImporting: true,
|
||||
InsertChannel: "ch0",
|
||||
NumOfRows: 1000,
|
||||
},
|
||||
}
|
||||
err := s.checker.meta.AddSegment(context.Background(), segment)
|
||||
s.NoError(err)
|
||||
targetSegmentID := rand.Int63()
|
||||
err = s.importMeta.UpdateTask(context.TODO(), t.GetTaskID(), UpdateState(datapb.ImportTaskStateV2_Completed),
|
||||
UpdateSegmentIDs([]int64{segment.GetID()}), UpdateStatsSegmentIDs([]int64{rand.Int63()}))
|
||||
UpdateSegmentIDs([]int64{segment.GetID()}), UpdateStatsSegmentIDs([]int64{targetSegmentID}))
|
||||
s.NoError(err)
|
||||
targetSegmentIDs = append(targetSegmentIDs, targetSegmentID)
|
||||
err = s.checker.meta.UpdateChannelCheckpoint(context.TODO(), segment.GetInsertChannel(), &msgpb.MsgPosition{MsgID: []byte{0}})
|
||||
s.NoError(err)
|
||||
}
|
||||
s.checker.checkImportingJob(job)
|
||||
s.Equal(internalpb.ImportJobState_Stats, s.importMeta.GetJob(context.TODO(), job.GetJobID()).GetState())
|
||||
s.Equal(internalpb.ImportJobState_Sorting, s.importMeta.GetJob(context.TODO(), job.GetJobID()).GetState())
|
||||
|
||||
// test check stats job
|
||||
alloc.EXPECT().AllocID(mock.Anything).Return(rand.Int63(), nil).Maybe()
|
||||
sjm := s.checker.si.(*MockStatsJobManager)
|
||||
sjm.EXPECT().SubmitStatsTask(mock.Anything, mock.Anything, mock.Anything, false).Return(nil)
|
||||
sjm.EXPECT().GetStatsTask(mock.Anything, mock.Anything).Return(&indexpb.StatsTask{
|
||||
State: indexpb.JobState_JobStateNone,
|
||||
})
|
||||
s.checker.checkStatsJob(job)
|
||||
s.Equal(internalpb.ImportJobState_Stats, s.importMeta.GetJob(context.TODO(), job.GetJobID()).GetState())
|
||||
sjm = NewMockStatsJobManager(s.T())
|
||||
sjm.EXPECT().GetStatsTask(mock.Anything, mock.Anything).Return(&indexpb.StatsTask{
|
||||
State: indexpb.JobState_JobStateInProgress,
|
||||
})
|
||||
s.checker.si = sjm
|
||||
s.checker.checkStatsJob(job)
|
||||
s.Equal(internalpb.ImportJobState_Stats, s.importMeta.GetJob(context.TODO(), job.GetJobID()).GetState())
|
||||
sjm = NewMockStatsJobManager(s.T())
|
||||
sjm.EXPECT().GetStatsTask(mock.Anything, mock.Anything).Return(&indexpb.StatsTask{
|
||||
State: indexpb.JobState_JobStateFinished,
|
||||
})
|
||||
s.checker.si = sjm
|
||||
s.checker.checkStatsJob(job)
|
||||
cim := s.checker.ci.(*MockCompactionInspector)
|
||||
cim.EXPECT().enqueueCompaction(mock.Anything).Return(nil)
|
||||
|
||||
s.checker.checkSortingJob(job)
|
||||
s.Equal(internalpb.ImportJobState_Sorting, s.importMeta.GetJob(context.TODO(), job.GetJobID()).GetState())
|
||||
|
||||
for _, segmentID := range targetSegmentIDs {
|
||||
segment := &SegmentInfo{
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
ID: segmentID,
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
IsImporting: true,
|
||||
InsertChannel: "ch0",
|
||||
IsSorted: true,
|
||||
},
|
||||
}
|
||||
err := s.checker.meta.AddSegment(context.Background(), segment)
|
||||
s.NoError(err)
|
||||
}
|
||||
|
||||
s.checker.checkSortingJob(job)
|
||||
s.Equal(internalpb.ImportJobState_IndexBuilding, s.importMeta.GetJob(context.TODO(), job.GetJobID()).GetState())
|
||||
|
||||
// test check IndexBuilding job
|
||||
@ -378,7 +389,7 @@ func (s *ImportCheckerSuite) TestCheckFailure() {
|
||||
TaskID: 1,
|
||||
State: datapb.ImportTaskStateV2_Pending,
|
||||
SegmentIDs: []int64{2},
|
||||
StatsSegmentIDs: []int64{3},
|
||||
SortedSegmentIDs: []int64{3},
|
||||
}
|
||||
it := &importTask{
|
||||
tr: timerecord.NewTimeRecorder("import task"),
|
||||
@ -387,20 +398,15 @@ func (s *ImportCheckerSuite) TestCheckFailure() {
|
||||
err := s.importMeta.AddTask(context.TODO(), it)
|
||||
s.NoError(err)
|
||||
|
||||
sjm := NewMockStatsJobManager(s.T())
|
||||
sjm.EXPECT().DropStatsTask(mock.Anything, mock.Anything).Return(errors.New("mock err"))
|
||||
s.checker.si = sjm
|
||||
s.checker.checkFailedJob(s.importMeta.GetJob(context.TODO(), s.jobID))
|
||||
tasks := s.importMeta.GetTaskBy(context.TODO(), WithJob(s.jobID), WithStates(datapb.ImportTaskStateV2_Failed))
|
||||
s.Equal(0, len(tasks))
|
||||
sjm.ExpectedCalls = nil
|
||||
sjm.EXPECT().DropStatsTask(mock.Anything, mock.Anything).Return(nil)
|
||||
s.Equal(1, len(tasks))
|
||||
|
||||
catalog.ExpectedCalls = nil
|
||||
catalog.EXPECT().SaveImportTask(mock.Anything, mock.Anything).Return(errors.New("mock error"))
|
||||
s.checker.checkFailedJob(s.importMeta.GetJob(context.TODO(), s.jobID))
|
||||
tasks = s.importMeta.GetTaskBy(context.TODO(), WithJob(s.jobID), WithStates(datapb.ImportTaskStateV2_Failed))
|
||||
s.Equal(0, len(tasks))
|
||||
s.Equal(1, len(tasks))
|
||||
|
||||
catalog.ExpectedCalls = nil
|
||||
catalog.EXPECT().SaveImportTask(mock.Anything, mock.Anything).Return(nil)
|
||||
@ -420,7 +426,7 @@ func (s *ImportCheckerSuite) TestCheckGC() {
|
||||
TaskID: 1,
|
||||
State: datapb.ImportTaskStateV2_Failed,
|
||||
SegmentIDs: []int64{2},
|
||||
StatsSegmentIDs: []int64{3},
|
||||
SortedSegmentIDs: []int64{3},
|
||||
}
|
||||
|
||||
task := &importTask{
|
||||
@ -580,14 +586,15 @@ func TestImportCheckerCompaction(t *testing.T) {
|
||||
importMeta, err := NewImportMeta(context.TODO(), catalog, alloc, meta)
|
||||
assert.NoError(t, err)
|
||||
|
||||
sjm := NewMockStatsJobManager(t)
|
||||
cim := NewMockCompactionInspector(t)
|
||||
handler := NewNMockHandler(t)
|
||||
l0CompactionTrigger := NewMockTriggerManager(t)
|
||||
compactionChan := make(chan struct{}, 1)
|
||||
close(compactionChan)
|
||||
l0CompactionTrigger.EXPECT().GetPauseCompactionChan(mock.Anything, mock.Anything).Return(compactionChan).Maybe()
|
||||
l0CompactionTrigger.EXPECT().GetResumeCompactionChan(mock.Anything, mock.Anything).Return(compactionChan).Maybe()
|
||||
|
||||
checker := NewImportChecker(context.TODO(), meta, broker, alloc, importMeta, sjm, l0CompactionTrigger).(*importChecker)
|
||||
checker := NewImportChecker(context.TODO(), meta, broker, alloc, importMeta, cim, handler, l0CompactionTrigger).(*importChecker)
|
||||
|
||||
job := &importJob{
|
||||
ImportJob: &datapb.ImportJob{
|
||||
@ -718,12 +725,13 @@ func TestImportCheckerCompaction(t *testing.T) {
|
||||
log.Info("job importing")
|
||||
|
||||
// check importing
|
||||
catalog.EXPECT().AddSegment(mock.Anything, mock.Anything).Return(nil).Once()
|
||||
catalog.EXPECT().AlterSegments(mock.Anything, mock.Anything).Return(nil).Once()
|
||||
catalog.EXPECT().AddSegment(mock.Anything, mock.Anything).Return(nil)
|
||||
catalog.EXPECT().AlterSegments(mock.Anything, mock.Anything).Return(nil)
|
||||
catalog.EXPECT().SaveChannelCheckpoint(mock.Anything, mock.Anything, mock.Anything).Return(nil).Once()
|
||||
catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once()
|
||||
catalog.EXPECT().SaveImportTask(mock.Anything, mock.Anything).Return(nil).Once()
|
||||
importTasks := importMeta.GetTaskBy(context.TODO(), WithJob(job.GetJobID()), WithType(ImportTaskType))
|
||||
targetSegmentIDs := make([]int64, 0)
|
||||
for _, it := range importTasks {
|
||||
segment := &SegmentInfo{
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
@ -735,23 +743,35 @@ func TestImportCheckerCompaction(t *testing.T) {
|
||||
}
|
||||
err := checker.meta.AddSegment(context.Background(), segment)
|
||||
assert.NoError(t, err)
|
||||
targetSegmentID := rand.Int63()
|
||||
err = importMeta.UpdateTask(context.TODO(), it.GetTaskID(), UpdateState(datapb.ImportTaskStateV2_Completed),
|
||||
UpdateSegmentIDs([]int64{segment.GetID()}), UpdateStatsSegmentIDs([]int64{rand.Int63()}))
|
||||
UpdateSegmentIDs([]int64{segment.GetID()}), UpdateStatsSegmentIDs([]int64{targetSegmentID}))
|
||||
assert.NoError(t, err)
|
||||
targetSegmentIDs = append(targetSegmentIDs, targetSegmentID)
|
||||
err = checker.meta.UpdateChannelCheckpoint(context.TODO(), segment.GetInsertChannel(), &msgpb.MsgPosition{MsgID: []byte{0}})
|
||||
assert.NoError(t, err)
|
||||
}
|
||||
assert.Eventually(t, func() bool {
|
||||
job := importMeta.GetJob(context.TODO(), jobID)
|
||||
return job.GetState() == internalpb.ImportJobState_Stats
|
||||
return job.GetState() == internalpb.ImportJobState_Sorting
|
||||
}, 2*time.Second, 100*time.Millisecond)
|
||||
log.Info("job stats")
|
||||
|
||||
// check stats
|
||||
catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once()
|
||||
sjm.EXPECT().GetStatsTask(mock.Anything, mock.Anything).Return(&indexpb.StatsTask{
|
||||
State: indexpb.JobState_JobStateFinished,
|
||||
}).Once()
|
||||
for _, targetSegmentID := range targetSegmentIDs {
|
||||
segment := &SegmentInfo{
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
ID: targetSegmentID,
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
IsImporting: true,
|
||||
InsertChannel: "ch0",
|
||||
IsSorted: true,
|
||||
},
|
||||
}
|
||||
err := checker.meta.AddSegment(context.Background(), segment)
|
||||
assert.NoError(t, err)
|
||||
}
|
||||
assert.Eventually(t, func() bool {
|
||||
job := importMeta.GetJob(context.TODO(), jobID)
|
||||
return job.GetState() == internalpb.ImportJobState_IndexBuilding
|
||||
|
||||
@ -132,7 +132,7 @@ func (s *importInspector) processPendingImport(task ImportTask) {
|
||||
func (s *importInspector) processFailed(task ImportTask) {
|
||||
if task.GetType() == ImportTaskType {
|
||||
originSegmentIDs := task.(*importTask).GetSegmentIDs()
|
||||
statsSegmentIDs := task.(*importTask).GetStatsSegmentIDs()
|
||||
statsSegmentIDs := task.(*importTask).GetSortedSegmentIDs()
|
||||
segments := append(originSegmentIDs, statsSegmentIDs...)
|
||||
for _, segment := range segments {
|
||||
op := UpdateStatusOperator(segment, commonpb.SegmentState_Dropped)
|
||||
|
||||
@ -210,7 +210,7 @@ func (s *ImportInspectorSuite) TestProcessFailed() {
|
||||
CollectionID: s.collectionID,
|
||||
NodeID: 6,
|
||||
SegmentIDs: []int64{2, 3},
|
||||
StatsSegmentIDs: []int64{4, 5},
|
||||
SortedSegmentIDs: []int64{4, 5},
|
||||
State: datapb.ImportTaskStateV2_Failed,
|
||||
}
|
||||
|
||||
|
||||
@ -140,7 +140,7 @@ func UpdateSegmentIDs(segmentIDs []UniqueID) UpdateAction {
|
||||
func UpdateStatsSegmentIDs(segmentIDs []UniqueID) UpdateAction {
|
||||
return func(t ImportTask) {
|
||||
if task, ok := t.(*importTask); ok {
|
||||
task.task.Load().StatsSegmentIDs = segmentIDs
|
||||
task.task.Load().SortedSegmentIDs = segmentIDs
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -96,8 +96,8 @@ func (t *importTask) GetSegmentIDs() []int64 {
|
||||
return t.task.Load().GetSegmentIDs()
|
||||
}
|
||||
|
||||
func (t *importTask) GetStatsSegmentIDs() []int64 {
|
||||
return t.task.Load().GetStatsSegmentIDs()
|
||||
func (t *importTask) GetSortedSegmentIDs() []int64 {
|
||||
return t.task.Load().GetSortedSegmentIDs()
|
||||
}
|
||||
|
||||
func (t *importTask) GetSource() datapb.ImportTaskSourceV2 {
|
||||
|
||||
@ -37,7 +37,6 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/util/importutilv2"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/taskcommon"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/conc"
|
||||
@ -125,13 +124,13 @@ func NewImportTasks(fileGroups [][]*datapb.ImportFileStats,
|
||||
return nil, err
|
||||
}
|
||||
taskProto.SegmentIDs = segments
|
||||
if paramtable.Get().DataCoordCfg.EnableStatsTask.GetAsBool() {
|
||||
statsSegIDBegin, _, err := alloc.AllocN(int64(len(segments)))
|
||||
if enableSortCompaction() {
|
||||
sortedSegIDBegin, _, err := alloc.AllocN(int64(len(segments)))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
taskProto.StatsSegmentIDs = lo.RangeFrom(statsSegIDBegin, len(segments))
|
||||
log.Info("preallocate stats segment ids", WrapTaskLog(task, zap.Int64s("segmentIDs", taskProto.StatsSegmentIDs))...)
|
||||
taskProto.SortedSegmentIDs = lo.RangeFrom(sortedSegIDBegin, len(segments))
|
||||
log.Info("preallocate sorted segment ids", WrapTaskLog(task, zap.Int64s("segmentIDs", taskProto.SortedSegmentIDs))...)
|
||||
}
|
||||
tasks = append(tasks, task)
|
||||
}
|
||||
@ -507,25 +506,25 @@ func getImportingProgress(ctx context.Context, jobID int64, importMeta ImportMet
|
||||
return float32(importedRows) / float32(totalRows), importedRows, totalRows
|
||||
}
|
||||
|
||||
func getStatsProgress(ctx context.Context, jobID int64, importMeta ImportMeta, sjm StatsInspector) float32 {
|
||||
if !Params.DataCoordCfg.EnableStatsTask.GetAsBool() {
|
||||
func getStatsProgress(ctx context.Context, jobID int64, importMeta ImportMeta, meta *meta) float32 {
|
||||
if !enableSortCompaction() {
|
||||
return 1
|
||||
}
|
||||
tasks := importMeta.GetTaskBy(ctx, WithJob(jobID), WithType(ImportTaskType))
|
||||
originSegmentIDs := lo.FlatMap(tasks, func(t ImportTask, _ int) []int64 {
|
||||
return t.(*importTask).GetSegmentIDs()
|
||||
targetSegmentIDs := lo.FlatMap(tasks, func(t ImportTask, _ int) []int64 {
|
||||
return t.(*importTask).GetSortedSegmentIDs()
|
||||
})
|
||||
if len(originSegmentIDs) == 0 {
|
||||
if len(targetSegmentIDs) == 0 {
|
||||
return 1
|
||||
}
|
||||
doneCnt := 0
|
||||
for _, originSegmentID := range originSegmentIDs {
|
||||
t := sjm.GetStatsTask(originSegmentID, indexpb.StatsSubJob_Sort)
|
||||
if t.GetState() == indexpb.JobState_JobStateFinished {
|
||||
for _, segID := range targetSegmentIDs {
|
||||
seg := meta.GetHealthySegment(ctx, segID)
|
||||
if seg != nil {
|
||||
doneCnt++
|
||||
}
|
||||
}
|
||||
return float32(doneCnt) / float32(len(originSegmentIDs))
|
||||
return float32(doneCnt) / float32(len(targetSegmentIDs))
|
||||
}
|
||||
|
||||
func getIndexBuildingProgress(ctx context.Context, jobID int64, importMeta ImportMeta, meta *meta) float32 {
|
||||
@ -538,12 +537,12 @@ func getIndexBuildingProgress(ctx context.Context, jobID int64, importMeta Impor
|
||||
return t.(*importTask).GetSegmentIDs()
|
||||
})
|
||||
targetSegmentIDs := lo.FlatMap(tasks, func(t ImportTask, _ int) []int64 {
|
||||
return t.(*importTask).GetStatsSegmentIDs()
|
||||
return t.(*importTask).GetSortedSegmentIDs()
|
||||
})
|
||||
if len(originSegmentIDs) == 0 {
|
||||
return 1
|
||||
}
|
||||
if !Params.DataCoordCfg.EnableStatsTask.GetAsBool() {
|
||||
if !enableSortCompaction() {
|
||||
targetSegmentIDs = originSegmentIDs
|
||||
}
|
||||
unindexed := meta.indexMeta.GetUnindexedSegments(job.GetCollectionID(), targetSegmentIDs)
|
||||
@ -561,7 +560,7 @@ func getIndexBuildingProgress(ctx context.Context, jobID int64, importMeta Impor
|
||||
// TODO: Wrap a function to map status to user status.
|
||||
// TODO: Save these progress to job instead of recalculating.
|
||||
func GetJobProgress(ctx context.Context, jobID int64,
|
||||
importMeta ImportMeta, meta *meta, sjm StatsInspector,
|
||||
importMeta ImportMeta, meta *meta,
|
||||
) (int64, internalpb.ImportJobState, int64, int64, string) {
|
||||
job := importMeta.GetJob(ctx, jobID)
|
||||
if job == nil {
|
||||
@ -580,8 +579,8 @@ func GetJobProgress(ctx context.Context, jobID int64,
|
||||
progress, importedRows, totalRows := getImportingProgress(ctx, jobID, importMeta, meta)
|
||||
return 10 + 30 + int64(progress*30), internalpb.ImportJobState_Importing, importedRows, totalRows, ""
|
||||
|
||||
case internalpb.ImportJobState_Stats:
|
||||
progress := getStatsProgress(ctx, jobID, importMeta, sjm)
|
||||
case internalpb.ImportJobState_Sorting:
|
||||
progress := getStatsProgress(ctx, jobID, importMeta, meta)
|
||||
_, totalRows := getImportRowsInfo(ctx, jobID, importMeta, meta)
|
||||
return 10 + 30 + 30 + int64(progress*10), internalpb.ImportJobState_Importing, totalRows, totalRows, ""
|
||||
|
||||
@ -786,3 +785,65 @@ func CalculateTaskSlot(task ImportTask, importMeta ImportMeta) int {
|
||||
}
|
||||
return memoryBasedSlots
|
||||
}
|
||||
|
||||
func createSortCompactionTask(ctx context.Context,
|
||||
originSegment *SegmentInfo,
|
||||
targetSegmentID int64,
|
||||
meta *meta,
|
||||
handler Handler,
|
||||
alloc allocator.Allocator) (*datapb.CompactionTask, error) {
|
||||
if originSegment.GetNumOfRows() == 0 {
|
||||
operator := UpdateStatusOperator(originSegment.GetID(), commonpb.SegmentState_Dropped)
|
||||
err := meta.UpdateSegmentsInfo(ctx, operator)
|
||||
if err != nil {
|
||||
log.Ctx(ctx).Warn("import zero num row segment, but mark it dropped failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
return nil, nil
|
||||
}
|
||||
collection, err := handler.GetCollection(ctx, originSegment.GetCollectionID())
|
||||
if err != nil {
|
||||
log.Warn("Failed to create sort compaction task because get collection fail", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
collectionTTL, err := getCollectionTTL(collection.Properties)
|
||||
if err != nil {
|
||||
log.Warn("failed to apply triggerSegmentSortCompaction, get collection ttl failed")
|
||||
return nil, err
|
||||
}
|
||||
|
||||
startID, _, err := alloc.AllocN(2)
|
||||
if err != nil {
|
||||
log.Warn("fFailed to submit compaction view to scheduler because allocate id fail", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
expectedSize := getExpectedSegmentSize(meta, collection)
|
||||
task := &datapb.CompactionTask{
|
||||
PlanID: startID + 1,
|
||||
TriggerID: startID,
|
||||
State: datapb.CompactionTaskState_pipelining,
|
||||
StartTime: time.Now().Unix(),
|
||||
CollectionTtl: collectionTTL.Nanoseconds(),
|
||||
TimeoutInSeconds: Params.DataCoordCfg.ClusteringCompactionTimeoutInSeconds.GetAsInt32(),
|
||||
Type: datapb.CompactionType_SortCompaction,
|
||||
CollectionID: originSegment.GetCollectionID(),
|
||||
PartitionID: originSegment.GetPartitionID(),
|
||||
Channel: originSegment.GetInsertChannel(),
|
||||
Schema: collection.Schema,
|
||||
InputSegments: []int64{originSegment.GetID()},
|
||||
ResultSegments: []int64{},
|
||||
TotalRows: originSegment.GetNumOfRows(),
|
||||
LastStateStartTime: time.Now().Unix(),
|
||||
MaxSize: getExpandedSize(expectedSize),
|
||||
PreAllocatedSegmentIDs: &datapb.IDRange{
|
||||
Begin: targetSegmentID,
|
||||
End: targetSegmentID + 1,
|
||||
},
|
||||
}
|
||||
|
||||
log.Ctx(ctx).Info("create sort compaction task success", zap.Int64("segmentID", originSegment.GetID()),
|
||||
zap.Int64("targetSegmentID", targetSegmentID), zap.Int64("num rows", originSegment.GetNumOfRows()))
|
||||
return task, nil
|
||||
}
|
||||
|
||||
@ -44,7 +44,6 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/importutilv2"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/rootcoordpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
@ -692,6 +691,7 @@ func TestImportUtil_GetImportProgress(t *testing.T) {
|
||||
TotalRows: 200,
|
||||
},
|
||||
},
|
||||
SortedSegmentIDs: []int64{100, 110, 120},
|
||||
}
|
||||
it1 := &importTask{}
|
||||
it1.task.Store(taskProto1)
|
||||
@ -721,6 +721,7 @@ func TestImportUtil_GetImportProgress(t *testing.T) {
|
||||
TotalRows: 300,
|
||||
},
|
||||
},
|
||||
SortedSegmentIDs: []int64{200, 210, 220},
|
||||
}
|
||||
it2 := &importTask{}
|
||||
it2.task.Store(taskProto2)
|
||||
@ -743,13 +744,13 @@ func TestImportUtil_GetImportProgress(t *testing.T) {
|
||||
err = importMeta.UpdateJob(context.TODO(), job.GetJobID(), UpdateJobState(internalpb.ImportJobState_Failed), UpdateJobReason(mockErr))
|
||||
assert.NoError(t, err)
|
||||
|
||||
progress, state, _, _, reason := GetJobProgress(ctx, job.GetJobID(), importMeta, meta, nil)
|
||||
progress, state, _, _, reason := GetJobProgress(ctx, job.GetJobID(), importMeta, meta)
|
||||
assert.Equal(t, int64(0), progress)
|
||||
assert.Equal(t, internalpb.ImportJobState_Failed, state)
|
||||
assert.Equal(t, mockErr, reason)
|
||||
|
||||
// job does not exist
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, -1, importMeta, meta, nil)
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, -1, importMeta, meta)
|
||||
assert.Equal(t, int64(0), progress)
|
||||
assert.Equal(t, internalpb.ImportJobState_Failed, state)
|
||||
assert.NotEqual(t, "", reason)
|
||||
@ -757,7 +758,7 @@ func TestImportUtil_GetImportProgress(t *testing.T) {
|
||||
// pending state
|
||||
err = importMeta.UpdateJob(context.TODO(), job.GetJobID(), UpdateJobState(internalpb.ImportJobState_Pending))
|
||||
assert.NoError(t, err)
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, job.GetJobID(), importMeta, meta, nil)
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, job.GetJobID(), importMeta, meta)
|
||||
assert.Equal(t, int64(10), progress)
|
||||
assert.Equal(t, internalpb.ImportJobState_Pending, state)
|
||||
assert.Equal(t, "", reason)
|
||||
@ -765,7 +766,7 @@ func TestImportUtil_GetImportProgress(t *testing.T) {
|
||||
// preImporting state
|
||||
err = importMeta.UpdateJob(context.TODO(), job.GetJobID(), UpdateJobState(internalpb.ImportJobState_PreImporting))
|
||||
assert.NoError(t, err)
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, job.GetJobID(), importMeta, meta, nil)
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, job.GetJobID(), importMeta, meta)
|
||||
assert.Equal(t, int64(10+30), progress)
|
||||
assert.Equal(t, internalpb.ImportJobState_Importing, state)
|
||||
assert.Equal(t, "", reason)
|
||||
@ -773,7 +774,7 @@ func TestImportUtil_GetImportProgress(t *testing.T) {
|
||||
// importing state, segmentImportedRows/totalRows = 0.5
|
||||
err = importMeta.UpdateJob(context.TODO(), job.GetJobID(), UpdateJobState(internalpb.ImportJobState_Importing))
|
||||
assert.NoError(t, err)
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, job.GetJobID(), importMeta, meta, nil)
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, job.GetJobID(), importMeta, meta)
|
||||
assert.Equal(t, int64(10+30+30*0.5), progress)
|
||||
assert.Equal(t, internalpb.ImportJobState_Importing, state)
|
||||
assert.Equal(t, "", reason)
|
||||
@ -791,36 +792,81 @@ func TestImportUtil_GetImportProgress(t *testing.T) {
|
||||
assert.NoError(t, err)
|
||||
err = meta.UpdateSegmentsInfo(context.TODO(), UpdateImportedRows(22, 100))
|
||||
assert.NoError(t, err)
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, job.GetJobID(), importMeta, meta, nil)
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, job.GetJobID(), importMeta, meta)
|
||||
assert.Equal(t, int64(float32(10+30+30)), progress)
|
||||
assert.Equal(t, internalpb.ImportJobState_Importing, state)
|
||||
assert.Equal(t, "", reason)
|
||||
|
||||
// stats state, len(statsSegmentIDs) / (len(originalSegmentIDs) = 0.5
|
||||
err = importMeta.UpdateJob(context.TODO(), job.GetJobID(), UpdateJobState(internalpb.ImportJobState_Stats))
|
||||
err = importMeta.UpdateJob(context.TODO(), job.GetJobID(), UpdateJobState(internalpb.ImportJobState_Sorting))
|
||||
assert.NoError(t, err)
|
||||
sjm := NewMockStatsJobManager(t)
|
||||
sjm.EXPECT().GetStatsTask(mock.Anything, mock.Anything).RunAndReturn(func(segmentID int64, _ indexpb.StatsSubJob) *indexpb.StatsTask {
|
||||
if lo.Contains([]int64{10, 11, 12}, segmentID) {
|
||||
return &indexpb.StatsTask{
|
||||
State: indexpb.JobState_JobStateFinished,
|
||||
}
|
||||
}
|
||||
return &indexpb.StatsTask{
|
||||
State: indexpb.JobState_JobStateInProgress,
|
||||
}
|
||||
|
||||
err = meta.AddSegment(ctx, &SegmentInfo{
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
ID: 100,
|
||||
IsImporting: true,
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
NumOfRows: 100,
|
||||
IsSorted: true,
|
||||
CompactionFrom: []int64{10},
|
||||
},
|
||||
})
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, job.GetJobID(), importMeta, meta, sjm)
|
||||
err = meta.AddSegment(ctx, &SegmentInfo{
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
ID: 110,
|
||||
IsImporting: true,
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
NumOfRows: 100,
|
||||
IsSorted: true,
|
||||
CompactionFrom: []int64{11},
|
||||
},
|
||||
})
|
||||
err = meta.AddSegment(ctx, &SegmentInfo{
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
ID: 120,
|
||||
IsImporting: true,
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
NumOfRows: 100,
|
||||
IsSorted: true,
|
||||
CompactionFrom: []int64{12},
|
||||
},
|
||||
})
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, job.GetJobID(), importMeta, meta)
|
||||
assert.Equal(t, int64(10+30+30+10*0.5), progress)
|
||||
assert.Equal(t, internalpb.ImportJobState_Importing, state)
|
||||
assert.Equal(t, "", reason)
|
||||
|
||||
// stats state, len(statsSegmentIDs) / (len(originalSegmentIDs) = 1
|
||||
sjm = NewMockStatsJobManager(t)
|
||||
sjm.EXPECT().GetStatsTask(mock.Anything, mock.Anything).Return(&indexpb.StatsTask{
|
||||
State: indexpb.JobState_JobStateFinished,
|
||||
err = meta.AddSegment(ctx, &SegmentInfo{
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
ID: 200,
|
||||
IsImporting: true,
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
NumOfRows: 100,
|
||||
IsSorted: true,
|
||||
CompactionFrom: []int64{20},
|
||||
},
|
||||
})
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, job.GetJobID(), importMeta, meta, sjm)
|
||||
err = meta.AddSegment(ctx, &SegmentInfo{
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
ID: 210,
|
||||
IsImporting: true,
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
NumOfRows: 100,
|
||||
IsSorted: true,
|
||||
CompactionFrom: []int64{21},
|
||||
},
|
||||
})
|
||||
err = meta.AddSegment(ctx, &SegmentInfo{
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
ID: 220,
|
||||
IsImporting: true,
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
NumOfRows: 100,
|
||||
IsSorted: true,
|
||||
CompactionFrom: []int64{22},
|
||||
},
|
||||
})
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, job.GetJobID(), importMeta, meta)
|
||||
assert.Equal(t, int64(10+30+30+10), progress)
|
||||
assert.Equal(t, internalpb.ImportJobState_Importing, state)
|
||||
assert.Equal(t, "", reason)
|
||||
@ -828,7 +874,7 @@ func TestImportUtil_GetImportProgress(t *testing.T) {
|
||||
// completed state
|
||||
err = importMeta.UpdateJob(context.TODO(), job.GetJobID(), UpdateJobState(internalpb.ImportJobState_Completed))
|
||||
assert.NoError(t, err)
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, job.GetJobID(), importMeta, meta, sjm)
|
||||
progress, state, _, _, reason = GetJobProgress(ctx, job.GetJobID(), importMeta, meta)
|
||||
assert.Equal(t, int64(100), progress)
|
||||
assert.Equal(t, internalpb.ImportJobState_Completed, state)
|
||||
assert.Equal(t, "", reason)
|
||||
|
||||
@ -106,7 +106,7 @@ func (i *indexInspector) createIndexForSegmentLoop(ctx context.Context) {
|
||||
case collectionID := <-i.notifyIndexChan:
|
||||
log.Info("receive create index notify", zap.Int64("collectionID", collectionID))
|
||||
segments := i.meta.SelectSegments(ctx, WithCollection(collectionID), SegmentFilterFunc(func(info *SegmentInfo) bool {
|
||||
return isFlush(info) && (!Params.DataCoordCfg.EnableStatsTask.GetAsBool() || info.GetIsSorted())
|
||||
return isFlush(info) && (!enableSortCompaction() || info.GetIsSorted())
|
||||
}))
|
||||
for _, segment := range segments {
|
||||
if err := i.createIndexesForSegment(ctx, segment); err != nil {
|
||||
@ -144,7 +144,7 @@ func (i *indexInspector) getUnIndexTaskSegments(ctx context.Context) []*SegmentI
|
||||
}
|
||||
|
||||
func (i *indexInspector) createIndexesForSegment(ctx context.Context, segment *SegmentInfo) error {
|
||||
if Params.DataCoordCfg.EnableStatsTask.GetAsBool() && !segment.GetIsSorted() {
|
||||
if enableSortCompaction() && !segment.GetIsSorted() {
|
||||
log.Ctx(ctx).Debug("segment is not sorted by pk, skip create indexes", zap.Int64("segmentID", segment.GetID()))
|
||||
return nil
|
||||
}
|
||||
|
||||
@ -67,8 +67,6 @@ type CompactionMeta interface {
|
||||
CheckAndSetSegmentsCompacting(ctx context.Context, segmentIDs []int64) (bool, bool)
|
||||
CompleteCompactionMutation(ctx context.Context, t *datapb.CompactionTask, result *datapb.CompactionPlanResult) ([]*SegmentInfo, *segMetricMutation, error)
|
||||
CleanPartitionStatsInfo(ctx context.Context, info *datapb.PartitionStatsInfo) error
|
||||
CheckSegmentsStating(ctx context.Context, segmentID []UniqueID) (bool, bool)
|
||||
SetSegmentStating(segmentID UniqueID, stating bool)
|
||||
|
||||
SaveCompactionTask(ctx context.Context, task *datapb.CompactionTask) error
|
||||
DropCompactionTask(ctx context.Context, task *datapb.CompactionTask) error
|
||||
@ -1487,31 +1485,6 @@ func (m *meta) SetLastWrittenTime(segmentID UniqueID) {
|
||||
m.segments.SetLastWrittenTime(segmentID)
|
||||
}
|
||||
|
||||
func (m *meta) CheckSegmentsStating(ctx context.Context, segmentIDs []UniqueID) (exist bool, hasStating bool) {
|
||||
m.segMu.RLock()
|
||||
defer m.segMu.RUnlock()
|
||||
exist = true
|
||||
for _, segmentID := range segmentIDs {
|
||||
seg := m.segments.GetSegment(segmentID)
|
||||
if seg != nil {
|
||||
if seg.isStating {
|
||||
hasStating = true
|
||||
}
|
||||
} else {
|
||||
exist = false
|
||||
break
|
||||
}
|
||||
}
|
||||
return exist, hasStating
|
||||
}
|
||||
|
||||
func (m *meta) SetSegmentStating(segmentID UniqueID, stating bool) {
|
||||
m.segMu.Lock()
|
||||
defer m.segMu.Unlock()
|
||||
|
||||
m.segments.SetIsStating(segmentID, stating)
|
||||
}
|
||||
|
||||
// SetSegmentCompacting sets compaction state for segment
|
||||
func (m *meta) SetSegmentCompacting(segmentID UniqueID, compacting bool) {
|
||||
m.segMu.Lock()
|
||||
@ -1653,7 +1626,10 @@ func (m *meta) completeClusterCompactionMutation(t *datapb.CompactionTask, resul
|
||||
return compactToSegInfos, metricMutation, nil
|
||||
}
|
||||
|
||||
func (m *meta) completeMixCompactionMutation(t *datapb.CompactionTask, result *datapb.CompactionPlanResult) ([]*SegmentInfo, *segMetricMutation, error) {
|
||||
func (m *meta) completeMixCompactionMutation(
|
||||
t *datapb.CompactionTask,
|
||||
result *datapb.CompactionPlanResult,
|
||||
) ([]*SegmentInfo, *segMetricMutation, error) {
|
||||
log := log.Ctx(context.TODO()).With(zap.Int64("planID", t.GetPlanID()),
|
||||
zap.String("type", t.GetType().String()),
|
||||
zap.Int64("collectionID", t.CollectionID),
|
||||
@ -1682,6 +1658,14 @@ func (m *meta) completeMixCompactionMutation(t *datapb.CompactionTask, result *d
|
||||
|
||||
log = log.With(zap.Int64s("compactFrom", compactFromSegIDs))
|
||||
|
||||
resultInvisible := false
|
||||
if t.GetType() == datapb.CompactionType_SortCompaction {
|
||||
resultInvisible = compactFromSegInfos[0].GetIsInvisible()
|
||||
if !compactFromSegInfos[0].GetCreatedByCompaction() {
|
||||
resultInvisible = false
|
||||
}
|
||||
}
|
||||
|
||||
compactToSegments := make([]*SegmentInfo, 0)
|
||||
for _, compactToSegment := range result.GetSegments() {
|
||||
compactToSegmentInfo := NewSegmentInfo(
|
||||
@ -1697,6 +1681,7 @@ func (m *meta) completeMixCompactionMutation(t *datapb.CompactionTask, result *d
|
||||
Statslogs: compactToSegment.GetField2StatslogPaths(),
|
||||
Deltalogs: compactToSegment.GetDeltalogs(),
|
||||
Bm25Statslogs: compactToSegment.GetBm25Logs(),
|
||||
TextStatsLogs: compactToSegment.GetTextStatsLogs(),
|
||||
|
||||
CreatedByCompaction: true,
|
||||
CompactionFrom: compactFromSegIDs,
|
||||
@ -1710,6 +1695,7 @@ func (m *meta) completeMixCompactionMutation(t *datapb.CompactionTask, result *d
|
||||
return info.GetDmlPosition()
|
||||
})),
|
||||
IsSorted: compactToSegment.GetIsSorted(),
|
||||
IsInvisible: resultInvisible,
|
||||
})
|
||||
|
||||
if compactToSegmentInfo.GetNumOfRows() == 0 {
|
||||
@ -1767,7 +1753,7 @@ func (m *meta) CompleteCompactionMutation(ctx context.Context, t *datapb.Compact
|
||||
m.segMu.Lock()
|
||||
defer m.segMu.Unlock()
|
||||
switch t.GetType() {
|
||||
case datapb.CompactionType_MixCompaction:
|
||||
case datapb.CompactionType_MixCompaction, datapb.CompactionType_SortCompaction:
|
||||
return m.completeMixCompactionMutation(t, result)
|
||||
case datapb.CompactionType_ClusteringCompaction:
|
||||
return m.completeClusterCompactionMutation(t, result)
|
||||
|
||||
@ -22,53 +22,6 @@ func (_m *MockCompactionInspector) EXPECT() *MockCompactionInspector_Expecter {
|
||||
return &MockCompactionInspector_Expecter{mock: &_m.Mock}
|
||||
}
|
||||
|
||||
// checkAndSetSegmentStating provides a mock function with given fields: channel, segmentID
|
||||
func (_m *MockCompactionInspector) checkAndSetSegmentStating(channel string, segmentID int64) bool {
|
||||
ret := _m.Called(channel, segmentID)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for checkAndSetSegmentStating")
|
||||
}
|
||||
|
||||
var r0 bool
|
||||
if rf, ok := ret.Get(0).(func(string, int64) bool); ok {
|
||||
r0 = rf(channel, segmentID)
|
||||
} else {
|
||||
r0 = ret.Get(0).(bool)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockCompactionInspector_checkAndSetSegmentStating_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'checkAndSetSegmentStating'
|
||||
type MockCompactionInspector_checkAndSetSegmentStating_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// checkAndSetSegmentStating is a helper method to define mock.On call
|
||||
// - channel string
|
||||
// - segmentID int64
|
||||
func (_e *MockCompactionInspector_Expecter) checkAndSetSegmentStating(channel interface{}, segmentID interface{}) *MockCompactionInspector_checkAndSetSegmentStating_Call {
|
||||
return &MockCompactionInspector_checkAndSetSegmentStating_Call{Call: _e.mock.On("checkAndSetSegmentStating", channel, segmentID)}
|
||||
}
|
||||
|
||||
func (_c *MockCompactionInspector_checkAndSetSegmentStating_Call) Run(run func(channel string, segmentID int64)) *MockCompactionInspector_checkAndSetSegmentStating_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(string), args[1].(int64))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockCompactionInspector_checkAndSetSegmentStating_Call) Return(_a0 bool) *MockCompactionInspector_checkAndSetSegmentStating_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockCompactionInspector_checkAndSetSegmentStating_Call) RunAndReturn(run func(string, int64) bool) *MockCompactionInspector_checkAndSetSegmentStating_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// enqueueCompaction provides a mock function with given fields: task
|
||||
func (_m *MockCompactionInspector) enqueueCompaction(task *datapb.CompactionTask) error {
|
||||
ret := _m.Called(task)
|
||||
|
||||
@ -79,63 +79,6 @@ func (_c *MockCompactionMeta_CheckAndSetSegmentsCompacting_Call) RunAndReturn(ru
|
||||
return _c
|
||||
}
|
||||
|
||||
// CheckSegmentsStating provides a mock function with given fields: ctx, segmentID
|
||||
func (_m *MockCompactionMeta) CheckSegmentsStating(ctx context.Context, segmentID []int64) (bool, bool) {
|
||||
ret := _m.Called(ctx, segmentID)
|
||||
|
||||
if len(ret) == 0 {
|
||||
panic("no return value specified for CheckSegmentsStating")
|
||||
}
|
||||
|
||||
var r0 bool
|
||||
var r1 bool
|
||||
if rf, ok := ret.Get(0).(func(context.Context, []int64) (bool, bool)); ok {
|
||||
return rf(ctx, segmentID)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(context.Context, []int64) bool); ok {
|
||||
r0 = rf(ctx, segmentID)
|
||||
} else {
|
||||
r0 = ret.Get(0).(bool)
|
||||
}
|
||||
|
||||
if rf, ok := ret.Get(1).(func(context.Context, []int64) bool); ok {
|
||||
r1 = rf(ctx, segmentID)
|
||||
} else {
|
||||
r1 = ret.Get(1).(bool)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// MockCompactionMeta_CheckSegmentsStating_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CheckSegmentsStating'
|
||||
type MockCompactionMeta_CheckSegmentsStating_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// CheckSegmentsStating is a helper method to define mock.On call
|
||||
// - ctx context.Context
|
||||
// - segmentID []int64
|
||||
func (_e *MockCompactionMeta_Expecter) CheckSegmentsStating(ctx interface{}, segmentID interface{}) *MockCompactionMeta_CheckSegmentsStating_Call {
|
||||
return &MockCompactionMeta_CheckSegmentsStating_Call{Call: _e.mock.On("CheckSegmentsStating", ctx, segmentID)}
|
||||
}
|
||||
|
||||
func (_c *MockCompactionMeta_CheckSegmentsStating_Call) Run(run func(ctx context.Context, segmentID []int64)) *MockCompactionMeta_CheckSegmentsStating_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(context.Context), args[1].([]int64))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockCompactionMeta_CheckSegmentsStating_Call) Return(_a0 bool, _a1 bool) *MockCompactionMeta_CheckSegmentsStating_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockCompactionMeta_CheckSegmentsStating_Call) RunAndReturn(run func(context.Context, []int64) (bool, bool)) *MockCompactionMeta_CheckSegmentsStating_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// CleanPartitionStatsInfo provides a mock function with given fields: ctx, info
|
||||
func (_m *MockCompactionMeta) CleanPartitionStatsInfo(ctx context.Context, info *datapb.PartitionStatsInfo) error {
|
||||
ret := _m.Called(ctx, info)
|
||||
@ -840,40 +783,6 @@ func (_c *MockCompactionMeta_SelectSegments_Call) RunAndReturn(run func(context.
|
||||
return _c
|
||||
}
|
||||
|
||||
// SetSegmentStating provides a mock function with given fields: segmentID, stating
|
||||
func (_m *MockCompactionMeta) SetSegmentStating(segmentID int64, stating bool) {
|
||||
_m.Called(segmentID, stating)
|
||||
}
|
||||
|
||||
// MockCompactionMeta_SetSegmentStating_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SetSegmentStating'
|
||||
type MockCompactionMeta_SetSegmentStating_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// SetSegmentStating is a helper method to define mock.On call
|
||||
// - segmentID int64
|
||||
// - stating bool
|
||||
func (_e *MockCompactionMeta_Expecter) SetSegmentStating(segmentID interface{}, stating interface{}) *MockCompactionMeta_SetSegmentStating_Call {
|
||||
return &MockCompactionMeta_SetSegmentStating_Call{Call: _e.mock.On("SetSegmentStating", segmentID, stating)}
|
||||
}
|
||||
|
||||
func (_c *MockCompactionMeta_SetSegmentStating_Call) Run(run func(segmentID int64, stating bool)) *MockCompactionMeta_SetSegmentStating_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(int64), args[1].(bool))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockCompactionMeta_SetSegmentStating_Call) Return() *MockCompactionMeta_SetSegmentStating_Call {
|
||||
_c.Call.Return()
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockCompactionMeta_SetSegmentStating_Call) RunAndReturn(run func(int64, bool)) *MockCompactionMeta_SetSegmentStating_Call {
|
||||
_c.Run(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// SetSegmentsCompacting provides a mock function with given fields: ctx, segmentID, compacting
|
||||
func (_m *MockCompactionMeta) SetSegmentsCompacting(ctx context.Context, segmentID []int64, compacting bool) {
|
||||
_m.Called(ctx, segmentID, compacting)
|
||||
|
||||
@ -59,9 +59,6 @@ type SegmentInfo struct {
|
||||
deltaRowcount atomic.Int64
|
||||
earliestTs atomic.Uint64
|
||||
lastWrittenTime time.Time
|
||||
|
||||
// It is only to ensure mutual exclusion between L0 compacting and stats tasks
|
||||
isStating bool
|
||||
}
|
||||
|
||||
func (s *SegmentInfo) GetResidualSegmentSize() int64 {
|
||||
@ -318,13 +315,6 @@ func (s *SegmentsInfo) SetIsCompacting(segmentID UniqueID, isCompacting bool) {
|
||||
}
|
||||
}
|
||||
|
||||
// SetIsStating sets stating status for segment
|
||||
func (s *SegmentsInfo) SetIsStating(segmentID UniqueID, isStating bool) {
|
||||
if segment, ok := s.segments[segmentID]; ok {
|
||||
s.segments[segmentID] = segment.ShadowClone(SetIsStating(isStating))
|
||||
}
|
||||
}
|
||||
|
||||
func (s *SegmentInfo) IsDeltaLogExists(logID int64) bool {
|
||||
for _, deltaLogs := range s.GetDeltalogs() {
|
||||
for _, l := range deltaLogs.GetBinlogs() {
|
||||
@ -509,13 +499,6 @@ func SetIsCompacting(isCompacting bool) SegmentInfoOption {
|
||||
}
|
||||
}
|
||||
|
||||
// SetIsStating is the option to set stats state for segment info
|
||||
func SetIsStating(isStating bool) SegmentInfoOption {
|
||||
return func(segment *SegmentInfo) {
|
||||
segment.isStating = isStating
|
||||
}
|
||||
}
|
||||
|
||||
// SetLevel is the option to set level for segment info
|
||||
func SetLevel(level datapb.SegmentLevel) SegmentInfoOption {
|
||||
return func(segment *SegmentInfo) {
|
||||
|
||||
@ -327,7 +327,7 @@ func (s *Server) initDataCoord() error {
|
||||
|
||||
s.importInspector = NewImportInspector(s.ctx, s.meta, s.importMeta, s.globalScheduler)
|
||||
|
||||
s.importChecker = NewImportChecker(s.ctx, s.meta, s.broker, s.allocator, s.importMeta, s.statsInspector, s.compactionTriggerManager)
|
||||
s.importChecker = NewImportChecker(s.ctx, s.meta, s.broker, s.allocator, s.importMeta, s.compactionInspector, s.handler, s.compactionTriggerManager)
|
||||
|
||||
s.serverLoopCtx, s.serverLoopCancel = context.WithCancel(s.ctx)
|
||||
|
||||
@ -664,7 +664,7 @@ func (s *Server) initStatsInspector() {
|
||||
}
|
||||
|
||||
func (s *Server) initCompaction() {
|
||||
cph := newCompactionInspector(s.meta, s.allocator, s.handler, s.globalScheduler)
|
||||
cph := newCompactionInspector(s.meta, s.allocator, s.handler, s.globalScheduler, s.indexEngineVersionManager)
|
||||
cph.loadMeta()
|
||||
s.compactionInspector = cph
|
||||
s.compactionTriggerManager = NewCompactionTriggerManager(s.allocator, s.handler, s.compactionInspector, s.meta, s.importMeta)
|
||||
@ -938,7 +938,7 @@ func (s *Server) postFlush(ctx context.Context, segmentID UniqueID) error {
|
||||
}
|
||||
// set segment to SegmentState_Flushed
|
||||
var operators []UpdateOperator
|
||||
if Params.DataCoordCfg.EnableStatsTask.GetAsBool() {
|
||||
if enableSortCompaction() {
|
||||
operators = append(operators, SetSegmentIsInvisible(segmentID, true))
|
||||
}
|
||||
operators = append(operators, UpdateStatusOperator(segmentID, commonpb.SegmentState_Flushed))
|
||||
@ -948,7 +948,7 @@ func (s *Server) postFlush(ctx context.Context, segmentID UniqueID) error {
|
||||
return err
|
||||
}
|
||||
|
||||
if Params.DataCoordCfg.EnableStatsTask.GetAsBool() {
|
||||
if enableSortCompaction() {
|
||||
select {
|
||||
case getStatsTaskChSingleton() <- segmentID:
|
||||
default:
|
||||
|
||||
@ -1669,7 +1669,7 @@ func TestGetCompactionState(t *testing.T) {
|
||||
{State: datapb.CompactionTaskState_timeout},
|
||||
{State: datapb.CompactionTaskState_timeout},
|
||||
})
|
||||
mockHandler := newCompactionInspector(mockMeta, nil, nil, nil)
|
||||
mockHandler := newCompactionInspector(mockMeta, nil, nil, nil, newMockVersionManager())
|
||||
svr.compactionInspector = mockHandler
|
||||
resp, err := svr.GetCompactionState(context.Background(), &milvuspb.GetCompactionStateRequest{CompactionID: 1})
|
||||
assert.NoError(t, err)
|
||||
|
||||
@ -1891,7 +1891,7 @@ func (s *Server) GetImportProgress(ctx context.Context, in *internalpb.GetImport
|
||||
resp.Status = merr.Status(merr.WrapErrImportFailed(fmt.Sprintf("import job does not exist, jobID=%d", jobID)))
|
||||
return resp, nil
|
||||
}
|
||||
progress, state, importedRows, totalRows, reason := GetJobProgress(ctx, jobID, s.importMeta, s.meta, s.statsInspector)
|
||||
progress, state, importedRows, totalRows, reason := GetJobProgress(ctx, jobID, s.importMeta, s.meta)
|
||||
resp.State = state
|
||||
resp.Reason = reason
|
||||
resp.Progress = progress
|
||||
@ -1928,7 +1928,7 @@ func (s *Server) ListImports(ctx context.Context, req *internalpb.ListImportsReq
|
||||
}
|
||||
|
||||
for _, job := range jobs {
|
||||
progress, state, _, _, reason := GetJobProgress(ctx, job.GetJobID(), s.importMeta, s.meta, s.statsInspector)
|
||||
progress, state, _, _, reason := GetJobProgress(ctx, job.GetJobID(), s.importMeta, s.meta)
|
||||
resp.JobIDs = append(resp.JobIDs, fmt.Sprintf("%d", job.GetJobID()))
|
||||
resp.States = append(resp.States, state)
|
||||
resp.Reasons = append(resp.Reasons, reason)
|
||||
|
||||
@ -82,13 +82,11 @@ func newStatsInspector(ctx context.Context,
|
||||
}
|
||||
|
||||
func (si *statsInspector) Start() {
|
||||
if Params.DataCoordCfg.EnableStatsTask.GetAsBool() {
|
||||
si.reloadFromMeta()
|
||||
si.loopWg.Add(2)
|
||||
go si.triggerStatsTaskLoop()
|
||||
go si.cleanupStatsTasksLoop()
|
||||
}
|
||||
}
|
||||
|
||||
func (si *statsInspector) Stop() {
|
||||
si.cancel()
|
||||
@ -112,7 +110,6 @@ func (si *statsInspector) reloadFromMeta() {
|
||||
proto.Clone(st).(*indexpb.StatsTask),
|
||||
taskSlot,
|
||||
si.mt,
|
||||
si.compactionInspector,
|
||||
si.handler,
|
||||
si.allocator,
|
||||
si.ievm,
|
||||
@ -135,58 +132,10 @@ func (si *statsInspector) triggerStatsTaskLoop() {
|
||||
log.Warn("DataCoord context done, exit checkStatsTaskLoop...")
|
||||
return
|
||||
case <-ticker.C:
|
||||
si.triggerSortStatsTask()
|
||||
si.triggerTextStatsTask()
|
||||
si.triggerBM25StatsTask()
|
||||
lastJSONStatsLastTrigger, maxJSONStatsTaskCount = si.triggerJsonKeyIndexStatsTask(lastJSONStatsLastTrigger, maxJSONStatsTaskCount)
|
||||
|
||||
case segID := <-getStatsTaskChSingleton():
|
||||
log.Info("receive new segment to trigger stats task", zap.Int64("segmentID", segID))
|
||||
segment := si.mt.GetSegment(si.ctx, segID)
|
||||
if segment == nil {
|
||||
log.Warn("segment is not exist, no need to do stats task", zap.Int64("segmentID", segID))
|
||||
continue
|
||||
}
|
||||
si.createSortStatsTaskForSegment(segment)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (si *statsInspector) triggerSortStatsTask() {
|
||||
invisibleSegments := si.mt.SelectSegments(si.ctx, SegmentFilterFunc(func(seg *SegmentInfo) bool {
|
||||
return isFlushed(seg) && seg.GetLevel() != datapb.SegmentLevel_L0 && !seg.GetIsSorted() &&
|
||||
!seg.GetIsImporting() && seg.GetIsInvisible()
|
||||
}))
|
||||
|
||||
for _, seg := range invisibleSegments {
|
||||
si.createSortStatsTaskForSegment(seg)
|
||||
}
|
||||
|
||||
visibleSegments := si.mt.SelectSegments(si.ctx, SegmentFilterFunc(func(seg *SegmentInfo) bool {
|
||||
return isFlushed(seg) && seg.GetLevel() != datapb.SegmentLevel_L0 && !seg.GetIsSorted() &&
|
||||
!seg.GetIsImporting() && !seg.GetIsInvisible()
|
||||
}))
|
||||
|
||||
for _, segment := range visibleSegments {
|
||||
// TODO @xiaocai2333: add trigger count limit
|
||||
// if jm.scheduler.pendingTasks.TaskCount() > Params.DataCoordCfg.StatsTaskTriggerCount.GetAsInt() {
|
||||
// break
|
||||
// }
|
||||
si.createSortStatsTaskForSegment(segment)
|
||||
}
|
||||
}
|
||||
|
||||
func (si *statsInspector) createSortStatsTaskForSegment(segment *SegmentInfo) {
|
||||
targetSegmentID, err := si.allocator.AllocID(si.ctx)
|
||||
if err != nil {
|
||||
log.Warn("allocID for segment stats task failed",
|
||||
zap.Int64("segmentID", segment.GetID()), zap.Error(err))
|
||||
return
|
||||
}
|
||||
if err := si.SubmitStatsTask(segment.GetID(), targetSegmentID, indexpb.StatsSubJob_Sort, true); err != nil {
|
||||
log.Warn("create stats task with sort for segment failed, wait for retry",
|
||||
zap.Int64("segmentID", segment.GetID()), zap.Error(err))
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
@ -384,7 +333,7 @@ func (si *statsInspector) SubmitStatsTask(originSegmentID, targetSegmentID int64
|
||||
}
|
||||
return err
|
||||
}
|
||||
si.scheduler.Enqueue(newStatsTask(proto.Clone(t).(*indexpb.StatsTask), taskSlot, si.mt, si.compactionInspector, si.handler, si.allocator, si.ievm))
|
||||
si.scheduler.Enqueue(newStatsTask(proto.Clone(t).(*indexpb.StatsTask), taskSlot, si.mt, si.handler, si.allocator, si.ievm))
|
||||
log.Ctx(si.ctx).Info("submit stats task success", zap.Int64("taskID", taskID),
|
||||
zap.String("subJobType", subJobType.String()),
|
||||
zap.Int64("collectionID", originSegment.GetCollectionID()),
|
||||
|
||||
@ -60,7 +60,6 @@ func (s *statsInspectorSuite) SetupTest() {
|
||||
|
||||
Params.Save(Params.DataCoordCfg.TaskCheckInterval.Key, "1")
|
||||
Params.Save(Params.DataCoordCfg.GCInterval.Key, "1")
|
||||
Params.Save(Params.DataCoordCfg.EnableStatsTask.Key, "true")
|
||||
|
||||
s.alloc = allocator.NewMockAllocator(s.T())
|
||||
var idCounter int64 = 1000
|
||||
@ -291,26 +290,6 @@ func (s *statsInspectorSuite) TestDropStatsTask() {
|
||||
s.NoError(err) // Non-existent tasks should return success
|
||||
}
|
||||
|
||||
func (s *statsInspectorSuite) TestTriggerSortStatsTask() {
|
||||
// Test triggering sort stats task
|
||||
s.inspector.triggerSortStatsTask()
|
||||
|
||||
// Verify AllocID was called
|
||||
s.alloc.AssertCalled(s.T(), "AllocID", mock.Anything)
|
||||
s.Nil(s.mt.statsTaskMeta.GetStatsTaskBySegmentID(30, indexpb.StatsSubJob_Sort))
|
||||
}
|
||||
|
||||
func (s *statsInspectorSuite) TestCreateSortStatsTaskForSegment() {
|
||||
// Get a test segment
|
||||
segment := s.mt.segments.segments[10]
|
||||
|
||||
// Test creating sort stats task for segment
|
||||
s.inspector.createSortStatsTaskForSegment(segment)
|
||||
|
||||
// Verify AllocID was called
|
||||
s.alloc.AssertCalled(s.T(), "AllocID", mock.Anything)
|
||||
}
|
||||
|
||||
func (s *statsInspectorSuite) TestTriggerTextStatsTask() {
|
||||
// Set up a sorted segment without text index
|
||||
segment := s.mt.segments.segments[20]
|
||||
|
||||
@ -74,6 +74,16 @@ func (stm *statsTaskMeta) reloadFromKV() error {
|
||||
return err
|
||||
}
|
||||
for _, t := range statsTasks {
|
||||
// sort stats task no need to reload
|
||||
if t.GetSubJobType() == indexpb.StatsSubJob_Sort {
|
||||
if err := stm.catalog.DropStatsTask(stm.ctx, t.GetTaskID()); err != nil {
|
||||
log.Warn("drop stats task failed",
|
||||
zap.Int64("taskID", t.GetTaskID()),
|
||||
zap.Int64("segmentID", t.GetSegmentID()),
|
||||
zap.Error(err))
|
||||
}
|
||||
continue
|
||||
}
|
||||
stm.tasks.Insert(t.GetTaskID(), t)
|
||||
|
||||
secondaryKey := createSecondaryIndexKey(t.GetSegmentID(), t.GetSubJobType().String())
|
||||
@ -153,7 +163,7 @@ func (stm *statsTaskMeta) DropStatsTask(ctx context.Context, taskID int64) error
|
||||
if err := stm.catalog.DropStatsTask(ctx, taskID); err != nil {
|
||||
log.Warn("drop stats task failed",
|
||||
zap.Int64("taskID", taskID),
|
||||
zap.Int64("segmentID", taskID),
|
||||
zap.Int64("segmentID", t.GetSegmentID()),
|
||||
zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
@ -69,6 +69,8 @@ func (s *statsTaskMetaSuite) Test_Method() {
|
||||
},
|
||||
}, nil)
|
||||
|
||||
catalog.EXPECT().DropStatsTask(mock.Anything, mock.Anything).Return(nil)
|
||||
|
||||
m, err := newStatsTaskMeta(context.Background(), catalog)
|
||||
s.NoError(err)
|
||||
s.NotNil(m)
|
||||
|
||||
@ -33,7 +33,6 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/taskcommon"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/tsoutil"
|
||||
)
|
||||
|
||||
type statsTask struct {
|
||||
@ -46,7 +45,6 @@ type statsTask struct {
|
||||
meta *meta
|
||||
handler Handler
|
||||
allocator allocator.Allocator
|
||||
compactionInspector CompactionInspector
|
||||
ievm IndexEngineVersionManager
|
||||
}
|
||||
|
||||
@ -55,7 +53,6 @@ var _ globalTask.Task = (*statsTask)(nil)
|
||||
func newStatsTask(t *indexpb.StatsTask,
|
||||
taskSlot int64,
|
||||
mt *meta,
|
||||
inspector CompactionInspector,
|
||||
handler Handler,
|
||||
allocator allocator.Allocator,
|
||||
ievm IndexEngineVersionManager,
|
||||
@ -67,7 +64,6 @@ func newStatsTask(t *indexpb.StatsTask,
|
||||
meta: mt,
|
||||
handler: handler,
|
||||
allocator: allocator,
|
||||
compactionInspector: inspector,
|
||||
ievm: ievm,
|
||||
}
|
||||
}
|
||||
@ -126,12 +122,6 @@ func (st *statsTask) UpdateTaskVersion(nodeID int64) error {
|
||||
}
|
||||
|
||||
func (st *statsTask) resetTask(ctx context.Context, reason string) {
|
||||
// reset isCompacting
|
||||
if st.GetSubJobType() == indexpb.StatsSubJob_Sort {
|
||||
st.meta.SetSegmentsCompacting(ctx, []UniqueID{st.GetSegmentID()}, false)
|
||||
st.meta.SetSegmentStating(st.GetSegmentID(), false)
|
||||
}
|
||||
|
||||
// reset state to init
|
||||
st.UpdateStateWithMeta(indexpb.JobState_JobStateInit, reason)
|
||||
}
|
||||
@ -154,33 +144,9 @@ func (st *statsTask) CreateTaskOnWorker(nodeID int64, cluster session.Cluster) {
|
||||
zap.String("subJobType", st.GetSubJobType().String()),
|
||||
)
|
||||
|
||||
// Check segment compaction state
|
||||
if st.GetSubJobType() == indexpb.StatsSubJob_Sort {
|
||||
if exist, canCompact := st.meta.CheckAndSetSegmentsCompacting(ctx, []UniqueID{st.GetSegmentID()}); !exist || !canCompact {
|
||||
log.Warn("segment is not exist or is compacting, skip stats and remove stats task",
|
||||
zap.Bool("exist", exist), zap.Bool("canCompact", canCompact))
|
||||
|
||||
if err := st.meta.statsTaskMeta.DropStatsTask(ctx, st.GetTaskID()); err != nil {
|
||||
log.Warn("remove stats task failed, will retry later", zap.Error(err))
|
||||
return
|
||||
}
|
||||
st.SetState(indexpb.JobState_JobStateNone, "segment is not exist or is compacting")
|
||||
return
|
||||
}
|
||||
|
||||
// Check if segment is part of L0 compaction
|
||||
if !st.compactionInspector.checkAndSetSegmentStating(st.GetInsertChannel(), st.GetSegmentID()) {
|
||||
log.Warn("segment is contained by L0 compaction, skipping stats task")
|
||||
// Reset isCompacting flag
|
||||
st.meta.SetSegmentsCompacting(ctx, []UniqueID{st.GetSegmentID()}, false)
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
var err error
|
||||
defer func() {
|
||||
if err != nil {
|
||||
// reset isCompacting flag and stating flag
|
||||
st.resetTask(ctx, err.Error())
|
||||
}
|
||||
}()
|
||||
@ -336,11 +302,6 @@ func (st *statsTask) prepareJobRequest(ctx context.Context, segment *SegmentInfo
|
||||
return nil, fmt.Errorf("failed to get collection info: %w", err)
|
||||
}
|
||||
|
||||
collTtl, err := getCollectionTTL(collInfo.Properties)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to get collection TTL: %w", err)
|
||||
}
|
||||
|
||||
// Calculate binlog allocation
|
||||
binlogNum := (segment.getSegmentSize()/Params.DataNodeCfg.BinLogMaxSize.GetAsInt64() + 1) *
|
||||
int64(len(collInfo.Schema.GetFields())) *
|
||||
@ -365,19 +326,14 @@ func (st *statsTask) prepareJobRequest(ctx context.Context, segment *SegmentInfo
|
||||
SubJobType: st.GetSubJobType(),
|
||||
TargetSegmentID: st.GetTargetSegmentID(),
|
||||
InsertLogs: segment.GetBinlogs(),
|
||||
DeltaLogs: segment.GetDeltalogs(),
|
||||
StartLogID: start,
|
||||
EndLogID: end,
|
||||
NumRows: segment.GetNumOfRows(),
|
||||
CollectionTtl: collTtl.Nanoseconds(),
|
||||
CurrentTs: tsoutil.GetCurrentTime(),
|
||||
// update version after check
|
||||
TaskVersion: st.GetVersion(),
|
||||
BinlogMaxSize: Params.DataNodeCfg.BinLogMaxSize.GetAsUint64(),
|
||||
EnableJsonKeyStats: Params.CommonCfg.EnabledJSONKeyStats.GetAsBool(),
|
||||
JsonKeyStatsTantivyMemory: Params.DataCoordCfg.JSONKeyStatsMemoryBudgetInTantivy.GetAsInt64(),
|
||||
JsonKeyStatsDataFormat: 1,
|
||||
EnableJsonKeyStatsInSort: Params.DataCoordCfg.EnabledJSONKeyStatsInSort.GetAsBool(),
|
||||
TaskSlot: st.taskSlot,
|
||||
StorageVersion: segment.StorageVersion,
|
||||
CurrentScalarIndexVersion: st.ievm.GetCurrentScalarIndexEngineVersion(),
|
||||
@ -389,21 +345,6 @@ func (st *statsTask) prepareJobRequest(ctx context.Context, segment *SegmentInfo
|
||||
func (st *statsTask) SetJobInfo(ctx context.Context, result *workerpb.StatsResult) error {
|
||||
var err error
|
||||
switch st.GetSubJobType() {
|
||||
case indexpb.StatsSubJob_Sort:
|
||||
// first update segment, failed state cannot generate new segment
|
||||
var metricMutation *segMetricMutation
|
||||
metricMutation, err = st.meta.SaveStatsResultSegment(st.GetSegmentID(), result)
|
||||
if err != nil {
|
||||
log.Ctx(ctx).Warn("save sort stats result failed", zap.Int64("taskID", st.GetTaskID()),
|
||||
zap.Int64("segmentID", st.GetSegmentID()), zap.Error(err))
|
||||
break
|
||||
}
|
||||
metricMutation.commit()
|
||||
|
||||
select {
|
||||
case getBuildIndexChSingleton() <- result.GetSegmentID():
|
||||
default:
|
||||
}
|
||||
case indexpb.StatsSubJob_TextIndexJob:
|
||||
err = st.meta.UpdateSegment(st.GetSegmentID(), SetTextIndexLogs(result.GetTextStatsLogs()))
|
||||
if err != nil {
|
||||
@ -420,6 +361,8 @@ func (st *statsTask) SetJobInfo(ctx context.Context, result *workerpb.StatsResul
|
||||
}
|
||||
case indexpb.StatsSubJob_BM25Job:
|
||||
// bm25 logs are generated during with segment flush.
|
||||
default:
|
||||
log.Ctx(ctx).Warn("unexpected sub job type", zap.String("type", st.GetSubJobType().String()))
|
||||
}
|
||||
|
||||
// if segment is not found, it means the segment is already dropped,
|
||||
@ -427,11 +370,6 @@ func (st *statsTask) SetJobInfo(ctx context.Context, result *workerpb.StatsResul
|
||||
if err != nil && !errors.Is(err, merr.ErrSegmentNotFound) {
|
||||
return err
|
||||
}
|
||||
// Reset isCompacting flag after stats task is finished
|
||||
if st.GetSubJobType() == indexpb.StatsSubJob_Sort {
|
||||
st.meta.SetSegmentsCompacting(ctx, []UniqueID{st.GetSegmentID()}, false)
|
||||
st.meta.SetSegmentStating(st.GetSegmentID(), false)
|
||||
}
|
||||
log.Ctx(ctx).Info("SetJobInfo for stats task success", zap.Int64("taskID", st.GetTaskID()),
|
||||
zap.Int64("oldSegmentID", st.GetSegmentID()), zap.Int64("targetSegmentID", st.GetTargetSegmentID()),
|
||||
zap.String("subJobType", st.GetSubJobType().String()), zap.String("state", st.GetState().String()))
|
||||
|
||||
@ -69,7 +69,7 @@ func (s *statsTaskSuite) SetupSuite() {
|
||||
SegmentID: s.segID,
|
||||
InsertChannel: "ch1",
|
||||
TaskID: s.taskID,
|
||||
SubJobType: indexpb.StatsSubJob_Sort,
|
||||
SubJobType: indexpb.StatsSubJob_JsonKeyIndexJob,
|
||||
Version: 0,
|
||||
NodeID: 0,
|
||||
State: indexpb.JobState_JobStateInit,
|
||||
@ -149,9 +149,9 @@ func (s *statsTaskSuite) TestBasicTaskOperations() {
|
||||
TaskID: s.taskID,
|
||||
SegmentID: s.segID,
|
||||
TargetSegmentID: s.targetID,
|
||||
SubJobType: indexpb.StatsSubJob_Sort,
|
||||
SubJobType: indexpb.StatsSubJob_JsonKeyIndexJob,
|
||||
State: indexpb.JobState_JobStateInit,
|
||||
}, 1, s.mt, nil, nil, nil, newIndexEngineVersionManager())
|
||||
}, 1, s.mt, nil, nil, newIndexEngineVersionManager())
|
||||
|
||||
s.Run("task type and state", func() {
|
||||
s.Equal(taskcommon.Stats, st.GetTaskType())
|
||||
@ -183,11 +183,11 @@ func (s *statsTaskSuite) TestUpdateStateAndVersion() {
|
||||
st := newStatsTask(&indexpb.StatsTask{
|
||||
TaskID: s.taskID,
|
||||
SegmentID: s.segID,
|
||||
SubJobType: indexpb.StatsSubJob_Sort,
|
||||
SubJobType: indexpb.StatsSubJob_JsonKeyIndexJob,
|
||||
State: indexpb.JobState_JobStateInit,
|
||||
Version: 1,
|
||||
NodeID: 0,
|
||||
}, 1, s.mt, nil, nil, nil, newIndexEngineVersionManager())
|
||||
}, 1, s.mt, nil, nil, newIndexEngineVersionManager())
|
||||
|
||||
s.Run("update state success", func() {
|
||||
catalog := catalogmocks.NewDataCoordCatalog(s.T())
|
||||
@ -235,9 +235,9 @@ func (s *statsTaskSuite) TestResetTask() {
|
||||
st := newStatsTask(&indexpb.StatsTask{
|
||||
TaskID: s.taskID,
|
||||
SegmentID: s.segID,
|
||||
SubJobType: indexpb.StatsSubJob_Sort,
|
||||
SubJobType: indexpb.StatsSubJob_JsonKeyIndexJob,
|
||||
State: indexpb.JobState_JobStateInProgress,
|
||||
}, 1, s.mt, nil, nil, nil, newIndexEngineVersionManager())
|
||||
}, 1, s.mt, nil, nil, newIndexEngineVersionManager())
|
||||
|
||||
s.Run("reset success", func() {
|
||||
catalog.EXPECT().SaveStatsTask(mock.Anything, mock.Anything).Return(nil)
|
||||
@ -262,9 +262,9 @@ func (s *statsTaskSuite) TestHandleEmptySegment() {
|
||||
st := newStatsTask(&indexpb.StatsTask{
|
||||
TaskID: s.taskID,
|
||||
SegmentID: s.segID,
|
||||
SubJobType: indexpb.StatsSubJob_Sort,
|
||||
SubJobType: indexpb.StatsSubJob_JsonKeyIndexJob,
|
||||
State: indexpb.JobState_JobStateInit,
|
||||
}, 1, s.mt, nil, handler, nil, newIndexEngineVersionManager())
|
||||
}, 1, s.mt, handler, nil, newIndexEngineVersionManager())
|
||||
|
||||
s.Run("handle empty segment success", func() {
|
||||
catalog := catalogmocks.NewDataCoordCatalog(s.T())
|
||||
@ -291,43 +291,9 @@ func (s *statsTaskSuite) TestCreateTaskOnWorker() {
|
||||
TaskID: s.taskID,
|
||||
SegmentID: s.segID,
|
||||
TargetSegmentID: s.targetID,
|
||||
SubJobType: indexpb.StatsSubJob_Sort,
|
||||
SubJobType: indexpb.StatsSubJob_JsonKeyIndexJob,
|
||||
State: indexpb.JobState_JobStateInit,
|
||||
}, 1, s.mt, nil, nil, nil, newIndexEngineVersionManager())
|
||||
|
||||
s.Run("segment is compacting", func() {
|
||||
s.mt.segments.segments[s.segID].isCompacting = true
|
||||
s.Run("drop task failed", func() {
|
||||
catalog := catalogmocks.NewDataCoordCatalog(s.T())
|
||||
catalog.EXPECT().DropStatsTask(mock.Anything, mock.Anything).Return(errors.New("mock error"))
|
||||
st.meta.statsTaskMeta.catalog = catalog
|
||||
st.CreateTaskOnWorker(1, session.NewMockCluster(s.T()))
|
||||
s.Equal(indexpb.JobState_JobStateInit, st.GetState())
|
||||
})
|
||||
|
||||
s.Run("drop task success", func() {
|
||||
catalog := catalogmocks.NewDataCoordCatalog(s.T())
|
||||
catalog.EXPECT().DropStatsTask(mock.Anything, mock.Anything).Return(nil)
|
||||
st.meta.statsTaskMeta.catalog = catalog
|
||||
st.CreateTaskOnWorker(1, session.NewMockCluster(s.T()))
|
||||
s.Equal(indexpb.JobState_JobStateNone, st.GetState())
|
||||
})
|
||||
s.mt.segments.segments[s.segID].isCompacting = false
|
||||
})
|
||||
|
||||
s.Run("segment in L0 compaction", func() {
|
||||
st.SetState(indexpb.JobState_JobStateInit, "")
|
||||
compactionInspector := NewMockCompactionInspector(s.T())
|
||||
compactionInspector.EXPECT().checkAndSetSegmentStating(mock.Anything, mock.Anything).Return(false)
|
||||
st.compactionInspector = compactionInspector
|
||||
st.CreateTaskOnWorker(1, session.NewMockCluster(s.T()))
|
||||
// No state change as we just log and exit
|
||||
s.Equal(indexpb.JobState_JobStateInit, st.GetState())
|
||||
})
|
||||
|
||||
compactionInspector := NewMockCompactionInspector(s.T())
|
||||
compactionInspector.EXPECT().checkAndSetSegmentStating(mock.Anything, mock.Anything).Return(true)
|
||||
st.compactionInspector = compactionInspector
|
||||
}, 1, s.mt, nil, nil, newIndexEngineVersionManager())
|
||||
|
||||
s.Run("segment not healthy", func() {
|
||||
// Set up a temporary nil segment return
|
||||
@ -335,10 +301,8 @@ func (s *statsTaskSuite) TestCreateTaskOnWorker() {
|
||||
|
||||
s.Run("drop task failed", func() {
|
||||
catalog := catalogmocks.NewDataCoordCatalog(s.T())
|
||||
catalog.EXPECT().SaveStatsTask(mock.Anything, mock.Anything).Return(nil)
|
||||
catalog.EXPECT().DropStatsTask(mock.Anything, mock.Anything).Return(errors.New("mock error"))
|
||||
st.meta.statsTaskMeta.catalog = catalog
|
||||
s.NoError(s.mt.statsTaskMeta.AddStatsTask(st.StatsTask))
|
||||
st.CreateTaskOnWorker(1, session.NewMockCluster(s.T()))
|
||||
s.Equal(indexpb.JobState_JobStateInit, st.GetState())
|
||||
})
|
||||
@ -373,7 +337,6 @@ func (s *statsTaskSuite) TestCreateTaskOnWorker() {
|
||||
s.mt.segments.segments[s.segID].isCompacting = false
|
||||
s.mt.segments.segments[s.segID].State = commonpb.SegmentState_Flushed
|
||||
s.mt.segments.segments[s.segID].NumOfRows = 1000
|
||||
compactionInspector.EXPECT().checkAndSetSegmentStating(mock.Anything, mock.Anything).Return(true)
|
||||
catalog := catalogmocks.NewDataCoordCatalog(s.T())
|
||||
catalog.EXPECT().SaveStatsTask(mock.Anything, mock.Anything).Return(errors.New("mock error"))
|
||||
st.meta.statsTaskMeta.catalog = catalog
|
||||
@ -386,7 +349,6 @@ func (s *statsTaskSuite) TestCreateTaskOnWorker() {
|
||||
st.SetState(indexpb.JobState_JobStateInit, "")
|
||||
s.mt.segments.segments[s.segID].isCompacting = false
|
||||
s.mt.segments.segments[s.segID].State = commonpb.SegmentState_Flushed
|
||||
compactionInspector.EXPECT().checkAndSetSegmentStating(mock.Anything, mock.Anything).Return(true)
|
||||
catalog := catalogmocks.NewDataCoordCatalog(s.T())
|
||||
catalog.EXPECT().SaveStatsTask(mock.Anything, mock.Anything).Return(nil)
|
||||
st.meta.statsTaskMeta.catalog = catalog
|
||||
@ -403,7 +365,6 @@ func (s *statsTaskSuite) TestCreateTaskOnWorker() {
|
||||
st.SetState(indexpb.JobState_JobStateInit, "")
|
||||
s.mt.segments.segments[s.segID].isCompacting = false
|
||||
s.mt.segments.segments[s.segID].State = commonpb.SegmentState_Flushed
|
||||
compactionInspector.EXPECT().checkAndSetSegmentStating(mock.Anything, mock.Anything).Return(true)
|
||||
catalog := catalogmocks.NewDataCoordCatalog(s.T())
|
||||
catalog.EXPECT().SaveStatsTask(mock.Anything, mock.Anything).Return(nil)
|
||||
st.meta.statsTaskMeta.catalog = catalog
|
||||
@ -432,7 +393,6 @@ func (s *statsTaskSuite) TestCreateTaskOnWorker() {
|
||||
st.SetState(indexpb.JobState_JobStateInit, "")
|
||||
s.mt.segments.segments[s.segID].isCompacting = false
|
||||
s.mt.segments.segments[s.segID].State = commonpb.SegmentState_Flushed
|
||||
compactionInspector.EXPECT().checkAndSetSegmentStating(mock.Anything, mock.Anything).Return(true)
|
||||
catalog := catalogmocks.NewDataCoordCatalog(s.T())
|
||||
catalog.EXPECT().SaveStatsTask(mock.Anything, mock.Anything).Return(nil).Once()
|
||||
catalog.EXPECT().SaveStatsTask(mock.Anything, mock.Anything).Return(errors.New("mock error")).Once()
|
||||
@ -449,7 +409,6 @@ func (s *statsTaskSuite) TestCreateTaskOnWorker() {
|
||||
|
||||
s.Run("success case", func() {
|
||||
s.mt.segments.segments[s.segID].isCompacting = false
|
||||
compactionInspector.EXPECT().checkAndSetSegmentStating(mock.Anything, mock.Anything).Return(true)
|
||||
catalog := catalogmocks.NewDataCoordCatalog(s.T())
|
||||
catalog.EXPECT().SaveStatsTask(mock.Anything, mock.Anything).Return(nil)
|
||||
st.meta.statsTaskMeta.catalog = catalog
|
||||
@ -466,10 +425,10 @@ func (s *statsTaskSuite) TestQueryTaskOnWorker() {
|
||||
st := newStatsTask(&indexpb.StatsTask{
|
||||
TaskID: s.taskID,
|
||||
SegmentID: s.segID,
|
||||
SubJobType: indexpb.StatsSubJob_Sort,
|
||||
SubJobType: indexpb.StatsSubJob_JsonKeyIndexJob,
|
||||
State: indexpb.JobState_JobStateInProgress,
|
||||
NodeID: 100,
|
||||
}, 1, s.mt, nil, nil, nil, newIndexEngineVersionManager())
|
||||
}, 1, s.mt, nil, nil, newIndexEngineVersionManager())
|
||||
|
||||
catalog := catalogmocks.NewDataCoordCatalog(s.T())
|
||||
catalog.EXPECT().AlterSegments(mock.Anything, mock.Anything, mock.Anything).Return(nil)
|
||||
@ -521,10 +480,10 @@ func (s *statsTaskSuite) TestDropTaskOnWorker() {
|
||||
st := newStatsTask(&indexpb.StatsTask{
|
||||
TaskID: s.taskID,
|
||||
SegmentID: s.segID,
|
||||
SubJobType: indexpb.StatsSubJob_Sort,
|
||||
SubJobType: indexpb.StatsSubJob_JsonKeyIndexJob,
|
||||
State: indexpb.JobState_JobStateInProgress,
|
||||
NodeID: 100,
|
||||
}, 1, s.mt, nil, nil, nil, newIndexEngineVersionManager())
|
||||
}, 1, s.mt, nil, nil, newIndexEngineVersionManager())
|
||||
|
||||
s.Run("drop task success", func() {
|
||||
cluster := session.NewMockCluster(s.T())
|
||||
@ -547,9 +506,9 @@ func (s *statsTaskSuite) TestSetJobInfo() {
|
||||
st := newStatsTask(&indexpb.StatsTask{
|
||||
TaskID: s.taskID,
|
||||
SegmentID: s.segID,
|
||||
SubJobType: indexpb.StatsSubJob_Sort,
|
||||
SubJobType: indexpb.StatsSubJob_JsonKeyIndexJob,
|
||||
State: indexpb.JobState_JobStateInProgress,
|
||||
}, 1, s.mt, nil, nil, nil, newIndexEngineVersionManager())
|
||||
}, 1, s.mt, nil, nil, newIndexEngineVersionManager())
|
||||
|
||||
result := &workerpb.StatsResult{
|
||||
TaskID: s.taskID,
|
||||
@ -584,8 +543,7 @@ func (s *statsTaskSuite) TestSetJobInfo() {
|
||||
s.mt.statsTaskMeta.catalog = catalog
|
||||
s.mt.catalog = catalog
|
||||
|
||||
// Test Sort job type
|
||||
st.SubJobType = indexpb.StatsSubJob_Sort
|
||||
st.SubJobType = indexpb.StatsSubJob_JsonKeyIndexJob
|
||||
err := st.SetJobInfo(context.Background(), result)
|
||||
s.NoError(err)
|
||||
|
||||
@ -609,9 +567,9 @@ func (s *statsTaskSuite) TestPrepareJobRequest() {
|
||||
st := newStatsTask(&indexpb.StatsTask{
|
||||
TaskID: s.taskID,
|
||||
SegmentID: s.segID,
|
||||
SubJobType: indexpb.StatsSubJob_Sort,
|
||||
SubJobType: indexpb.StatsSubJob_JsonKeyIndexJob,
|
||||
State: indexpb.JobState_JobStateInit,
|
||||
}, 1, s.mt, nil, nil, nil, newIndexEngineVersionManager())
|
||||
}, 1, s.mt, nil, nil, newIndexEngineVersionManager())
|
||||
|
||||
segment := &SegmentInfo{
|
||||
SegmentInfo: &datapb.SegmentInfo{
|
||||
@ -634,24 +592,6 @@ func (s *statsTaskSuite) TestPrepareJobRequest() {
|
||||
s.Contains(err.Error(), "failed to get collection info")
|
||||
})
|
||||
|
||||
s.Run("invalid collection TTL", func() {
|
||||
// Create handler with collection that has invalid TTL
|
||||
collection := &collectionInfo{
|
||||
Schema: newTestSchema(),
|
||||
Properties: map[string]string{
|
||||
common.CollectionTTLConfigKey: "invalid-ttl", // This will cause strconv.Atoi to fail
|
||||
},
|
||||
}
|
||||
|
||||
handler := NewNMockHandler(s.T())
|
||||
handler.EXPECT().GetCollection(mock.Anything, s.collID).Return(collection, nil)
|
||||
st.handler = handler
|
||||
|
||||
_, err := st.prepareJobRequest(context.Background(), segment)
|
||||
s.Error(err)
|
||||
s.Contains(err.Error(), "failed to get collection TTL")
|
||||
})
|
||||
|
||||
s.Run("allocation failure", func() {
|
||||
// Create a handler with valid collection
|
||||
collection := &collectionInfo{
|
||||
@ -716,6 +656,5 @@ func (s *statsTaskSuite) TestPrepareJobRequest() {
|
||||
s.Equal(startID, req.StartLogID)
|
||||
s.Equal(endID, req.EndLogID)
|
||||
s.Equal(int64(1000), req.NumRows)
|
||||
s.Equal(int64(3600000000000), req.CollectionTtl) // 1 hour in nanoseconds
|
||||
})
|
||||
}
|
||||
|
||||
@ -406,3 +406,7 @@ func calculateStatsTaskSlot(segmentSize int64) int64 {
|
||||
}
|
||||
return max(defaultSlots/8, 1)
|
||||
}
|
||||
|
||||
func enableSortCompaction() bool {
|
||||
return paramtable.Get().DataCoordCfg.EnableSortCompaction.GetAsBool() && paramtable.Get().DataCoordCfg.EnableCompaction.GetAsBool()
|
||||
}
|
||||
|
||||
@ -154,6 +154,7 @@ func NewClusteringCompactionTask(
|
||||
ctx context.Context,
|
||||
binlogIO io.BinlogIO,
|
||||
plan *datapb.CompactionPlan,
|
||||
compactionParams compaction.Params,
|
||||
) *clusteringCompactionTask {
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
return &clusteringCompactionTask{
|
||||
@ -166,6 +167,7 @@ func NewClusteringCompactionTask(
|
||||
clusterBuffers: make([]*ClusterBuffer, 0),
|
||||
flushCount: atomic.NewInt64(0),
|
||||
writtenRowNum: atomic.NewInt64(0),
|
||||
compactionParams: compactionParams,
|
||||
}
|
||||
}
|
||||
|
||||
@ -198,11 +200,6 @@ func (t *clusteringCompactionTask) init() error {
|
||||
if t.plan.GetType() != datapb.CompactionType_ClusteringCompaction {
|
||||
return merr.WrapErrIllegalCompactionPlan("illegal compaction type")
|
||||
}
|
||||
var err error
|
||||
t.compactionParams, err = compaction.ParseParamsFromJSON(t.plan.GetJsonParams())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if len(t.plan.GetSegmentBinlogs()) == 0 {
|
||||
return merr.WrapErrIllegalCompactionPlan("empty segment binlogs")
|
||||
@ -264,7 +261,7 @@ func (t *clusteringCompactionTask) Compact() (*datapb.CompactionPlanResult, erro
|
||||
defer t.cleanUp(ctx)
|
||||
|
||||
// 1, decompose binlogs as preparation for later mapping
|
||||
if err := binlog.DecompressCompactionBinlogs(t.plan.SegmentBinlogs); err != nil {
|
||||
if err := binlog.DecompressCompactionBinlogsWithRootPath(t.compactionParams.StorageConfig.GetRootPath(), t.plan.SegmentBinlogs); err != nil {
|
||||
log.Warn("compact wrong, fail to decompress compaction binlogs", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
@ -587,8 +584,7 @@ func (t *clusteringCompactionTask) mappingSegment(
|
||||
return merr.WrapErrIllegalCompactionPlan()
|
||||
}
|
||||
|
||||
// TODO bucketName shall be passed via StorageConfig like index/stats task
|
||||
bucketName := paramtable.Get().ServiceParam.MinioCfg.BucketName.GetValue()
|
||||
bucketName := t.compactionParams.StorageConfig.GetBucketName()
|
||||
|
||||
rr, err := storage.NewBinlogRecordReader(ctx,
|
||||
segment.GetFieldBinlogs(),
|
||||
@ -597,7 +593,7 @@ func (t *clusteringCompactionTask) mappingSegment(
|
||||
return t.binlogIO.Download(ctx, paths)
|
||||
}),
|
||||
storage.WithVersion(segment.StorageVersion), storage.WithBufferSize(t.memoryBufferSize),
|
||||
storage.WithBucketName(bucketName),
|
||||
storage.WithBucketName(bucketName), storage.WithStorageConfig(t.compactionParams.StorageConfig),
|
||||
)
|
||||
if err != nil {
|
||||
log.Warn("new binlog record reader wrong", zap.Error(err))
|
||||
@ -865,8 +861,7 @@ func (t *clusteringCompactionTask) scalarAnalyzeSegment(
|
||||
|
||||
expiredFilter := compaction.NewEntityFilter(nil, t.plan.GetCollectionTtl(), t.currentTime)
|
||||
|
||||
// TODO bucketName shall be passed via StorageConfig like index/stats task
|
||||
bucketName := paramtable.Get().ServiceParam.MinioCfg.BucketName.GetValue()
|
||||
bucketName := t.compactionParams.StorageConfig.GetBucketName()
|
||||
|
||||
rr, err := storage.NewBinlogRecordReader(ctx,
|
||||
segment.GetFieldBinlogs(),
|
||||
@ -876,7 +871,7 @@ func (t *clusteringCompactionTask) scalarAnalyzeSegment(
|
||||
}),
|
||||
storage.WithVersion(segment.StorageVersion),
|
||||
storage.WithBufferSize(t.memoryBufferSize),
|
||||
storage.WithBucketName(bucketName),
|
||||
storage.WithBucketName(bucketName), storage.WithStorageConfig(t.compactionParams.StorageConfig),
|
||||
)
|
||||
if err != nil {
|
||||
log.Warn("new binlog record reader wrong", zap.Error(err))
|
||||
|
||||
@ -19,6 +19,7 @@ package compactor
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/milvus-io/milvus/internal/compaction"
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
@ -53,7 +54,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) SetupTest() {
|
||||
paramtable.Get().Save("common.storageType", "local")
|
||||
paramtable.Get().Save("common.storage.enableV2", "true")
|
||||
initcore.InitStorageV2FileSystem(paramtable.Get())
|
||||
refreshPlanParams(s.plan)
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
}
|
||||
|
||||
func (s *ClusteringCompactionTaskStorageV2Suite) TearDownTest() {
|
||||
@ -144,7 +145,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) TestScalarCompactionNormal_V2To
|
||||
|
||||
func (s *ClusteringCompactionTaskStorageV2Suite) TestScalarCompactionNormal_V2ToV1Format() {
|
||||
paramtable.Get().Save("common.storage.enableV2", "false")
|
||||
refreshPlanParams(s.plan)
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
|
||||
var segmentID int64 = 1001
|
||||
|
||||
@ -222,7 +223,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) TestCompactionWithBM25Function(
|
||||
// writer will automatically flush after 1024 rows.
|
||||
paramtable.Get().Save(paramtable.Get().DataNodeCfg.BinLogMaxSize.Key, "45000")
|
||||
defer paramtable.Get().Reset(paramtable.Get().DataNodeCfg.BinLogMaxSize.Key)
|
||||
refreshPlanParams(s.plan)
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
s.prepareCompactionWithBM25FunctionTask()
|
||||
compactionResult, err := s.task.Compact()
|
||||
s.Require().NoError(err)
|
||||
@ -245,7 +246,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) TestScalarCompactionNormalByMem
|
||||
defer paramtable.Get().Reset(paramtable.Get().DataNodeCfg.BinLogMaxSize.Key)
|
||||
paramtable.Get().Save(paramtable.Get().DataCoordCfg.ClusteringCompactionPreferSegmentSizeRatio.Key, "1")
|
||||
defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.ClusteringCompactionPreferSegmentSizeRatio.Key)
|
||||
refreshPlanParams(s.plan)
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
|
||||
compactionResult, err := s.task.Compact()
|
||||
s.Require().NoError(err)
|
||||
|
||||
@ -42,14 +42,6 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
func refreshPlanParams(plan *datapb.CompactionPlan) {
|
||||
params, err := compaction.GenerateJSONParams()
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
plan.JsonParams = params
|
||||
}
|
||||
|
||||
func TestClusteringCompactionTaskSuite(t *testing.T) {
|
||||
suite.Run(t, new(ClusteringCompactionTaskSuite))
|
||||
}
|
||||
@ -87,7 +79,7 @@ func (s *ClusteringCompactionTaskSuite) setupTest() {
|
||||
return end, nil
|
||||
}).Maybe()
|
||||
|
||||
s.task = NewClusteringCompactionTask(context.Background(), s.mockBinlogIO, nil)
|
||||
s.task = NewClusteringCompactionTask(context.Background(), s.mockBinlogIO, nil, compaction.GenParams())
|
||||
|
||||
paramtable.Get().Save(paramtable.Get().CommonCfg.EntityExpirationTTL.Key, "0")
|
||||
params, err := compaction.GenerateJSONParams()
|
||||
@ -252,7 +244,7 @@ func (s *ClusteringCompactionTaskSuite) TestScalarCompactionNormal() {
|
||||
// writer will automatically flush after 1024 rows.
|
||||
paramtable.Get().Save(paramtable.Get().DataNodeCfg.BinLogMaxSize.Key, "60000")
|
||||
defer paramtable.Get().Reset(paramtable.Get().DataNodeCfg.BinLogMaxSize.Key)
|
||||
refreshPlanParams(s.plan)
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
|
||||
compactionResult, err := s.task.Compact()
|
||||
s.Require().NoError(err)
|
||||
@ -346,7 +338,7 @@ func (s *ClusteringCompactionTaskSuite) TestScalarCompactionNormalByMemoryLimit(
|
||||
defer paramtable.Get().Reset(paramtable.Get().DataNodeCfg.BinLogMaxSize.Key)
|
||||
paramtable.Get().Save(paramtable.Get().DataCoordCfg.ClusteringCompactionPreferSegmentSizeRatio.Key, "1")
|
||||
defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.ClusteringCompactionPreferSegmentSizeRatio.Key)
|
||||
refreshPlanParams(s.plan)
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
|
||||
compactionResult, err := s.task.Compact()
|
||||
s.Require().NoError(err)
|
||||
@ -427,7 +419,7 @@ func (s *ClusteringCompactionTaskSuite) TestCompactionWithBM25Function() {
|
||||
// writer will automatically flush after 1024 rows.
|
||||
paramtable.Get().Save(paramtable.Get().DataNodeCfg.BinLogMaxSize.Key, "50000")
|
||||
defer paramtable.Get().Reset(paramtable.Get().DataNodeCfg.BinLogMaxSize.Key)
|
||||
refreshPlanParams(s.plan)
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
s.prepareCompactionWithBM25FunctionTask()
|
||||
|
||||
compactionResult, err := s.task.Compact()
|
||||
|
||||
@ -69,6 +69,7 @@ func NewLevelZeroCompactionTask(
|
||||
binlogIO io.BinlogIO,
|
||||
cm storage.ChunkManager,
|
||||
plan *datapb.CompactionPlan,
|
||||
compactionParams compaction.Params,
|
||||
) *LevelZeroCompactionTask {
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
alloc := allocator.NewLocalAllocator(plan.GetPreAllocatedLogIDs().GetBegin(), plan.GetPreAllocatedLogIDs().GetEnd())
|
||||
@ -82,6 +83,7 @@ func NewLevelZeroCompactionTask(
|
||||
plan: plan,
|
||||
tr: timerecord.NewTimeRecorder("levelzero compaction"),
|
||||
done: make(chan struct{}, 1),
|
||||
compactionParams: compactionParams,
|
||||
}
|
||||
}
|
||||
|
||||
@ -123,11 +125,6 @@ func (t *LevelZeroCompactionTask) Compact() (*datapb.CompactionPlanResult, error
|
||||
}
|
||||
|
||||
var err error
|
||||
t.compactionParams, err = compaction.ParseParamsFromJSON(t.plan.GetJsonParams())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
l0Segments := lo.Filter(t.plan.GetSegmentBinlogs(), func(s *datapb.CompactionSegmentBinlogs, _ int) bool {
|
||||
return s.Level == datapb.SegmentLevel_L0
|
||||
})
|
||||
@ -139,7 +136,7 @@ func (t *LevelZeroCompactionTask) Compact() (*datapb.CompactionPlanResult, error
|
||||
log.Warn("compact wrong, not target sealed segments")
|
||||
return nil, errors.New("illegal compaction plan with empty target segments")
|
||||
}
|
||||
err = binlog.DecompressCompactionBinlogs(l0Segments)
|
||||
err = binlog.DecompressCompactionBinlogsWithRootPath(t.compactionParams.StorageConfig.GetRootPath(), l0Segments)
|
||||
if err != nil {
|
||||
log.Warn("DecompressCompactionBinlogs failed", zap.Error(err))
|
||||
return nil, err
|
||||
|
||||
@ -65,7 +65,7 @@ func (s *LevelZeroCompactionTaskSuite) SetupTest() {
|
||||
End: 2000,
|
||||
},
|
||||
}
|
||||
s.task = NewLevelZeroCompactionTask(context.Background(), s.mockBinlogIO, nil, plan)
|
||||
s.task = NewLevelZeroCompactionTask(context.Background(), s.mockBinlogIO, nil, plan, compaction.GenParams())
|
||||
var err error
|
||||
s.task.compactionParams, err = compaction.ParseParamsFromJSON("")
|
||||
s.Require().NoError(err)
|
||||
|
||||
@ -18,7 +18,6 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/timerecord"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
@ -31,6 +30,7 @@ func mergeSortMultipleSegments(ctx context.Context,
|
||||
tr *timerecord.TimeRecorder,
|
||||
currentTime time.Time,
|
||||
collectionTtl int64,
|
||||
compactionParams compaction.Params,
|
||||
) ([]*datapb.CompactionSegment, error) {
|
||||
_ = tr.RecordSpan()
|
||||
|
||||
@ -42,10 +42,6 @@ func mergeSortMultipleSegments(ctx context.Context,
|
||||
segIDAlloc := allocator.NewLocalAllocator(plan.GetPreAllocatedSegmentIDs().GetBegin(), plan.GetPreAllocatedSegmentIDs().GetEnd())
|
||||
logIDAlloc := allocator.NewLocalAllocator(plan.GetPreAllocatedLogIDs().GetBegin(), plan.GetPreAllocatedLogIDs().GetEnd())
|
||||
compAlloc := NewCompactionAllocator(segIDAlloc, logIDAlloc)
|
||||
compactionParams, err := compaction.ParseParamsFromJSON(plan.GetJsonParams())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
writer, err := NewMultiSegmentWriter(ctx, binlogIO, compAlloc, plan.GetMaxSize(), plan.GetSchema(), compactionParams, maxRows, partitionID, collectionID, plan.GetChannel(), 4096,
|
||||
storage.WithStorageConfig(compactionParams.StorageConfig))
|
||||
if err != nil {
|
||||
@ -58,8 +54,7 @@ func mergeSortMultipleSegments(ctx context.Context,
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// TODO bucketName shall be passed via StorageConfig like index/stats task
|
||||
bucketName := paramtable.Get().ServiceParam.MinioCfg.BucketName.GetValue()
|
||||
bucketName := compactionParams.StorageConfig.GetBucketName()
|
||||
|
||||
segmentReaders := make([]storage.RecordReader, len(binlogs))
|
||||
segmentFilters := make([]compaction.EntityFilter, len(binlogs))
|
||||
|
||||
@ -73,6 +73,7 @@ func NewMixCompactionTask(
|
||||
ctx context.Context,
|
||||
binlogIO io.BinlogIO,
|
||||
plan *datapb.CompactionPlan,
|
||||
compactionParams compaction.Params,
|
||||
) *mixCompactionTask {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
return &mixCompactionTask{
|
||||
@ -83,6 +84,7 @@ func NewMixCompactionTask(
|
||||
tr: timerecord.NewTimeRecorder("mergeSplit compaction"),
|
||||
currentTime: time.Now(),
|
||||
done: make(chan struct{}, 1),
|
||||
compactionParams: compactionParams,
|
||||
}
|
||||
}
|
||||
|
||||
@ -92,12 +94,6 @@ func (t *mixCompactionTask) preCompact() error {
|
||||
return t.ctx.Err()
|
||||
}
|
||||
|
||||
var err error
|
||||
t.compactionParams, err = compaction.ParseParamsFromJSON(t.plan.GetJsonParams())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if len(t.plan.GetSegmentBinlogs()) < 1 {
|
||||
return errors.Newf("compaction plan is illegal, there's no segments in compaction plan, planID = %d", t.GetPlanID())
|
||||
}
|
||||
@ -217,8 +213,7 @@ func (t *mixCompactionTask) writeSegment(ctx context.Context,
|
||||
}
|
||||
entityFilter := compaction.NewEntityFilter(delta, t.plan.GetCollectionTtl(), t.currentTime)
|
||||
|
||||
// TODO bucketName shall be passed via StorageConfig like index/stats task
|
||||
bucketName := paramtable.Get().ServiceParam.MinioCfg.BucketName.GetValue()
|
||||
bucketName := t.compactionParams.StorageConfig.GetBucketName()
|
||||
|
||||
reader, err := storage.NewBinlogRecordReader(ctx,
|
||||
seg.GetFieldBinlogs(),
|
||||
@ -335,7 +330,7 @@ func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
|
||||
|
||||
log.Info("compact start")
|
||||
// Decompress compaction binlogs first
|
||||
if err := binlog.DecompressCompactionBinlogs(t.plan.SegmentBinlogs); err != nil {
|
||||
if err := binlog.DecompressCompactionBinlogsWithRootPath(t.compactionParams.StorageConfig.GetRootPath(), t.plan.SegmentBinlogs); err != nil {
|
||||
log.Warn("compact wrong, fail to decompress compaction binlogs", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
@ -372,7 +367,7 @@ func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
|
||||
// TODO: the implementation of mergeSortMultipleSegments is not correct, also see issue: https://github.com/milvus-io/milvus/issues/43034
|
||||
log.Info("compact by merge sort")
|
||||
res, err = mergeSortMultipleSegments(ctxTimeout, t.plan, t.collectionID, t.partitionID, t.maxRows, t.binlogIO,
|
||||
t.plan.GetSegmentBinlogs(), t.tr, t.currentTime, t.plan.GetCollectionTtl())
|
||||
t.plan.GetSegmentBinlogs(), t.tr, t.currentTime, t.plan.GetCollectionTtl(), t.compactionParams)
|
||||
if err != nil {
|
||||
log.Warn("compact wrong, fail to merge sort segments", zap.Error(err))
|
||||
return nil, err
|
||||
@ -385,7 +380,7 @@ func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
|
||||
}
|
||||
}
|
||||
|
||||
log.Info("compact done", zap.Duration("compact elapse", time.Since(compactStart)))
|
||||
log.Info("compact done", zap.Duration("compact elapse", time.Since(compactStart)), zap.Any("res", res))
|
||||
|
||||
metrics.DataNodeCompactionLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), t.plan.GetType().String()).Observe(float64(t.tr.ElapseSpan().Milliseconds()))
|
||||
metrics.DataNodeCompactionLatencyInQueue.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(durInQueue.Milliseconds()))
|
||||
|
||||
@ -19,6 +19,7 @@ package compactor
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/milvus-io/milvus/internal/compaction"
|
||||
"math"
|
||||
"os"
|
||||
"testing"
|
||||
@ -57,7 +58,7 @@ func (s *MixCompactionTaskStorageV2Suite) SetupTest() {
|
||||
paramtable.Get().Save("common.storageType", "local")
|
||||
paramtable.Get().Save("common.storage.enableV2", "true")
|
||||
initcore.InitStorageV2FileSystem(paramtable.Get())
|
||||
refreshPlanParams(s.task.plan)
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
}
|
||||
|
||||
func (s *MixCompactionTaskStorageV2Suite) TearDownTest() {
|
||||
@ -178,7 +179,7 @@ func (s *MixCompactionTaskStorageV2Suite) TestCompactDupPK_V2ToV2Format() {
|
||||
|
||||
func (s *MixCompactionTaskStorageV2Suite) TestCompactDupPK_V2ToV1Format() {
|
||||
paramtable.Get().Save("common.storage.enableV2", "false")
|
||||
refreshPlanParams(s.task.plan)
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil)
|
||||
alloc := allocator.NewLocalAllocator(7777777, math.MaxInt64)
|
||||
|
||||
@ -243,7 +244,7 @@ func (s *MixCompactionTaskStorageV2Suite) TestCompactSortedSegment() {
|
||||
s.prepareCompactSortedSegment()
|
||||
paramtable.Get().Save("dataNode.compaction.useMergeSort", "true")
|
||||
defer paramtable.Get().Reset("dataNode.compaction.useMergeSort")
|
||||
refreshPlanParams(s.task.plan)
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
|
||||
result, err := s.task.Compact()
|
||||
s.NoError(err)
|
||||
@ -268,7 +269,8 @@ func (s *MixCompactionTaskStorageV2Suite) TestCompactSortedSegmentLackBinlog() {
|
||||
s.prepareCompactSortedSegmentLackBinlog()
|
||||
paramtable.Get().Save("dataNode.compaction.useMergeSort", "true")
|
||||
defer paramtable.Get().Reset("dataNode.compaction.useMergeSort")
|
||||
refreshPlanParams(s.task.plan)
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
fmt.Println(s.task.compactionParams)
|
||||
|
||||
result, err := s.task.Compact()
|
||||
s.NoError(err)
|
||||
|
||||
@ -93,7 +93,7 @@ func (s *MixCompactionTaskSuite) setupTest() {
|
||||
JsonParams: params,
|
||||
}
|
||||
|
||||
s.task = NewMixCompactionTask(context.Background(), s.mockBinlogIO, plan)
|
||||
s.task = NewMixCompactionTask(context.Background(), s.mockBinlogIO, plan, compaction.GenParams())
|
||||
}
|
||||
|
||||
func (s *MixCompactionTaskSuite) SetupTest() {
|
||||
@ -125,7 +125,7 @@ func (s *MixCompactionTaskSuite) SetupBM25() {
|
||||
JsonParams: params,
|
||||
}
|
||||
|
||||
s.task = NewMixCompactionTask(context.Background(), s.mockBinlogIO, plan)
|
||||
s.task = NewMixCompactionTask(context.Background(), s.mockBinlogIO, plan, compaction.GenParams())
|
||||
}
|
||||
|
||||
func (s *MixCompactionTaskSuite) SetupSubTest() {
|
||||
@ -335,7 +335,7 @@ func (s *MixCompactionTaskSuite) TestCompactSortedSegment() {
|
||||
s.prepareCompactSortedSegment()
|
||||
paramtable.Get().Save("dataNode.compaction.useMergeSort", "true")
|
||||
defer paramtable.Get().Reset("dataNode.compaction.useMergeSort")
|
||||
refreshPlanParams(s.task.plan)
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
|
||||
result, err := s.task.Compact()
|
||||
s.NoError(err)
|
||||
@ -415,7 +415,7 @@ func (s *MixCompactionTaskSuite) TestCompactSortedSegmentLackBinlog() {
|
||||
s.prepareCompactSortedSegmentLackBinlog()
|
||||
paramtable.Get().Save("dataNode.compaction.useMergeSort", "true")
|
||||
defer paramtable.Get().Reset("dataNode.compaction.useMergeSort")
|
||||
refreshPlanParams(s.task.plan)
|
||||
s.task.compactionParams = compaction.GenParams()
|
||||
|
||||
result, err := s.task.Compact()
|
||||
s.NoError(err)
|
||||
@ -818,11 +818,13 @@ func (s *MixCompactionTaskSuite) TestCompactFail() {
|
||||
})
|
||||
}
|
||||
|
||||
func getRow(magic int64) map[int64]interface{} {
|
||||
ts := tsoutil.ComposeTSByTime(getMilvusBirthday(), 0)
|
||||
func getRow(magic int64, ts int64) map[int64]interface{} {
|
||||
if ts == 0 {
|
||||
ts = int64(tsoutil.ComposeTSByTime(getMilvusBirthday(), 0))
|
||||
}
|
||||
return map[int64]interface{}{
|
||||
common.RowIDField: magic,
|
||||
common.TimeStampField: int64(ts), // should be int64 here
|
||||
common.TimeStampField: ts, // should be int64 here
|
||||
BoolField: true,
|
||||
Int8Field: int8(magic),
|
||||
Int16Field: int16(magic),
|
||||
@ -863,7 +865,7 @@ func (s *MixCompactionTaskSuite) initMultiRowsSegBuffer(magic, numRows, step int
|
||||
v := storage.Value{
|
||||
PK: storage.NewInt64PrimaryKey(magic + i*step),
|
||||
Timestamp: int64(tsoutil.ComposeTSByTime(getMilvusBirthday(), 0)),
|
||||
Value: getRow(magic + i*step),
|
||||
Value: getRow(magic+i*step, 0),
|
||||
}
|
||||
err = segWriter.Write(&v)
|
||||
s.Require().NoError(err)
|
||||
@ -898,7 +900,7 @@ func (s *MixCompactionTaskSuite) initSegBuffer(size int, seed int64) {
|
||||
v := storage.Value{
|
||||
PK: storage.NewInt64PrimaryKey(seed),
|
||||
Timestamp: int64(tsoutil.ComposeTSByTime(getMilvusBirthday(), 0)),
|
||||
Value: getRow(seed),
|
||||
Value: getRow(seed, int64(tsoutil.ComposeTSByTime(getMilvusBirthday(), 0))),
|
||||
}
|
||||
err = segWriter.Write(&v)
|
||||
s.Require().NoError(err)
|
||||
|
||||
@ -32,7 +32,6 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/compaction"
|
||||
"github.com/milvus-io/milvus/internal/flushcommon/io"
|
||||
"github.com/milvus-io/milvus/internal/flushcommon/writebuffer"
|
||||
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/v2/common"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
@ -68,6 +67,7 @@ type MultiSegmentWriter struct {
|
||||
// DONOT leave it empty of all segments are deleted, just return a segment with zero meta for datacoord
|
||||
|
||||
storageVersion int64
|
||||
params compaction.Params
|
||||
rwOption []storage.RwOption
|
||||
}
|
||||
|
||||
@ -91,12 +91,6 @@ func NewMultiSegmentWriter(ctx context.Context, binlogIO io.BinlogIO, allocator
|
||||
schema *schemapb.CollectionSchema, params compaction.Params,
|
||||
maxRows int64, partitionID, collectionID int64, channel string, batchSize int, rwOption ...storage.RwOption,
|
||||
) (*MultiSegmentWriter, error) {
|
||||
storageVersion := storage.StorageV1
|
||||
|
||||
if params.EnableStorageV2 {
|
||||
storageVersion = storage.StorageV2
|
||||
}
|
||||
|
||||
rwOpts := rwOption
|
||||
if len(rwOption) == 0 {
|
||||
rwOpts = make([]storage.RwOption, 0)
|
||||
@ -114,7 +108,8 @@ func NewMultiSegmentWriter(ctx context.Context, binlogIO io.BinlogIO, allocator
|
||||
batchSize: batchSize,
|
||||
binLogMaxSize: params.BinLogMaxSize,
|
||||
res: make([]*datapb.CompactionSegment, 0),
|
||||
storageVersion: storageVersion,
|
||||
storageVersion: params.StorageVersion,
|
||||
params: params,
|
||||
rwOption: rwOpts,
|
||||
}, nil
|
||||
}
|
||||
@ -162,7 +157,7 @@ func (w *MultiSegmentWriter) rotateWriter() error {
|
||||
w.currentSegmentID = newSegmentID
|
||||
|
||||
chunkSize := w.binLogMaxSize
|
||||
rootPath := binlog.GetRootPath()
|
||||
rootPath := w.params.StorageConfig.GetRootPath()
|
||||
|
||||
w.rwOption = append(w.rwOption,
|
||||
storage.WithUploader(func(ctx context.Context, kvs map[string][]byte) error {
|
||||
@ -171,7 +166,7 @@ func (w *MultiSegmentWriter) rotateWriter() error {
|
||||
storage.WithVersion(w.storageVersion),
|
||||
)
|
||||
// TODO bucketName shall be passed via StorageConfig like index/stats task
|
||||
bucketName := paramtable.Get().ServiceParam.MinioCfg.BucketName.GetValue()
|
||||
bucketName := w.params.StorageConfig.GetBucketName()
|
||||
rw, err := storage.NewBinlogRecordWriter(w.ctx, w.collectionID, w.partitionID, newSegmentID,
|
||||
w.schema, w.allocator.logIDAlloc, chunkSize, bucketName, rootPath, w.maxRows, w.rwOption...,
|
||||
)
|
||||
|
||||
462
internal/datanode/compactor/sort_compaction.go
Normal file
462
internal/datanode/compactor/sort_compaction.go
Normal file
@ -0,0 +1,462 @@
|
||||
// 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.
|
||||
|
||||
package compactor
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"runtime/debug"
|
||||
"time"
|
||||
|
||||
"github.com/apache/arrow/go/v17/arrow/array"
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/samber/lo"
|
||||
"go.opentelemetry.io/otel"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/compaction"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/flushcommon/io"
|
||||
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/indexcgowrapper"
|
||||
"github.com/milvus-io/milvus/pkg/v2/common"
|
||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/metautil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/timerecord"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
|
||||
type sortCompactionTask struct {
|
||||
binlogIO io.BinlogIO
|
||||
currentTime time.Time
|
||||
|
||||
plan *datapb.CompactionPlan
|
||||
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
|
||||
collectionID int64
|
||||
partitionID int64
|
||||
segmentID int64
|
||||
deltaLogs []string
|
||||
insertLogs []*datapb.FieldBinlog
|
||||
storageVersion int64
|
||||
segmentStorageVersion int64
|
||||
|
||||
done chan struct{}
|
||||
tr *timerecord.TimeRecorder
|
||||
|
||||
compactionParams compaction.Params
|
||||
}
|
||||
|
||||
var _ Compactor = (*sortCompactionTask)(nil)
|
||||
|
||||
func NewSortCompactionTask(
|
||||
ctx context.Context,
|
||||
binlogIO io.BinlogIO,
|
||||
plan *datapb.CompactionPlan,
|
||||
compactionParams compaction.Params,
|
||||
) *sortCompactionTask {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
return &sortCompactionTask{
|
||||
ctx: ctx1,
|
||||
cancel: cancel,
|
||||
binlogIO: binlogIO,
|
||||
plan: plan,
|
||||
tr: timerecord.NewTimeRecorder("sort compaction"),
|
||||
currentTime: time.Now(),
|
||||
done: make(chan struct{}, 1),
|
||||
compactionParams: compactionParams,
|
||||
}
|
||||
}
|
||||
|
||||
// preCompact examines whether it's a valid compaction plan, and initializes the collectionID and partitionID
|
||||
func (t *sortCompactionTask) preCompact() error {
|
||||
if ok := funcutil.CheckCtxValid(t.ctx); !ok {
|
||||
return t.ctx.Err()
|
||||
}
|
||||
|
||||
if len(t.plan.GetSegmentBinlogs()) != 1 {
|
||||
return errors.Newf("sort compaction should handle exactly one segment, but got %d segments, planID = %d",
|
||||
len(t.plan.GetSegmentBinlogs()), t.GetPlanID())
|
||||
}
|
||||
|
||||
segment := t.plan.GetSegmentBinlogs()[0]
|
||||
t.collectionID = segment.GetCollectionID()
|
||||
t.partitionID = segment.GetPartitionID()
|
||||
t.segmentID = segment.GetSegmentID()
|
||||
|
||||
if err := binlog.DecompressBinLogWithRootPath(t.compactionParams.StorageConfig.GetRootPath(),
|
||||
storage.InsertBinlog, t.collectionID, t.partitionID,
|
||||
t.segmentID, segment.GetFieldBinlogs()); err != nil {
|
||||
log.Ctx(t.ctx).Warn("Decompress insert binlog error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
if err := binlog.DecompressBinLogWithRootPath(t.compactionParams.StorageConfig.GetRootPath(),
|
||||
storage.DeleteBinlog, t.collectionID, t.partitionID,
|
||||
t.segmentID, segment.GetDeltalogs()); err != nil {
|
||||
log.Ctx(t.ctx).Warn("Decompress delta binlog error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
for _, d := range segment.GetDeltalogs() {
|
||||
for _, l := range d.GetBinlogs() {
|
||||
t.deltaLogs = append(t.deltaLogs, l.GetLogPath())
|
||||
}
|
||||
}
|
||||
|
||||
t.insertLogs = segment.GetFieldBinlogs()
|
||||
t.storageVersion = t.compactionParams.StorageVersion
|
||||
t.segmentStorageVersion = segment.GetStorageVersion()
|
||||
|
||||
log.Ctx(t.ctx).Info("preCompaction analyze",
|
||||
zap.Int64("planID", t.GetPlanID()),
|
||||
zap.Int64("collectionID", t.collectionID),
|
||||
zap.Int64("partitionID", t.partitionID),
|
||||
zap.Int64("segmentID", t.segmentID),
|
||||
zap.Int64("storageVersion", t.storageVersion),
|
||||
zap.Any("compactionParams", t.compactionParams),
|
||||
)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (t *sortCompactionTask) sortSegment(ctx context.Context) (*datapb.CompactionPlanResult, error) {
|
||||
log := log.Ctx(ctx).With(
|
||||
zap.Int64("planID", t.plan.GetPlanID()),
|
||||
zap.Int64("collectionID", t.collectionID),
|
||||
zap.Int64("partitionID", t.partitionID),
|
||||
zap.Int64("segmentID", t.segmentID),
|
||||
)
|
||||
sortStartTime := time.Now()
|
||||
numRows := t.plan.GetTotalRows()
|
||||
pkField, err := typeutil.GetPrimaryFieldSchema(t.plan.GetSchema())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
alloc := allocator.NewLocalAllocator(t.plan.GetPreAllocatedLogIDs().GetBegin(), t.plan.GetPreAllocatedLogIDs().GetEnd())
|
||||
targetSegmentID := t.plan.GetPreAllocatedSegmentIDs().GetBegin()
|
||||
|
||||
srw, err := storage.NewBinlogRecordWriter(ctx,
|
||||
t.collectionID,
|
||||
t.partitionID,
|
||||
targetSegmentID,
|
||||
t.plan.GetSchema(),
|
||||
alloc,
|
||||
t.compactionParams.BinLogMaxSize,
|
||||
t.compactionParams.StorageConfig.GetBucketName(),
|
||||
t.compactionParams.StorageConfig.GetRootPath(),
|
||||
numRows,
|
||||
storage.WithUploader(func(ctx context.Context, kvs map[string][]byte) error {
|
||||
return t.binlogIO.Upload(ctx, kvs)
|
||||
}),
|
||||
storage.WithVersion(t.storageVersion),
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig),
|
||||
)
|
||||
if err != nil {
|
||||
log.Warn("sort segment wrong, unable to init segment writer",
|
||||
zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
deletePKs, err := compaction.ComposeDeleteFromDeltalogs(ctx, t.binlogIO, t.deltaLogs)
|
||||
if err != nil {
|
||||
log.Warn("load deletePKs failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
entityFilter := compaction.NewEntityFilter(deletePKs, t.plan.GetCollectionTtl(), t.currentTime)
|
||||
var predicate func(r storage.Record, ri, i int) bool
|
||||
switch pkField.DataType {
|
||||
case schemapb.DataType_Int64:
|
||||
predicate = func(r storage.Record, ri, i int) bool {
|
||||
pk := r.Column(pkField.FieldID).(*array.Int64).Value(i)
|
||||
ts := r.Column(common.TimeStampField).(*array.Int64).Value(i)
|
||||
return !entityFilter.Filtered(pk, uint64(ts))
|
||||
}
|
||||
case schemapb.DataType_VarChar:
|
||||
predicate = func(r storage.Record, ri, i int) bool {
|
||||
pk := r.Column(pkField.FieldID).(*array.String).Value(i)
|
||||
ts := r.Column(common.TimeStampField).(*array.Int64).Value(i)
|
||||
return !entityFilter.Filtered(pk, uint64(ts))
|
||||
}
|
||||
default:
|
||||
log.Warn("sort task only support int64 and varchar pk field")
|
||||
}
|
||||
|
||||
rr, err := storage.NewBinlogRecordReader(ctx, t.insertLogs, t.plan.Schema,
|
||||
storage.WithVersion(t.segmentStorageVersion),
|
||||
storage.WithDownloader(t.binlogIO.Download),
|
||||
storage.WithBucketName(t.compactionParams.StorageConfig.BucketName),
|
||||
storage.WithStorageConfig(t.compactionParams.StorageConfig),
|
||||
)
|
||||
if err != nil {
|
||||
log.Warn("error creating insert binlog reader", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
defer rr.Close()
|
||||
rrs := []storage.RecordReader{rr}
|
||||
numValidRows, err := storage.Sort(t.compactionParams.BinLogMaxSize, t.plan.GetSchema(), rrs, srw, predicate)
|
||||
if err != nil {
|
||||
log.Warn("sort failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
if err := srw.Close(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
binlogs, stats, bm25stats := srw.GetLogs()
|
||||
insertLogs := lo.Values(binlogs)
|
||||
if err := binlog.CompressFieldBinlogs(insertLogs); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
statsLogs := []*datapb.FieldBinlog{stats}
|
||||
if err := binlog.CompressFieldBinlogs(statsLogs); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
bm25StatsLogs := lo.Values(bm25stats)
|
||||
if err := binlog.CompressFieldBinlogs(bm25StatsLogs); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
debug.FreeOSMemory()
|
||||
log.Info("sort segment end",
|
||||
zap.Int64("target segmentID", targetSegmentID),
|
||||
zap.Int64("old rows", numRows),
|
||||
zap.Int("valid rows", numValidRows),
|
||||
zap.Int("deleted rows", entityFilter.GetDeletedCount()),
|
||||
zap.Int("expired rows", entityFilter.GetExpiredCount()),
|
||||
zap.Duration("total elapse", time.Since(sortStartTime)))
|
||||
|
||||
res := []*datapb.CompactionSegment{
|
||||
{
|
||||
PlanID: t.GetPlanID(),
|
||||
SegmentID: targetSegmentID,
|
||||
NumOfRows: int64(numValidRows),
|
||||
InsertLogs: insertLogs,
|
||||
Field2StatslogPaths: statsLogs,
|
||||
Bm25Logs: bm25StatsLogs,
|
||||
Channel: t.GetChannelName(),
|
||||
IsSorted: true,
|
||||
StorageVersion: t.storageVersion,
|
||||
},
|
||||
}
|
||||
planResult := &datapb.CompactionPlanResult{
|
||||
State: datapb.CompactionTaskState_completed,
|
||||
PlanID: t.GetPlanID(),
|
||||
Channel: t.GetChannelName(),
|
||||
Segments: res,
|
||||
Type: t.plan.GetType(),
|
||||
}
|
||||
return planResult, nil
|
||||
}
|
||||
|
||||
func (t *sortCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
|
||||
if !funcutil.CheckCtxValid(t.ctx) {
|
||||
return nil, t.ctx.Err()
|
||||
}
|
||||
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(t.ctx, fmt.Sprintf("MixCompact-%d", t.GetPlanID()))
|
||||
defer span.End()
|
||||
if err := t.preCompact(); err != nil {
|
||||
log.Warn("failed to preCompact", zap.Error(err))
|
||||
return &datapb.CompactionPlanResult{
|
||||
PlanID: t.GetPlanID(),
|
||||
State: datapb.CompactionTaskState_failed,
|
||||
}, nil
|
||||
}
|
||||
|
||||
compactStart := time.Now()
|
||||
|
||||
ctx, cancelAll := context.WithTimeout(ctx, time.Duration(t.plan.GetTimeoutInSeconds())*time.Second)
|
||||
defer cancelAll()
|
||||
|
||||
log := log.Ctx(ctx).With(zap.Int64("planID", t.GetPlanID()),
|
||||
zap.Int64("collectionID", t.collectionID),
|
||||
zap.Int64("partitionID", t.partitionID),
|
||||
zap.Int64("segmentID", t.segmentID),
|
||||
zap.Int64("totalRows", t.plan.GetTotalRows()),
|
||||
zap.Int64("slotUsage", t.plan.GetSlotUsage()))
|
||||
|
||||
log.Info("compact start")
|
||||
|
||||
res, err := t.sortSegment(ctx)
|
||||
if err != nil {
|
||||
log.Warn("failed to sort segment",
|
||||
zap.Error(err))
|
||||
return &datapb.CompactionPlanResult{
|
||||
PlanID: t.GetPlanID(),
|
||||
State: datapb.CompactionTaskState_failed,
|
||||
}, nil
|
||||
}
|
||||
targetSegemntID := res.GetSegments()[0].GetSegmentID()
|
||||
insertLogs := res.GetSegments()[0].GetInsertLogs()
|
||||
if len(insertLogs) == 0 || res.GetSegments()[0].GetNumOfRows() == 0 {
|
||||
log.Info("compact done, but target segment is zero num rows",
|
||||
zap.Int64("targetSegmentID", targetSegemntID),
|
||||
zap.Duration("compact cost", time.Since(compactStart)))
|
||||
return res, nil
|
||||
}
|
||||
textStatsLogs, err := t.createTextIndex(ctx,
|
||||
t.collectionID, t.partitionID, targetSegemntID, t.GetPlanID(),
|
||||
res.GetSegments()[0].GetInsertLogs())
|
||||
if err != nil {
|
||||
log.Warn("failed to create text indexes", zap.Int64("targetSegmentID", targetSegemntID),
|
||||
zap.Error(err))
|
||||
return &datapb.CompactionPlanResult{
|
||||
PlanID: t.GetPlanID(),
|
||||
State: datapb.CompactionTaskState_failed,
|
||||
}, nil
|
||||
}
|
||||
res.Segments[0].TextStatsLogs = textStatsLogs
|
||||
log.Info("compact done", zap.Int64("targetSegmentID", targetSegemntID),
|
||||
zap.Duration("compact cost", time.Since(compactStart)))
|
||||
return res, nil
|
||||
}
|
||||
|
||||
func (t *sortCompactionTask) Complete() {
|
||||
t.done <- struct{}{}
|
||||
}
|
||||
|
||||
func (t *sortCompactionTask) Stop() {
|
||||
t.cancel()
|
||||
<-t.done
|
||||
}
|
||||
|
||||
func (t *sortCompactionTask) GetPlanID() typeutil.UniqueID {
|
||||
return t.plan.GetPlanID()
|
||||
}
|
||||
|
||||
func (t *sortCompactionTask) GetChannelName() string {
|
||||
return t.plan.GetChannel()
|
||||
}
|
||||
|
||||
func (t *sortCompactionTask) GetCompactionType() datapb.CompactionType {
|
||||
return datapb.CompactionType_SortCompaction
|
||||
}
|
||||
|
||||
func (t *sortCompactionTask) GetCollection() typeutil.UniqueID {
|
||||
return t.collectionID
|
||||
}
|
||||
|
||||
func (t *sortCompactionTask) GetSlotUsage() int64 {
|
||||
return t.plan.GetSlotUsage()
|
||||
}
|
||||
|
||||
func (t *sortCompactionTask) createTextIndex(ctx context.Context,
|
||||
collectionID int64,
|
||||
partitionID int64,
|
||||
segmentID int64,
|
||||
taskID int64,
|
||||
insertBinlogs []*datapb.FieldBinlog,
|
||||
) (map[int64]*datapb.TextIndexStats, error) {
|
||||
log := log.Ctx(ctx).With(
|
||||
zap.Int64("collectionID", collectionID),
|
||||
zap.Int64("partitionID", partitionID),
|
||||
zap.Int64("segmentID", segmentID),
|
||||
)
|
||||
|
||||
fieldBinlogs := lo.GroupBy(insertBinlogs, func(binlog *datapb.FieldBinlog) int64 {
|
||||
return binlog.GetFieldID()
|
||||
})
|
||||
|
||||
getInsertFiles := func(fieldID int64) ([]string, error) {
|
||||
if t.storageVersion == storage.StorageV2 {
|
||||
return []string{}, nil
|
||||
}
|
||||
binlogs, ok := fieldBinlogs[fieldID]
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("field binlog not found for field %d", fieldID)
|
||||
}
|
||||
result := make([]string, 0, len(binlogs))
|
||||
for _, binlog := range binlogs {
|
||||
for _, file := range binlog.GetBinlogs() {
|
||||
result = append(result, metautil.BuildInsertLogPath(t.compactionParams.StorageConfig.GetRootPath(),
|
||||
collectionID, partitionID, segmentID, fieldID, file.GetLogID()))
|
||||
}
|
||||
}
|
||||
return result, nil
|
||||
}
|
||||
|
||||
newStorageConfig, err := util.ParseStorageConfig(t.compactionParams.StorageConfig)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
textIndexLogs := make(map[int64]*datapb.TextIndexStats)
|
||||
for _, field := range t.plan.GetSchema().GetFields() {
|
||||
h := typeutil.CreateFieldSchemaHelper(field)
|
||||
if !h.EnableMatch() {
|
||||
continue
|
||||
}
|
||||
log.Info("field enable match, ready to create text index", zap.Int64("field id", field.GetFieldID()))
|
||||
// create text index and upload the text index files.
|
||||
files, err := getInsertFiles(field.GetFieldID())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
buildIndexParams := &indexcgopb.BuildIndexInfo{
|
||||
BuildID: t.GetPlanID(),
|
||||
CollectionID: collectionID,
|
||||
PartitionID: partitionID,
|
||||
SegmentID: segmentID,
|
||||
IndexVersion: 0, // always zero
|
||||
InsertFiles: files,
|
||||
FieldSchema: field,
|
||||
StorageConfig: newStorageConfig,
|
||||
CurrentScalarIndexVersion: t.plan.GetCurrentScalarIndexVersion(),
|
||||
StorageVersion: t.storageVersion,
|
||||
}
|
||||
|
||||
if t.storageVersion == storage.StorageV2 {
|
||||
buildIndexParams.SegmentInsertFiles = util.GetSegmentInsertFiles(
|
||||
insertBinlogs,
|
||||
t.compactionParams.StorageConfig,
|
||||
collectionID,
|
||||
partitionID,
|
||||
segmentID)
|
||||
}
|
||||
uploaded, err := indexcgowrapper.CreateTextIndex(ctx, buildIndexParams)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
textIndexLogs[field.GetFieldID()] = &datapb.TextIndexStats{
|
||||
FieldID: field.GetFieldID(),
|
||||
Version: 0,
|
||||
BuildID: taskID,
|
||||
Files: lo.Keys(uploaded),
|
||||
}
|
||||
elapse := t.tr.RecordSpan()
|
||||
log.Info("field enable match, create text index done",
|
||||
zap.Int64("segmentID", segmentID),
|
||||
zap.Int64("field id", field.GetFieldID()),
|
||||
zap.Strings("files", lo.Keys(uploaded)),
|
||||
zap.Duration("elapse", elapse),
|
||||
)
|
||||
}
|
||||
return textIndexLogs, nil
|
||||
}
|
||||
377
internal/datanode/compactor/sort_compaction_test.go
Normal file
377
internal/datanode/compactor/sort_compaction_test.go
Normal file
@ -0,0 +1,377 @@
|
||||
// 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.
|
||||
|
||||
package compactor
|
||||
|
||||
import (
|
||||
"context"
|
||||
"math"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/samber/lo"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/suite"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/compaction"
|
||||
"github.com/milvus-io/milvus/internal/mocks/flushcommon/mock_util"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/etcdpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/tsoutil"
|
||||
)
|
||||
|
||||
func TestSortCompactionTaskSuite(t *testing.T) {
|
||||
suite.Run(t, new(SortCompactionTaskSuite))
|
||||
}
|
||||
|
||||
type SortCompactionTaskSuite struct {
|
||||
suite.Suite
|
||||
|
||||
mockBinlogIO *mock_util.MockBinlogIO
|
||||
|
||||
meta *etcdpb.CollectionMeta
|
||||
segWriter *SegmentWriter
|
||||
|
||||
task *sortCompactionTask
|
||||
}
|
||||
|
||||
func (s *SortCompactionTaskSuite) SetupSuite() {
|
||||
paramtable.Get().Init(paramtable.NewBaseTable())
|
||||
}
|
||||
|
||||
func (s *SortCompactionTaskSuite) setupTest() {
|
||||
s.mockBinlogIO = mock_util.NewMockBinlogIO(s.T())
|
||||
|
||||
s.meta = genTestCollectionMeta()
|
||||
|
||||
params, err := compaction.GenerateJSONParams()
|
||||
s.NoError(err)
|
||||
|
||||
plan := &datapb.CompactionPlan{
|
||||
PlanID: 999,
|
||||
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{{
|
||||
SegmentID: 100,
|
||||
FieldBinlogs: nil,
|
||||
Field2StatslogPaths: nil,
|
||||
Deltalogs: nil,
|
||||
}},
|
||||
TimeoutInSeconds: 10,
|
||||
Type: datapb.CompactionType_SortCompaction,
|
||||
Schema: s.meta.GetSchema(),
|
||||
PreAllocatedSegmentIDs: &datapb.IDRange{Begin: 19531, End: math.MaxInt64},
|
||||
PreAllocatedLogIDs: &datapb.IDRange{Begin: 9530, End: 19530},
|
||||
MaxSize: 64 * 1024 * 1024,
|
||||
SlotUsage: 8,
|
||||
JsonParams: params,
|
||||
TotalRows: 1000,
|
||||
CollectionTtl: time.Since(getMilvusBirthday().Add(-time.Hour)).Nanoseconds(),
|
||||
}
|
||||
|
||||
s.task = NewSortCompactionTask(context.Background(), s.mockBinlogIO, plan, compaction.GenParams())
|
||||
}
|
||||
|
||||
func (s *SortCompactionTaskSuite) SetupTest() {
|
||||
s.setupTest()
|
||||
}
|
||||
|
||||
func (s *SortCompactionTaskSuite) TearDownTest() {
|
||||
}
|
||||
|
||||
func (s *SortCompactionTaskSuite) TestNewSortCompactionTask() {
|
||||
plan := &datapb.CompactionPlan{
|
||||
PlanID: 123,
|
||||
Type: datapb.CompactionType_SortCompaction,
|
||||
SlotUsage: 8,
|
||||
}
|
||||
|
||||
task := NewSortCompactionTask(context.Background(), s.mockBinlogIO, plan, compaction.GenParams())
|
||||
|
||||
s.NotNil(task)
|
||||
s.Equal(plan.GetPlanID(), task.GetPlanID())
|
||||
s.Equal(datapb.CompactionType_SortCompaction, task.GetCompactionType())
|
||||
s.Equal(int64(8), task.GetSlotUsage())
|
||||
}
|
||||
|
||||
func (s *SortCompactionTaskSuite) TestPreCompactValidation() {
|
||||
// Test with multiple segments (should fail)
|
||||
s.task.plan.SegmentBinlogs = []*datapb.CompactionSegmentBinlogs{
|
||||
{SegmentID: 100},
|
||||
{SegmentID: 101},
|
||||
}
|
||||
|
||||
err := s.task.preCompact()
|
||||
s.Error(err)
|
||||
s.Contains(err.Error(), "sort compaction should handle exactly one segment")
|
||||
|
||||
// Test with single segment (should pass basic validation)
|
||||
s.task.plan.SegmentBinlogs = []*datapb.CompactionSegmentBinlogs{
|
||||
{
|
||||
SegmentID: 100,
|
||||
CollectionID: 1001,
|
||||
PartitionID: 1002,
|
||||
FieldBinlogs: []*datapb.FieldBinlog{},
|
||||
Deltalogs: []*datapb.FieldBinlog{},
|
||||
},
|
||||
}
|
||||
|
||||
err = s.task.preCompact()
|
||||
s.NoError(err)
|
||||
s.Equal(int64(1001), s.task.collectionID)
|
||||
s.Equal(int64(1002), s.task.partitionID)
|
||||
s.Equal(int64(100), s.task.segmentID)
|
||||
}
|
||||
|
||||
func (s *SortCompactionTaskSuite) prepareSortCompactionTask() {
|
||||
segmentID := int64(1001)
|
||||
alloc := allocator.NewLocalAllocator(100, math.MaxInt64)
|
||||
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil)
|
||||
|
||||
// Initialize segment buffer with test data
|
||||
s.initSegBuffer(1000, segmentID)
|
||||
kvs, fBinlogs, err := serializeWrite(context.TODO(), alloc, s.segWriter)
|
||||
s.Require().NoError(err)
|
||||
|
||||
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.MatchedBy(func(keys []string) bool {
|
||||
left, right := lo.Difference(keys, lo.Keys(kvs))
|
||||
return len(left) == 0 && len(right) == 0
|
||||
})).Return(lo.Values(kvs), nil).Once()
|
||||
|
||||
// Create delta log for deletion
|
||||
deleteTs := tsoutil.ComposeTSByTime(getMilvusBirthday(), 5)
|
||||
blob, err := getInt64DeltaBlobs(segmentID, []int64{segmentID}, []uint64{deleteTs})
|
||||
s.Require().NoError(err)
|
||||
deltaPath := "deltalog/1001"
|
||||
s.mockBinlogIO.EXPECT().Download(mock.Anything, []string{deltaPath}).
|
||||
Return([][]byte{blob.GetValue()}, nil).Once()
|
||||
|
||||
s.task.plan.SegmentBinlogs = []*datapb.CompactionSegmentBinlogs{
|
||||
{
|
||||
SegmentID: segmentID,
|
||||
CollectionID: CollectionID,
|
||||
PartitionID: PartitionID,
|
||||
FieldBinlogs: lo.Values(fBinlogs),
|
||||
Deltalogs: []*datapb.FieldBinlog{
|
||||
{Binlogs: []*datapb.Binlog{{LogPath: deltaPath}}},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (s *SortCompactionTaskSuite) TestSortCompactionBasic() {
|
||||
s.prepareSortCompactionTask()
|
||||
|
||||
result, err := s.task.Compact()
|
||||
s.NoError(err)
|
||||
s.NotNil(result)
|
||||
|
||||
s.Equal(s.task.plan.GetPlanID(), result.GetPlanID())
|
||||
s.Equal(datapb.CompactionTaskState_completed, result.GetState())
|
||||
s.Equal(datapb.CompactionType_SortCompaction, result.GetType())
|
||||
s.Equal(1, len(result.GetSegments()))
|
||||
|
||||
segment := result.GetSegments()[0]
|
||||
s.EqualValues(19531, segment.GetSegmentID())
|
||||
s.True(segment.GetIsSorted()) // Sort compaction should mark segment as sorted
|
||||
s.EqualValues(len(s.task.plan.Schema.Fields), len(segment.InsertLogs))
|
||||
s.EqualValues(1, len(segment.Field2StatslogPaths))
|
||||
s.Empty(segment.Deltalogs)
|
||||
// delete 5 counts
|
||||
s.Equal(int64(995), segment.GetNumOfRows())
|
||||
}
|
||||
|
||||
func (s *SortCompactionTaskSuite) TestSortCompactionWithBM25() {
|
||||
s.setupBM25Test()
|
||||
s.prepareSortCompactionWithBM25Task()
|
||||
|
||||
result, err := s.task.Compact()
|
||||
s.NoError(err)
|
||||
s.NotNil(result)
|
||||
|
||||
s.Equal(s.task.plan.GetPlanID(), result.GetPlanID())
|
||||
s.Equal(1, len(result.GetSegments()))
|
||||
|
||||
segment := result.GetSegments()[0]
|
||||
s.EqualValues(19531, segment.GetSegmentID())
|
||||
s.True(segment.GetIsSorted())
|
||||
s.EqualValues(len(s.task.plan.Schema.Fields), len(segment.InsertLogs))
|
||||
s.EqualValues(1, len(segment.Field2StatslogPaths))
|
||||
s.EqualValues(1, len(segment.Bm25Logs)) // Should have BM25 logs
|
||||
s.Empty(segment.Deltalogs)
|
||||
}
|
||||
|
||||
func (s *SortCompactionTaskSuite) setupBM25Test() {
|
||||
s.mockBinlogIO = mock_util.NewMockBinlogIO(s.T())
|
||||
s.meta = genTestCollectionMetaWithBM25()
|
||||
params, err := compaction.GenerateJSONParams()
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
plan := &datapb.CompactionPlan{
|
||||
PlanID: 999,
|
||||
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{{
|
||||
SegmentID: 100,
|
||||
FieldBinlogs: nil,
|
||||
Field2StatslogPaths: nil,
|
||||
Deltalogs: nil,
|
||||
}},
|
||||
TimeoutInSeconds: 10,
|
||||
Type: datapb.CompactionType_SortCompaction,
|
||||
Schema: s.meta.GetSchema(),
|
||||
PreAllocatedSegmentIDs: &datapb.IDRange{Begin: 19531, End: math.MaxInt64},
|
||||
PreAllocatedLogIDs: &datapb.IDRange{Begin: 9530, End: 19530},
|
||||
MaxSize: 64 * 1024 * 1024,
|
||||
JsonParams: params,
|
||||
TotalRows: 3,
|
||||
}
|
||||
|
||||
s.task = NewSortCompactionTask(context.Background(), s.mockBinlogIO, plan, compaction.GenParams())
|
||||
}
|
||||
|
||||
func (s *SortCompactionTaskSuite) prepareSortCompactionWithBM25Task() {
|
||||
segmentID := int64(1001)
|
||||
alloc := allocator.NewLocalAllocator(100, math.MaxInt64)
|
||||
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil)
|
||||
|
||||
s.initSegBufferWithBM25(segmentID)
|
||||
kvs, fBinlogs, err := serializeWrite(context.TODO(), alloc, s.segWriter)
|
||||
s.Require().NoError(err)
|
||||
|
||||
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.MatchedBy(func(keys []string) bool {
|
||||
left, right := lo.Difference(keys, lo.Keys(kvs))
|
||||
return len(left) == 0 && len(right) == 0
|
||||
})).Return(lo.Values(kvs), nil).Once()
|
||||
|
||||
s.task.plan.SegmentBinlogs = []*datapb.CompactionSegmentBinlogs{
|
||||
{
|
||||
SegmentID: segmentID,
|
||||
CollectionID: CollectionID,
|
||||
PartitionID: PartitionID,
|
||||
FieldBinlogs: lo.Values(fBinlogs),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (s *SortCompactionTaskSuite) TestSortCompactionWithExpiredData() {
|
||||
segmentID := int64(1001)
|
||||
|
||||
s.initSegBuffer(1, segmentID)
|
||||
collTTL := 864000 // 10 days
|
||||
s.task.currentTime = getMilvusBirthday().Add(time.Second * (time.Duration(collTTL) + 1))
|
||||
s.task.plan.CollectionTtl = int64(collTTL)
|
||||
alloc := allocator.NewLocalAllocator(888888, math.MaxInt64)
|
||||
|
||||
kvs, fBinlogs, err := serializeWrite(context.TODO(), alloc, s.segWriter)
|
||||
s.Require().NoError(err)
|
||||
|
||||
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).RunAndReturn(
|
||||
func(ctx context.Context, paths []string) ([][]byte, error) {
|
||||
s.Require().Equal(len(paths), len(kvs))
|
||||
return lo.Values(kvs), nil
|
||||
})
|
||||
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
|
||||
s.task.plan.SegmentBinlogs = []*datapb.CompactionSegmentBinlogs{
|
||||
{
|
||||
SegmentID: segmentID,
|
||||
CollectionID: CollectionID,
|
||||
PartitionID: PartitionID,
|
||||
FieldBinlogs: lo.Values(fBinlogs),
|
||||
},
|
||||
}
|
||||
|
||||
result, err := s.task.Compact()
|
||||
s.NoError(err)
|
||||
s.NotNil(result)
|
||||
|
||||
// Due to TTL expiration, some data might be filtered out
|
||||
segment := result.GetSegments()[0]
|
||||
s.True(segment.GetIsSorted())
|
||||
}
|
||||
|
||||
func (s *SortCompactionTaskSuite) TestSortCompactionFail() {
|
||||
// Test with invalid plan (no segments)
|
||||
s.task.plan.SegmentBinlogs = []*datapb.CompactionSegmentBinlogs{}
|
||||
|
||||
result, err := s.task.Compact()
|
||||
s.NoError(err) // Should not return error, but result should indicate failure
|
||||
s.NotNil(result)
|
||||
s.Equal(datapb.CompactionTaskState_failed, result.GetState())
|
||||
}
|
||||
|
||||
func (s *SortCompactionTaskSuite) TestTaskInterface() {
|
||||
// Test Compactor interface methods
|
||||
s.Equal(int64(999), s.task.GetPlanID())
|
||||
s.Equal(datapb.CompactionType_SortCompaction, s.task.GetCompactionType())
|
||||
s.Equal(int64(8), s.task.GetSlotUsage())
|
||||
|
||||
// Test task lifecycle
|
||||
s.task.Complete()
|
||||
|
||||
s.task.Stop()
|
||||
}
|
||||
|
||||
// Helper methods (copied/adapted from mix compaction tests)
|
||||
func (s *SortCompactionTaskSuite) initSegBuffer(size int, seed int64) {
|
||||
s.segWriter, _ = NewSegmentWriter(s.meta.GetSchema(), int64(size), compactionBatchSize, 1, PartitionID, CollectionID, []int64{})
|
||||
|
||||
for i := 0; i < size; i++ {
|
||||
v := storage.Value{
|
||||
PK: storage.NewInt64PrimaryKey(seed),
|
||||
Timestamp: int64(tsoutil.ComposeTSByTime(getMilvusBirthday(), int64(i))),
|
||||
Value: getRow(seed, int64(tsoutil.ComposeTSByTime(getMilvusBirthday(), int64(i)))),
|
||||
}
|
||||
err := s.segWriter.Write(&v)
|
||||
s.Require().NoError(err)
|
||||
}
|
||||
s.segWriter.FlushAndIsFull()
|
||||
}
|
||||
|
||||
func (s *SortCompactionTaskSuite) initSegBufferWithBM25(seed int64) {
|
||||
s.segWriter, _ = NewSegmentWriter(s.meta.GetSchema(), 1, compactionBatchSize, 1, PartitionID, CollectionID, []int64{102})
|
||||
|
||||
v := storage.Value{
|
||||
PK: storage.NewInt64PrimaryKey(seed),
|
||||
Timestamp: int64(tsoutil.ComposeTSByTime(getMilvusBirthday(), 0)),
|
||||
Value: genRowWithBM25(seed),
|
||||
}
|
||||
err := s.segWriter.Write(&v)
|
||||
s.Require().NoError(err)
|
||||
s.segWriter.FlushAndIsFull()
|
||||
}
|
||||
|
||||
func TestSortCompactionTaskBasic(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
mockBinlogIO := mock_util.NewMockBinlogIO(t)
|
||||
|
||||
plan := &datapb.CompactionPlan{
|
||||
PlanID: 123,
|
||||
Type: datapb.CompactionType_SortCompaction,
|
||||
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
|
||||
{SegmentID: 100},
|
||||
},
|
||||
}
|
||||
|
||||
task := NewSortCompactionTask(ctx, mockBinlogIO, plan, compaction.GenParams())
|
||||
|
||||
assert.NotNil(t, task)
|
||||
assert.Equal(t, int64(123), task.GetPlanID())
|
||||
assert.Equal(t, datapb.CompactionType_SortCompaction, task.GetCompactionType())
|
||||
}
|
||||
@ -27,6 +27,7 @@ import (
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/indexcgowrapper"
|
||||
"github.com/milvus-io/milvus/internal/util/vecindexmgr"
|
||||
@ -297,7 +298,7 @@ func (it *indexBuildTask) Execute(ctx context.Context) error {
|
||||
}
|
||||
|
||||
if buildIndexParams.StorageVersion == storage.StorageV2 {
|
||||
buildIndexParams.SegmentInsertFiles = GetSegmentInsertFiles(
|
||||
buildIndexParams.SegmentInsertFiles = util.GetSegmentInsertFiles(
|
||||
it.req.GetInsertLogs(),
|
||||
it.req.GetStorageConfig(),
|
||||
it.req.GetCollectionID(),
|
||||
|
||||
@ -33,6 +33,7 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/compaction"
|
||||
"github.com/milvus-io/milvus/internal/datanode/compactor"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/flushcommon/io"
|
||||
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
@ -620,7 +621,7 @@ func buildIndexParams(
|
||||
}
|
||||
|
||||
if req.GetStorageVersion() == storage.StorageV2 {
|
||||
params.SegmentInsertFiles = GetSegmentInsertFiles(
|
||||
params.SegmentInsertFiles = util.GetSegmentInsertFiles(
|
||||
req.GetInsertLogs(),
|
||||
req.GetStorageConfig(),
|
||||
req.GetCollectionID(),
|
||||
|
||||
@ -28,18 +28,12 @@ package index
|
||||
import "C"
|
||||
|
||||
import (
|
||||
"path"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/common"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/hardware"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/metautil"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||
)
|
||||
@ -102,24 +96,3 @@ func CalculateNodeSlots() int64 {
|
||||
}
|
||||
return totalSlot
|
||||
}
|
||||
|
||||
func GetSegmentInsertFiles(fieldBinlogs []*datapb.FieldBinlog, storageConfig *indexpb.StorageConfig, collectionID int64, partitionID int64, segmentID int64) *indexcgopb.SegmentInsertFiles {
|
||||
insertLogs := make([]*indexcgopb.FieldInsertFiles, 0)
|
||||
for _, insertLog := range fieldBinlogs {
|
||||
filePaths := make([]string, 0)
|
||||
columnGroupID := insertLog.GetFieldID()
|
||||
for _, binlog := range insertLog.GetBinlogs() {
|
||||
filePath := metautil.BuildInsertLogPath(storageConfig.GetRootPath(), collectionID, partitionID, segmentID, columnGroupID, binlog.GetLogID())
|
||||
if storageConfig.StorageType != "local" {
|
||||
filePath = path.Join(storageConfig.GetBucketName(), filePath)
|
||||
}
|
||||
filePaths = append(filePaths, filePath)
|
||||
}
|
||||
insertLogs = append(insertLogs, &indexcgopb.FieldInsertFiles{
|
||||
FilePaths: filePaths,
|
||||
})
|
||||
}
|
||||
return &indexcgopb.SegmentInsertFiles{
|
||||
FieldInsertFiles: insertLogs,
|
||||
}
|
||||
}
|
||||
|
||||
@ -207,6 +207,7 @@ func (node *DataNode) CompactionV2(ctx context.Context, req *datapb.CompactionPl
|
||||
binlogIO,
|
||||
cm,
|
||||
req,
|
||||
compactionParams,
|
||||
)
|
||||
case datapb.CompactionType_MixCompaction:
|
||||
if req.GetPreAllocatedSegmentIDs() == nil || req.GetPreAllocatedSegmentIDs().GetBegin() == 0 {
|
||||
@ -216,6 +217,7 @@ func (node *DataNode) CompactionV2(ctx context.Context, req *datapb.CompactionPl
|
||||
taskCtx,
|
||||
binlogIO,
|
||||
req,
|
||||
compactionParams,
|
||||
)
|
||||
case datapb.CompactionType_ClusteringCompaction:
|
||||
if req.GetPreAllocatedSegmentIDs() == nil || req.GetPreAllocatedSegmentIDs().GetBegin() == 0 {
|
||||
@ -225,6 +227,17 @@ func (node *DataNode) CompactionV2(ctx context.Context, req *datapb.CompactionPl
|
||||
taskCtx,
|
||||
binlogIO,
|
||||
req,
|
||||
compactionParams,
|
||||
)
|
||||
case datapb.CompactionType_SortCompaction:
|
||||
if req.GetPreAllocatedSegmentIDs() == nil || req.GetPreAllocatedSegmentIDs().GetBegin() == 0 {
|
||||
return merr.Status(merr.WrapErrParameterInvalidMsg("invalid pre-allocated segmentID range")), nil
|
||||
}
|
||||
task = compactor.NewSortCompactionTask(
|
||||
taskCtx,
|
||||
binlogIO,
|
||||
req,
|
||||
compactionParams,
|
||||
)
|
||||
default:
|
||||
log.Warn("Unknown compaction type", zap.String("type", req.GetType().String()))
|
||||
|
||||
43
internal/datanode/util/util.go
Normal file
43
internal/datanode/util/util.go
Normal file
@ -0,0 +1,43 @@
|
||||
package util
|
||||
|
||||
import (
|
||||
"path"
|
||||
|
||||
"google.golang.org/protobuf/proto"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/pkg/v2/util/metautil"
|
||||
)
|
||||
|
||||
func ParseStorageConfig(s *indexpb.StorageConfig) (*indexcgopb.StorageConfig, error) {
|
||||
bs, err := proto.Marshal(s)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
res := &indexcgopb.StorageConfig{}
|
||||
err = proto.Unmarshal(bs, res)
|
||||
return res, err
|
||||
}
|
||||
|
||||
func GetSegmentInsertFiles(fieldBinlogs []*datapb.FieldBinlog, storageConfig *indexpb.StorageConfig, collectionID int64, partitionID int64, segmentID int64) *indexcgopb.SegmentInsertFiles {
|
||||
insertLogs := make([]*indexcgopb.FieldInsertFiles, 0)
|
||||
for _, insertLog := range fieldBinlogs {
|
||||
filePaths := make([]string, 0)
|
||||
columnGroupID := insertLog.GetFieldID()
|
||||
for _, binlog := range insertLog.GetBinlogs() {
|
||||
filePath := metautil.BuildInsertLogPath(storageConfig.GetRootPath(), collectionID, partitionID, segmentID, columnGroupID, binlog.GetLogID())
|
||||
if storageConfig.StorageType != "local" {
|
||||
filePath = path.Join(storageConfig.GetBucketName(), filePath)
|
||||
}
|
||||
filePaths = append(filePaths, filePath)
|
||||
}
|
||||
insertLogs = append(insertLogs, &indexcgopb.FieldInsertFiles{
|
||||
FilePaths: filePaths,
|
||||
})
|
||||
}
|
||||
return &indexcgopb.SegmentInsertFiles{
|
||||
FieldInsertFiles: insertLogs,
|
||||
}
|
||||
}
|
||||
@ -108,18 +108,18 @@ func DecompressMultiBinLogs(infos []*datapb.SegmentInfo) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func DecompressCompactionBinlogs(binlogs []*datapb.CompactionSegmentBinlogs) error {
|
||||
func DecompressCompactionBinlogsWithRootPath(rootPath string, binlogs []*datapb.CompactionSegmentBinlogs) error {
|
||||
for _, binlog := range binlogs {
|
||||
collectionID, partitionID, segmentID := binlog.GetCollectionID(), binlog.GetPartitionID(), binlog.GetSegmentID()
|
||||
err := DecompressBinLog(storage.InsertBinlog, collectionID, partitionID, segmentID, binlog.GetFieldBinlogs())
|
||||
err := DecompressBinLogWithRootPath(rootPath, storage.InsertBinlog, collectionID, partitionID, segmentID, binlog.GetFieldBinlogs())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = DecompressBinLog(storage.DeleteBinlog, collectionID, partitionID, segmentID, binlog.GetDeltalogs())
|
||||
err = DecompressBinLogWithRootPath(rootPath, storage.DeleteBinlog, collectionID, partitionID, segmentID, binlog.GetDeltalogs())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = DecompressBinLog(storage.StatsBinlog, collectionID, partitionID, segmentID, binlog.GetField2StatslogPaths())
|
||||
err = DecompressBinLogWithRootPath(rootPath, storage.StatsBinlog, collectionID, partitionID, segmentID, binlog.GetField2StatslogPaths())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
@ -581,6 +581,7 @@ enum CompactionType {
|
||||
MajorCompaction = 6;
|
||||
Level0DeleteCompaction = 7;
|
||||
ClusteringCompaction = 8;
|
||||
SortCompaction = 9;
|
||||
}
|
||||
|
||||
message CompactionStateRequest {
|
||||
@ -651,6 +652,7 @@ message CompactionPlan {
|
||||
// collection is importing
|
||||
IDRange pre_allocated_logIDs = 21;
|
||||
string json_params = 22;
|
||||
int32 current_scalar_index_version = 23;
|
||||
}
|
||||
|
||||
message CompactionSegment {
|
||||
@ -664,6 +666,7 @@ message CompactionSegment {
|
||||
bool is_sorted = 8;
|
||||
repeated FieldBinlog bm25logs = 9;
|
||||
int64 storage_version = 10;
|
||||
map<int64, data.TextIndexStats> text_stats_logs = 11;
|
||||
}
|
||||
|
||||
message CompactionPlanResult {
|
||||
@ -974,7 +977,7 @@ message ImportTaskV2 {
|
||||
string reason = 7;
|
||||
string complete_time = 8;
|
||||
repeated ImportFileStats file_stats = 9;
|
||||
repeated int64 stats_segmentIDs = 10;
|
||||
repeated int64 sorted_segmentIDs = 10;
|
||||
string created_time = 11;
|
||||
ImportTaskSourceV2 source = 12;
|
||||
}
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@ -324,6 +324,7 @@ message StatsTask {
|
||||
|
||||
enum StatsSubJob {
|
||||
None = 0;
|
||||
// deprecated, after v2.6.0, move to compaction
|
||||
Sort = 1;
|
||||
TextIndexJob = 2;
|
||||
BM25Job=3;
|
||||
|
||||
@ -138,6 +138,7 @@ type StatsSubJob int32
|
||||
|
||||
const (
|
||||
StatsSubJob_None StatsSubJob = 0
|
||||
// deprecated, after v2.6.0, move to compaction
|
||||
StatsSubJob_Sort StatsSubJob = 1
|
||||
StatsSubJob_TextIndexJob StatsSubJob = 2
|
||||
StatsSubJob_BM25Job StatsSubJob = 3
|
||||
|
||||
@ -322,7 +322,7 @@ enum ImportJobState {
|
||||
Failed = 4;
|
||||
Completed = 5;
|
||||
IndexBuilding = 6;
|
||||
Stats = 7;
|
||||
Sorting = 7;
|
||||
}
|
||||
|
||||
message ImportFile {
|
||||
|
||||
@ -161,7 +161,7 @@ const (
|
||||
ImportJobState_Failed ImportJobState = 4
|
||||
ImportJobState_Completed ImportJobState = 5
|
||||
ImportJobState_IndexBuilding ImportJobState = 6
|
||||
ImportJobState_Stats ImportJobState = 7
|
||||
ImportJobState_Sorting ImportJobState = 7
|
||||
)
|
||||
|
||||
// Enum value maps for ImportJobState.
|
||||
@ -174,7 +174,7 @@ var (
|
||||
4: "Failed",
|
||||
5: "Completed",
|
||||
6: "IndexBuilding",
|
||||
7: "Stats",
|
||||
7: "Sorting",
|
||||
}
|
||||
ImportJobState_value = map[string]int32{
|
||||
"None": 0,
|
||||
@ -184,7 +184,7 @@ var (
|
||||
"Failed": 4,
|
||||
"Completed": 5,
|
||||
"IndexBuilding": 6,
|
||||
"Stats": 7,
|
||||
"Sorting": 7,
|
||||
}
|
||||
)
|
||||
|
||||
@ -4695,19 +4695,19 @@ var file_internal_proto_rawDesc = []byte{
|
||||
0x09, 0x44, 0x51, 0x4c, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x10, 0x08, 0x12, 0x0c, 0x0a, 0x08,
|
||||
0x44, 0x51, 0x4c, 0x51, 0x75, 0x65, 0x72, 0x79, 0x10, 0x09, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d,
|
||||
0x4c, 0x55, 0x70, 0x73, 0x65, 0x72, 0x74, 0x10, 0x0a, 0x12, 0x09, 0x0a, 0x05, 0x44, 0x44, 0x4c,
|
||||
0x44, 0x42, 0x10, 0x0b, 0x2a, 0x81, 0x01, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a,
|
||||
0x44, 0x42, 0x10, 0x0b, 0x2a, 0x83, 0x01, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a,
|
||||
0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10,
|
||||
0x00, 0x12, 0x0b, 0x0a, 0x07, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, 0x10,
|
||||
0x0a, 0x0c, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x02,
|
||||
0x12, 0x0d, 0x0a, 0x09, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x03, 0x12,
|
||||
0x0a, 0x0a, 0x06, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x43,
|
||||
0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x49, 0x6e,
|
||||
0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x12, 0x09, 0x0a,
|
||||
0x05, 0x53, 0x74, 0x61, 0x74, 0x73, 0x10, 0x07, 0x42, 0x35, 0x5a, 0x33, 0x67, 0x69, 0x74, 0x68,
|
||||
0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f,
|
||||
0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70,
|
||||
0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x62,
|
||||
0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x12, 0x0b, 0x0a,
|
||||
0x07, 0x53, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x07, 0x42, 0x35, 0x5a, 0x33, 0x67, 0x69,
|
||||
0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d,
|
||||
0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32,
|
||||
0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70,
|
||||
0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
|
||||
@ -165,6 +165,7 @@ message CreateStatsRequest {
|
||||
string insert_channel = 5;
|
||||
int64 segmentID = 6;
|
||||
repeated data.FieldBinlog insert_logs = 7;
|
||||
// deprecated, after sort stats moved, its not used.
|
||||
repeated data.FieldBinlog delta_logs = 8;
|
||||
index.StorageConfig storage_config = 9;
|
||||
schema.CollectionSchema schema = 10;
|
||||
@ -173,13 +174,17 @@ message CreateStatsRequest {
|
||||
int64 startLogID = 13;
|
||||
int64 endLogID = 14;
|
||||
int64 num_rows = 15;
|
||||
// deprecated, after sort stats moved, its not used.
|
||||
int64 collection_ttl = 16;
|
||||
// deprecated, after sort stats moved, its not used.
|
||||
uint64 current_ts = 17;
|
||||
int64 task_version = 18;
|
||||
// deprecated, after sort stats moved, its not used.
|
||||
uint64 binlogMaxSize = 19;
|
||||
bool enable_json_key_stats = 20;
|
||||
int64 json_key_stats_tantivy_memory = 21;
|
||||
int64 json_key_stats_data_format = 22;
|
||||
// deprecated, the sort logic has been moved into the compaction process.
|
||||
bool enable_json_key_stats_in_sort = 23;
|
||||
int64 task_slot = 24;
|
||||
int64 storage_version = 25;
|
||||
|
||||
@ -1051,6 +1051,7 @@ type CreateStatsRequest struct {
|
||||
InsertChannel string `protobuf:"bytes,5,opt,name=insert_channel,json=insertChannel,proto3" json:"insert_channel,omitempty"`
|
||||
SegmentID int64 `protobuf:"varint,6,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
|
||||
InsertLogs []*datapb.FieldBinlog `protobuf:"bytes,7,rep,name=insert_logs,json=insertLogs,proto3" json:"insert_logs,omitempty"`
|
||||
// deprecated, after sort stats moved, its not used.
|
||||
DeltaLogs []*datapb.FieldBinlog `protobuf:"bytes,8,rep,name=delta_logs,json=deltaLogs,proto3" json:"delta_logs,omitempty"`
|
||||
StorageConfig *indexpb.StorageConfig `protobuf:"bytes,9,opt,name=storage_config,json=storageConfig,proto3" json:"storage_config,omitempty"`
|
||||
Schema *schemapb.CollectionSchema `protobuf:"bytes,10,opt,name=schema,proto3" json:"schema,omitempty"`
|
||||
@ -1059,13 +1060,17 @@ type CreateStatsRequest struct {
|
||||
StartLogID int64 `protobuf:"varint,13,opt,name=startLogID,proto3" json:"startLogID,omitempty"`
|
||||
EndLogID int64 `protobuf:"varint,14,opt,name=endLogID,proto3" json:"endLogID,omitempty"`
|
||||
NumRows int64 `protobuf:"varint,15,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"`
|
||||
// deprecated, after sort stats moved, its not used.
|
||||
CollectionTtl int64 `protobuf:"varint,16,opt,name=collection_ttl,json=collectionTtl,proto3" json:"collection_ttl,omitempty"`
|
||||
// deprecated, after sort stats moved, its not used.
|
||||
CurrentTs uint64 `protobuf:"varint,17,opt,name=current_ts,json=currentTs,proto3" json:"current_ts,omitempty"`
|
||||
TaskVersion int64 `protobuf:"varint,18,opt,name=task_version,json=taskVersion,proto3" json:"task_version,omitempty"`
|
||||
// deprecated, after sort stats moved, its not used.
|
||||
BinlogMaxSize uint64 `protobuf:"varint,19,opt,name=binlogMaxSize,proto3" json:"binlogMaxSize,omitempty"`
|
||||
EnableJsonKeyStats bool `protobuf:"varint,20,opt,name=enable_json_key_stats,json=enableJsonKeyStats,proto3" json:"enable_json_key_stats,omitempty"`
|
||||
JsonKeyStatsTantivyMemory int64 `protobuf:"varint,21,opt,name=json_key_stats_tantivy_memory,json=jsonKeyStatsTantivyMemory,proto3" json:"json_key_stats_tantivy_memory,omitempty"`
|
||||
JsonKeyStatsDataFormat int64 `protobuf:"varint,22,opt,name=json_key_stats_data_format,json=jsonKeyStatsDataFormat,proto3" json:"json_key_stats_data_format,omitempty"`
|
||||
// deprecated, the sort logic has been moved into the compaction process.
|
||||
EnableJsonKeyStatsInSort bool `protobuf:"varint,23,opt,name=enable_json_key_stats_in_sort,json=enableJsonKeyStatsInSort,proto3" json:"enable_json_key_stats_in_sort,omitempty"`
|
||||
TaskSlot int64 `protobuf:"varint,24,opt,name=task_slot,json=taskSlot,proto3" json:"task_slot,omitempty"`
|
||||
StorageVersion int64 `protobuf:"varint,25,opt,name=storage_version,json=storageVersion,proto3" json:"storage_version,omitempty"`
|
||||
|
||||
@ -4066,12 +4066,11 @@ type dataCoordConfig struct {
|
||||
StatsTaskSlotUsage ParamItem `refreshable:"true"`
|
||||
AnalyzeTaskSlotUsage ParamItem `refreshable:"true"`
|
||||
|
||||
EnableStatsTask ParamItem `refreshable:"true"`
|
||||
EnableSortCompaction ParamItem `refreshable:"true"`
|
||||
TaskCheckInterval ParamItem `refreshable:"true"`
|
||||
StatsTaskTriggerCount ParamItem `refreshable:"true"`
|
||||
SortCompactionTriggerCount ParamItem `refreshable:"true"`
|
||||
JSONStatsTriggerCount ParamItem `refreshable:"true"`
|
||||
JSONStatsTriggerInterval ParamItem `refreshable:"true"`
|
||||
EnabledJSONKeyStatsInSort ParamItem `refreshable:"true"`
|
||||
JSONKeyStatsMemoryBudgetInTantivy ParamItem `refreshable:"false"`
|
||||
|
||||
RequestTimeoutSeconds ParamItem `refreshable:"true"`
|
||||
@ -5097,16 +5096,17 @@ if param targetVecIndexVersion is not set, the default value is -1, which means
|
||||
}
|
||||
p.AnalyzeTaskSlotUsage.Init(base.mgr)
|
||||
|
||||
p.EnableStatsTask = ParamItem{
|
||||
Key: "dataCoord.statsTask.enable",
|
||||
p.EnableSortCompaction = ParamItem{
|
||||
Key: "dataCoord.sortCompaction.enable",
|
||||
Version: "2.5.0",
|
||||
Doc: "enable stats task",
|
||||
Doc: "enable sort compaction",
|
||||
FallbackKeys: []string{"dataCoord.statsTask.enable"},
|
||||
DefaultValue: "true",
|
||||
PanicIfEmpty: false,
|
||||
Export: false,
|
||||
Forbidden: true,
|
||||
}
|
||||
p.EnableStatsTask.Init(base.mgr)
|
||||
p.EnableSortCompaction.Init(base.mgr)
|
||||
|
||||
p.TaskCheckInterval = ParamItem{
|
||||
Key: "dataCoord.taskCheckInterval",
|
||||
@ -5118,15 +5118,16 @@ if param targetVecIndexVersion is not set, the default value is -1, which means
|
||||
}
|
||||
p.TaskCheckInterval.Init(base.mgr)
|
||||
|
||||
p.StatsTaskTriggerCount = ParamItem{
|
||||
Key: "dataCoord.statsTaskTriggerCount",
|
||||
p.SortCompactionTriggerCount = ParamItem{
|
||||
Key: "dataCoord.sortCompactionTriggerCount",
|
||||
Version: "2.5.5",
|
||||
FallbackKeys: []string{"dataCoord.statsTaskTriggerCount"},
|
||||
Doc: "stats task count per trigger",
|
||||
DefaultValue: "100",
|
||||
PanicIfEmpty: false,
|
||||
Export: false,
|
||||
}
|
||||
p.StatsTaskTriggerCount.Init(base.mgr)
|
||||
p.SortCompactionTriggerCount.Init(base.mgr)
|
||||
|
||||
p.JSONStatsTriggerCount = ParamItem{
|
||||
Key: "dataCoord.jsonStatsTriggerCount",
|
||||
@ -5158,15 +5159,6 @@ if param targetVecIndexVersion is not set, the default value is -1, which means
|
||||
}
|
||||
p.RequestTimeoutSeconds.Init(base.mgr)
|
||||
|
||||
p.StatsTaskTriggerCount = ParamItem{
|
||||
Key: "dataCoord.statsTaskTriggerCount",
|
||||
Version: "2.5.5",
|
||||
Doc: "stats task count per trigger",
|
||||
DefaultValue: "100",
|
||||
PanicIfEmpty: false,
|
||||
Export: false,
|
||||
}
|
||||
p.StatsTaskTriggerCount.Init(base.mgr)
|
||||
p.JSONKeyStatsMemoryBudgetInTantivy = ParamItem{
|
||||
Key: "dataCoord.jsonKeyStatsMemoryBudgetInTantivy",
|
||||
Version: "2.5.5",
|
||||
@ -5175,15 +5167,6 @@ if param targetVecIndexVersion is not set, the default value is -1, which means
|
||||
Export: true,
|
||||
}
|
||||
p.JSONKeyStatsMemoryBudgetInTantivy.Init(base.mgr)
|
||||
|
||||
p.EnabledJSONKeyStatsInSort = ParamItem{
|
||||
Key: "dataCoord.enabledJSONKeyStatsInSort",
|
||||
Version: "2.5.5",
|
||||
DefaultValue: "false",
|
||||
Doc: "Indicates whether to enable JSON key stats task with sort",
|
||||
Export: true,
|
||||
}
|
||||
p.EnabledJSONKeyStatsInSort.Init(base.mgr)
|
||||
}
|
||||
|
||||
// /////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
@ -565,11 +565,11 @@ func TestComponentParam(t *testing.T) {
|
||||
assert.Equal(t, 1000*time.Second, Params.TaskSlowThreshold.GetAsDuration(time.Second))
|
||||
|
||||
params.Save("datacoord.statsTask.enable", "true")
|
||||
assert.True(t, Params.EnableStatsTask.GetAsBool())
|
||||
assert.True(t, Params.EnableSortCompaction.GetAsBool())
|
||||
params.Save("datacoord.taskCheckInterval", "500")
|
||||
assert.Equal(t, 500*time.Second, Params.TaskCheckInterval.GetAsDuration(time.Second))
|
||||
params.Save("datacoord.statsTaskTriggerCount", "3")
|
||||
assert.Equal(t, 3, Params.StatsTaskTriggerCount.GetAsInt())
|
||||
assert.Equal(t, 3, Params.SortCompactionTriggerCount.GetAsInt())
|
||||
})
|
||||
|
||||
t.Run("test dataNodeConfig", func(t *testing.T) {
|
||||
|
||||
@ -34,7 +34,6 @@ func (s *ClusteringCompactionNullDataSuite) SetupSuite() {
|
||||
s.WithMilvusConfig(paramtable.Get().DataCoordCfg.TaskScheduleInterval.Key, "100")
|
||||
s.WithMilvusConfig(paramtable.Get().PulsarCfg.MaxMessageSize.Key, strconv.Itoa(500*1024))
|
||||
s.WithMilvusConfig(paramtable.Get().DataNodeCfg.ClusteringCompactionWorkerPoolSize.Key, strconv.Itoa(8))
|
||||
s.WithMilvusConfig(paramtable.Get().DataNodeCfg.BinLogMaxSize.Key, strconv.Itoa(102400))
|
||||
s.WithMilvusConfig(paramtable.Get().DataCoordCfg.EnableAutoCompaction.Key, "false")
|
||||
s.WithMilvusConfig(paramtable.Get().DataCoordCfg.ClusteringCompactionMaxSegmentSizeRatio.Key, "1.0")
|
||||
s.WithMilvusConfig(paramtable.Get().DataCoordCfg.ClusteringCompactionPreferSegmentSizeRatio.Key, "1.0")
|
||||
|
||||
@ -49,7 +49,6 @@ func (s *ClusteringCompactionSuite) SetupSuite() {
|
||||
// 2000 rows for each segment, about 1MB.
|
||||
s.WithMilvusConfig(paramtable.Get().PulsarCfg.MaxMessageSize.Key, strconv.Itoa(500*1024))
|
||||
s.WithMilvusConfig(paramtable.Get().DataNodeCfg.ClusteringCompactionWorkerPoolSize.Key, strconv.Itoa(8))
|
||||
s.WithMilvusConfig(paramtable.Get().DataNodeCfg.BinLogMaxSize.Key, strconv.Itoa(102400))
|
||||
s.WithMilvusConfig(paramtable.Get().DataCoordCfg.EnableAutoCompaction.Key, "false")
|
||||
s.WithMilvusConfig(paramtable.Get().DataCoordCfg.ClusteringCompactionMaxSegmentSizeRatio.Key, "1.0")
|
||||
s.WithMilvusConfig(paramtable.Get().DataCoordCfg.ClusteringCompactionPreferSegmentSizeRatio.Key, "1.0")
|
||||
|
||||
@ -41,12 +41,17 @@ type LevelZeroSuite struct {
|
||||
}
|
||||
|
||||
func (s *LevelZeroSuite) SetupSuite() {
|
||||
s.WithMilvusConfig(paramtable.Get().DataCoordCfg.EnableStatsTask.Key, "false")
|
||||
s.WithMilvusConfig(paramtable.Get().DataCoordCfg.EnableSortCompaction.Key, "false")
|
||||
|
||||
s.MiniClusterSuite.SetupSuite()
|
||||
s.dim = 768
|
||||
}
|
||||
|
||||
func (s *LevelZeroSuite) TearDownSuite() {
|
||||
s.MiniClusterSuite.TearDownSuite()
|
||||
paramtable.Get().Reset(paramtable.Get().DataCoordCfg.EnableSortCompaction.Key)
|
||||
}
|
||||
|
||||
func TestLevelZero(t *testing.T) {
|
||||
suite.Run(t, new(LevelZeroSuite))
|
||||
}
|
||||
|
||||
@ -86,6 +86,7 @@ func (s *MiniClusterSuite) SetupSuite() {
|
||||
s.cancelFunc = cancel
|
||||
|
||||
s.Cluster = cluster.NewMiniClusterV3(ctx, cluster.WithExtraEnv(s.envConfigs), cluster.WithWorkDir(s.WorkDir()))
|
||||
s.T().Log("Setup test success")
|
||||
}
|
||||
|
||||
func (s *MiniClusterSuite) SetupTest() {
|
||||
|
||||
@ -374,8 +374,6 @@ class TestCompactionParams(TestcaseBase):
|
||||
"""
|
||||
# create collection shard_num=1, insert 2 segments, each with tmp_nb entities
|
||||
collection_w = self.init_collection_wrap(name=cf.gen_unique_str(prefix), shards_num=1)
|
||||
collection_w.create_index(ct.default_float_vec_field_name, ct.default_index)
|
||||
collection_w.compact()
|
||||
|
||||
# Notice:The merge segments compaction triggered by max_compaction_interval also needs to meet
|
||||
# the compaction_segment_ num_threshold
|
||||
@ -384,6 +382,8 @@ class TestCompactionParams(TestcaseBase):
|
||||
collection_w.insert(df)
|
||||
assert collection_w.num_entities == tmp_nb * (i + 1)
|
||||
|
||||
collection_w.create_index(ct.default_float_vec_field_name, ct.default_index)
|
||||
collection_w.compact()
|
||||
sleep(ct.max_compaction_interval + 1)
|
||||
|
||||
# verify queryNode load the compacted segments
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user