mirror of https://github.com/milvus-io/milvus.git
enhance: Pre-allocate ids for compaction (#34187)
This PR removes the dependency of compaction on the ID allocator by pre-allocating the logID and segmentID. issue: https://github.com/milvus-io/milvus/issues/33957 --------- Signed-off-by: bigsheeper <yihao.dai@zilliz.com>pull/34581/head
parent
90e765db87
commit
ca758c36cc
|
@ -577,18 +577,21 @@ func (c *compactionPlanHandler) createCompactTask(t *datapb.CompactionTask) (Com
|
|||
case datapb.CompactionType_MixCompaction:
|
||||
task = &mixCompactionTask{
|
||||
CompactionTask: t,
|
||||
allocator: c.allocator,
|
||||
meta: c.meta,
|
||||
sessions: c.sessions,
|
||||
}
|
||||
case datapb.CompactionType_Level0DeleteCompaction:
|
||||
task = &l0CompactionTask{
|
||||
CompactionTask: t,
|
||||
allocator: c.allocator,
|
||||
meta: c.meta,
|
||||
sessions: c.sessions,
|
||||
}
|
||||
case datapb.CompactionType_ClusteringCompaction:
|
||||
task = &clusteringCompactionTask{
|
||||
CompactionTask: t,
|
||||
allocator: c.allocator,
|
||||
meta: c.meta,
|
||||
sessions: c.sessions,
|
||||
handler: c.handler,
|
||||
|
|
|
@ -48,8 +48,9 @@ type clusteringCompactionTask struct {
|
|||
*datapb.CompactionTask
|
||||
plan *datapb.CompactionPlan
|
||||
result *datapb.CompactionPlanResult
|
||||
span trace.Span
|
||||
|
||||
span trace.Span
|
||||
allocator allocator
|
||||
meta CompactionMeta
|
||||
sessions SessionManager
|
||||
handler Handler
|
||||
|
@ -133,6 +134,10 @@ func (t *clusteringCompactionTask) retryableProcess() error {
|
|||
}
|
||||
|
||||
func (t *clusteringCompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, error) {
|
||||
beginLogID, _, err := t.allocator.allocN(1)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
plan := &datapb.CompactionPlan{
|
||||
PlanID: t.GetPlanID(),
|
||||
StartTime: t.GetStartTime(),
|
||||
|
@ -147,6 +152,11 @@ func (t *clusteringCompactionTask) BuildCompactionRequest() (*datapb.CompactionP
|
|||
PreferSegmentRows: t.GetPreferSegmentRows(),
|
||||
AnalyzeResultPath: path.Join(t.meta.(*meta).chunkManager.RootPath(), common.AnalyzeStatsPath, metautil.JoinIDPath(t.AnalyzeTaskID, t.AnalyzeVersion)),
|
||||
AnalyzeSegmentIds: t.GetInputSegments(), // todo: if need
|
||||
BeginLogID: beginLogID,
|
||||
PreAllocatedSegments: &datapb.IDRange{
|
||||
Begin: t.GetResultSegments()[0],
|
||||
End: t.GetResultSegments()[1],
|
||||
},
|
||||
}
|
||||
log := log.With(zap.Int64("taskID", t.GetTriggerID()), zap.Int64("planID", plan.GetPlanID()))
|
||||
|
||||
|
|
|
@ -20,6 +20,8 @@ import (
|
|||
"context"
|
||||
"fmt"
|
||||
|
||||
"github.com/stretchr/testify/mock"
|
||||
|
||||
"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/metastore/kv/datacoord"
|
||||
|
@ -50,6 +52,8 @@ func (s *CompactionTaskSuite) TestClusteringCompactionSegmentMetaChange() {
|
|||
},
|
||||
})
|
||||
session := NewSessionManagerImpl()
|
||||
alloc := NewNMockAllocator(s.T())
|
||||
alloc.EXPECT().allocN(mock.Anything).Return(100, 200, nil)
|
||||
|
||||
schema := ConstructScalarClusteringSchema("TestClusteringCompactionTask", 32, true)
|
||||
pk := &schemapb.FieldSchema{
|
||||
|
@ -77,9 +81,11 @@ func (s *CompactionTaskSuite) TestClusteringCompactionSegmentMetaChange() {
|
|||
Schema: schema,
|
||||
ClusteringKeyField: pk,
|
||||
InputSegments: []int64{101, 102},
|
||||
ResultSegments: []int64{1000, 1100},
|
||||
},
|
||||
meta: meta,
|
||||
sessions: session,
|
||||
meta: meta,
|
||||
sessions: session,
|
||||
allocator: alloc,
|
||||
}
|
||||
|
||||
task.processPipelining()
|
||||
|
|
|
@ -37,11 +37,13 @@ var _ CompactionTask = (*l0CompactionTask)(nil)
|
|||
|
||||
type l0CompactionTask struct {
|
||||
*datapb.CompactionTask
|
||||
plan *datapb.CompactionPlan
|
||||
result *datapb.CompactionPlanResult
|
||||
span trace.Span
|
||||
sessions SessionManager
|
||||
meta CompactionMeta
|
||||
plan *datapb.CompactionPlan
|
||||
result *datapb.CompactionPlanResult
|
||||
|
||||
span trace.Span
|
||||
allocator allocator
|
||||
sessions SessionManager
|
||||
meta CompactionMeta
|
||||
}
|
||||
|
||||
// Note: return True means exit this state machine.
|
||||
|
@ -230,6 +232,10 @@ func (t *l0CompactionTask) CleanLogPath() {
|
|||
}
|
||||
|
||||
func (t *l0CompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, error) {
|
||||
beginLogID, _, err := t.allocator.allocN(1)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
plan := &datapb.CompactionPlan{
|
||||
PlanID: t.GetPlanID(),
|
||||
StartTime: t.GetStartTime(),
|
||||
|
@ -239,6 +245,7 @@ func (t *l0CompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, err
|
|||
CollectionTtl: t.GetCollectionTtl(),
|
||||
TotalRows: t.GetTotalRows(),
|
||||
Schema: t.GetSchema(),
|
||||
BeginLogID: beginLogID,
|
||||
}
|
||||
|
||||
log := log.With(zap.Int64("taskID", t.GetTriggerID()), zap.Int64("planID", plan.GetPlanID()))
|
||||
|
|
|
@ -1,3 +1,19 @@
|
|||
// 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 datacoord
|
||||
|
||||
import (
|
||||
|
@ -58,6 +74,9 @@ func (s *CompactionTaskSuite) TestProcessRefreshPlan_NormalL0() {
|
|||
},
|
||||
meta: s.mockMeta,
|
||||
}
|
||||
alloc := NewNMockAllocator(s.T())
|
||||
alloc.EXPECT().allocN(mock.Anything).Return(100, 200, nil)
|
||||
task.allocator = alloc
|
||||
plan, err := task.BuildCompactionRequest()
|
||||
s.Require().NoError(err)
|
||||
|
||||
|
@ -88,6 +107,9 @@ func (s *CompactionTaskSuite) TestProcessRefreshPlan_SegmentNotFoundL0() {
|
|||
},
|
||||
meta: s.mockMeta,
|
||||
}
|
||||
alloc := NewNMockAllocator(s.T())
|
||||
alloc.EXPECT().allocN(mock.Anything).Return(100, 200, nil)
|
||||
task.allocator = alloc
|
||||
|
||||
_, err := task.BuildCompactionRequest()
|
||||
s.Error(err)
|
||||
|
@ -121,10 +143,41 @@ func (s *CompactionTaskSuite) TestProcessRefreshPlan_SelectZeroSegmentsL0() {
|
|||
},
|
||||
meta: s.mockMeta,
|
||||
}
|
||||
alloc := NewNMockAllocator(s.T())
|
||||
alloc.EXPECT().allocN(mock.Anything).Return(100, 200, nil)
|
||||
task.allocator = alloc
|
||||
_, err := task.BuildCompactionRequest()
|
||||
s.Error(err)
|
||||
}
|
||||
|
||||
func (s *CompactionTaskSuite) TestBuildCompactionRequestFailed_AllocFailed() {
|
||||
var task CompactionTask
|
||||
|
||||
alloc := NewNMockAllocator(s.T())
|
||||
alloc.EXPECT().allocN(mock.Anything).Return(100, 200, errors.New("mock alloc err"))
|
||||
|
||||
task = &l0CompactionTask{
|
||||
allocator: alloc,
|
||||
}
|
||||
_, err := task.BuildCompactionRequest()
|
||||
s.T().Logf("err=%v", err)
|
||||
s.Error(err)
|
||||
|
||||
task = &mixCompactionTask{
|
||||
allocator: alloc,
|
||||
}
|
||||
_, err = task.BuildCompactionRequest()
|
||||
s.T().Logf("err=%v", err)
|
||||
s.Error(err)
|
||||
|
||||
task = &clusteringCompactionTask{
|
||||
allocator: alloc,
|
||||
}
|
||||
_, err = task.BuildCompactionRequest()
|
||||
s.T().Logf("err=%v", err)
|
||||
s.Error(err)
|
||||
}
|
||||
|
||||
func generateTestL0Task(state datapb.CompactionTaskState) *l0CompactionTask {
|
||||
return &l0CompactionTask{
|
||||
CompactionTask: &datapb.CompactionTask{
|
||||
|
@ -145,9 +198,13 @@ func (s *CompactionTaskSuite) SetupSubTest() {
|
|||
}
|
||||
|
||||
func (s *CompactionTaskSuite) TestProcessStateTrans() {
|
||||
alloc := NewNMockAllocator(s.T())
|
||||
alloc.EXPECT().allocN(mock.Anything).Return(100, 200, nil)
|
||||
|
||||
s.Run("test pipelining needReassignNodeID", func() {
|
||||
t := generateTestL0Task(datapb.CompactionTaskState_pipelining)
|
||||
t.NodeID = NullNodeID
|
||||
t.allocator = alloc
|
||||
got := t.Process()
|
||||
s.False(got)
|
||||
s.Equal(datapb.CompactionTaskState_pipelining, t.State)
|
||||
|
@ -157,6 +214,7 @@ func (s *CompactionTaskSuite) TestProcessStateTrans() {
|
|||
s.Run("test pipelining BuildCompactionRequest failed", func() {
|
||||
t := generateTestL0Task(datapb.CompactionTaskState_pipelining)
|
||||
t.NodeID = 100
|
||||
t.allocator = alloc
|
||||
channel := "ch-1"
|
||||
deltaLogs := []*datapb.FieldBinlog{getFieldBinlogIDs(101, 3)}
|
||||
|
||||
|
@ -194,6 +252,7 @@ func (s *CompactionTaskSuite) TestProcessStateTrans() {
|
|||
s.Run("test pipelining Compaction failed", func() {
|
||||
t := generateTestL0Task(datapb.CompactionTaskState_pipelining)
|
||||
t.NodeID = 100
|
||||
t.allocator = alloc
|
||||
channel := "ch-1"
|
||||
deltaLogs := []*datapb.FieldBinlog{getFieldBinlogIDs(101, 3)}
|
||||
|
||||
|
@ -234,6 +293,7 @@ func (s *CompactionTaskSuite) TestProcessStateTrans() {
|
|||
s.Run("test pipelining success", func() {
|
||||
t := generateTestL0Task(datapb.CompactionTaskState_pipelining)
|
||||
t.NodeID = 100
|
||||
t.allocator = alloc
|
||||
channel := "ch-1"
|
||||
deltaLogs := []*datapb.FieldBinlog{getFieldBinlogIDs(101, 3)}
|
||||
|
||||
|
|
|
@ -18,9 +18,11 @@ var _ CompactionTask = (*mixCompactionTask)(nil)
|
|||
|
||||
type mixCompactionTask struct {
|
||||
*datapb.CompactionTask
|
||||
plan *datapb.CompactionPlan
|
||||
result *datapb.CompactionPlanResult
|
||||
plan *datapb.CompactionPlan
|
||||
result *datapb.CompactionPlanResult
|
||||
|
||||
span trace.Span
|
||||
allocator allocator
|
||||
sessions SessionManager
|
||||
meta CompactionMeta
|
||||
newSegment *SegmentInfo
|
||||
|
@ -328,6 +330,10 @@ func (t *mixCompactionTask) CleanLogPath() {
|
|||
}
|
||||
|
||||
func (t *mixCompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, error) {
|
||||
beginLogID, _, err := t.allocator.allocN(1)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
plan := &datapb.CompactionPlan{
|
||||
PlanID: t.GetPlanID(),
|
||||
StartTime: t.GetStartTime(),
|
||||
|
@ -337,6 +343,10 @@ func (t *mixCompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, er
|
|||
CollectionTtl: t.GetCollectionTtl(),
|
||||
TotalRows: t.GetTotalRows(),
|
||||
Schema: t.GetSchema(),
|
||||
BeginLogID: beginLogID,
|
||||
PreAllocatedSegments: &datapb.IDRange{
|
||||
Begin: t.GetResultSegments()[0],
|
||||
},
|
||||
}
|
||||
log := log.With(zap.Int64("taskID", t.GetTriggerID()), zap.Int64("planID", plan.GetPlanID()))
|
||||
|
||||
|
|
|
@ -23,18 +23,22 @@ func (s *CompactionTaskSuite) TestProcessRefreshPlan_NormalMix() {
|
|||
}).Times(2)
|
||||
task := &mixCompactionTask{
|
||||
CompactionTask: &datapb.CompactionTask{
|
||||
PlanID: 1,
|
||||
TriggerID: 19530,
|
||||
CollectionID: 1,
|
||||
PartitionID: 10,
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
NodeID: 1,
|
||||
State: datapb.CompactionTaskState_executing,
|
||||
InputSegments: []int64{200, 201},
|
||||
PlanID: 1,
|
||||
TriggerID: 19530,
|
||||
CollectionID: 1,
|
||||
PartitionID: 10,
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
NodeID: 1,
|
||||
State: datapb.CompactionTaskState_executing,
|
||||
InputSegments: []int64{200, 201},
|
||||
ResultSegments: []int64{100},
|
||||
},
|
||||
// plan: plan,
|
||||
meta: s.mockMeta,
|
||||
}
|
||||
alloc := NewNMockAllocator(s.T())
|
||||
alloc.EXPECT().allocN(mock.Anything).Return(100, 200, nil)
|
||||
task.allocator = alloc
|
||||
plan, err := task.BuildCompactionRequest()
|
||||
s.Require().NoError(err)
|
||||
|
||||
|
@ -53,18 +57,22 @@ func (s *CompactionTaskSuite) TestProcessRefreshPlan_MixSegmentNotFound() {
|
|||
}).Once()
|
||||
task := &mixCompactionTask{
|
||||
CompactionTask: &datapb.CompactionTask{
|
||||
PlanID: 1,
|
||||
TriggerID: 19530,
|
||||
CollectionID: 1,
|
||||
PartitionID: 10,
|
||||
Channel: channel,
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
State: datapb.CompactionTaskState_executing,
|
||||
NodeID: 1,
|
||||
InputSegments: []int64{200, 201},
|
||||
PlanID: 1,
|
||||
TriggerID: 19530,
|
||||
CollectionID: 1,
|
||||
PartitionID: 10,
|
||||
Channel: channel,
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
State: datapb.CompactionTaskState_executing,
|
||||
NodeID: 1,
|
||||
InputSegments: []int64{200, 201},
|
||||
ResultSegments: []int64{100},
|
||||
},
|
||||
meta: s.mockMeta,
|
||||
}
|
||||
alloc := NewNMockAllocator(s.T())
|
||||
alloc.EXPECT().allocN(mock.Anything).Return(100, 200, nil)
|
||||
task.allocator = alloc
|
||||
_, err := task.BuildCompactionRequest()
|
||||
s.Error(err)
|
||||
s.ErrorIs(err, merr.ErrSegmentNotFound)
|
||||
|
|
|
@ -399,7 +399,7 @@ func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) error {
|
|||
}
|
||||
|
||||
plans := t.generatePlans(group.segments, signal, ct)
|
||||
currentID, _, err := t.allocator.allocN(int64(len(plans)))
|
||||
currentID, _, err := t.allocator.allocN(int64(len(plans) * 2))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -413,6 +413,8 @@ func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) error {
|
|||
start := time.Now()
|
||||
planID := currentID
|
||||
currentID++
|
||||
targetSegmentID := currentID
|
||||
currentID++
|
||||
pts, _ := tsoutil.ParseTS(ct.startTime)
|
||||
task := &datapb.CompactionTask{
|
||||
PlanID: planID,
|
||||
|
@ -426,6 +428,7 @@ func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) error {
|
|||
PartitionID: group.partitionID,
|
||||
Channel: group.channelName,
|
||||
InputSegments: segIDs,
|
||||
ResultSegments: []int64{targetSegmentID}, // pre-allocated target segment
|
||||
TotalRows: totalRows,
|
||||
Schema: coll.Schema,
|
||||
}
|
||||
|
@ -503,7 +506,7 @@ func (t *compactionTrigger) handleSignal(signal *compactionSignal) {
|
|||
}
|
||||
|
||||
plans := t.generatePlans(segments, signal, ct)
|
||||
currentID, _, err := t.allocator.allocN(int64(len(plans)))
|
||||
currentID, _, err := t.allocator.allocN(int64(len(plans) * 2))
|
||||
if err != nil {
|
||||
log.Warn("fail to allocate id", zap.Error(err))
|
||||
return
|
||||
|
@ -518,6 +521,8 @@ func (t *compactionTrigger) handleSignal(signal *compactionSignal) {
|
|||
start := time.Now()
|
||||
planID := currentID
|
||||
currentID++
|
||||
targetSegmentID := currentID
|
||||
currentID++
|
||||
pts, _ := tsoutil.ParseTS(ct.startTime)
|
||||
if err := t.compactionHandler.enqueueCompaction(&datapb.CompactionTask{
|
||||
PlanID: planID,
|
||||
|
@ -531,6 +536,7 @@ func (t *compactionTrigger) handleSignal(signal *compactionSignal) {
|
|||
PartitionID: partitionID,
|
||||
Channel: channel,
|
||||
InputSegments: segmentIDS,
|
||||
ResultSegments: []int64{targetSegmentID}, // pre-allocated target segment
|
||||
TotalRows: totalRows,
|
||||
Schema: coll.Schema,
|
||||
}); err != nil {
|
||||
|
|
|
@ -66,6 +66,9 @@ func (h *spyCompactionHandler) enqueueCompaction(task *datapb.CompactionTask) er
|
|||
CompactionTask: task,
|
||||
meta: h.meta,
|
||||
}
|
||||
alloc := &MockAllocator0{}
|
||||
t.allocator = alloc
|
||||
t.ResultSegments = []int64{100}
|
||||
plan, err := t.BuildCompactionRequest()
|
||||
h.spyChan <- plan
|
||||
return err
|
||||
|
@ -474,11 +477,12 @@ func Test_compactionTrigger_force(t *testing.T) {
|
|||
},
|
||||
},
|
||||
// StartTime: 0,
|
||||
TimeoutInSeconds: Params.DataCoordCfg.CompactionTimeoutInSeconds.GetAsInt32(),
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
Channel: "ch1",
|
||||
TotalRows: 200,
|
||||
Schema: schema,
|
||||
TimeoutInSeconds: Params.DataCoordCfg.CompactionTimeoutInSeconds.GetAsInt32(),
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
Channel: "ch1",
|
||||
TotalRows: 200,
|
||||
Schema: schema,
|
||||
PreAllocatedSegments: &datapb.IDRange{Begin: 100},
|
||||
},
|
||||
},
|
||||
},
|
||||
|
|
|
@ -250,6 +250,7 @@ func (m *CompactionTriggerManager) SubmitL0ViewToScheduler(ctx context.Context,
|
|||
zap.Int64("planID", task.GetPlanID()),
|
||||
zap.Int64s("segmentIDs", task.GetInputSegments()),
|
||||
zap.Error(err))
|
||||
return
|
||||
}
|
||||
log.Info("Finish to submit a LevelZeroCompaction plan",
|
||||
zap.Int64("taskID", taskID),
|
||||
|
@ -272,6 +273,12 @@ func (m *CompactionTriggerManager) SubmitClusteringViewToScheduler(ctx context.C
|
|||
return
|
||||
}
|
||||
_, totalRows, maxSegmentRows, preferSegmentRows := calculateClusteringCompactionConfig(view)
|
||||
resultSegmentNum := totalRows / preferSegmentRows * 2
|
||||
start, end, err := m.allocator.allocN(resultSegmentNum)
|
||||
if err != nil {
|
||||
log.Warn("pre-allocate result segments failed", zap.String("view", view.String()))
|
||||
return
|
||||
}
|
||||
task := &datapb.CompactionTask{
|
||||
PlanID: taskID,
|
||||
TriggerID: view.(*ClusteringSegmentsView).triggerID,
|
||||
|
@ -286,6 +293,7 @@ func (m *CompactionTriggerManager) SubmitClusteringViewToScheduler(ctx context.C
|
|||
Schema: collection.Schema,
|
||||
ClusteringKeyField: view.(*ClusteringSegmentsView).clusteringKeyField,
|
||||
InputSegments: lo.Map(view.GetSegmentsView(), func(segmentView *SegmentView, _ int) int64 { return segmentView.ID }),
|
||||
ResultSegments: []int64{start, end}, // pre-allocated result segments range
|
||||
MaxSegmentRows: maxSegmentRows,
|
||||
PreferSegmentRows: preferSegmentRows,
|
||||
TotalRows: totalRows,
|
||||
|
@ -299,6 +307,7 @@ func (m *CompactionTriggerManager) SubmitClusteringViewToScheduler(ctx context.C
|
|||
zap.Int64("planID", task.GetPlanID()),
|
||||
zap.Int64s("segmentIDs", task.GetInputSegments()),
|
||||
zap.Error(err))
|
||||
return
|
||||
}
|
||||
log.Info("Finish to submit a clustering compaction task",
|
||||
zap.Int64("taskID", taskID),
|
||||
|
|
|
@ -221,7 +221,7 @@ func AssembleImportRequest(task ImportTask, job ImportJob, meta *meta, alloc all
|
|||
Files: importFiles,
|
||||
Options: job.GetOptions(),
|
||||
Ts: ts,
|
||||
AutoIDRange: &datapb.AutoIDRange{Begin: idBegin, End: idEnd},
|
||||
IDRange: &datapb.IDRange{Begin: idBegin, End: idEnd},
|
||||
RequestSegments: requestSegments,
|
||||
}, nil
|
||||
}
|
||||
|
|
|
@ -36,7 +36,7 @@ import (
|
|||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datanode/io"
|
||||
"github.com/milvus-io/milvus/internal/proto/clusteringpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
|
@ -59,8 +59,9 @@ import (
|
|||
var _ Compactor = (*clusteringCompactionTask)(nil)
|
||||
|
||||
type clusteringCompactionTask struct {
|
||||
binlogIO io.BinlogIO
|
||||
allocator allocator.Allocator
|
||||
binlogIO io.BinlogIO
|
||||
logIDAlloc allocator.Interface
|
||||
segIDAlloc allocator.Interface
|
||||
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
|
@ -127,15 +128,17 @@ type FlushSignal struct {
|
|||
func NewClusteringCompactionTask(
|
||||
ctx context.Context,
|
||||
binlogIO io.BinlogIO,
|
||||
alloc allocator.Allocator,
|
||||
plan *datapb.CompactionPlan,
|
||||
) *clusteringCompactionTask {
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
logIDAlloc := allocator.NewLocalAllocator(plan.GetBeginLogID(), math.MaxInt64)
|
||||
segIDAlloc := allocator.NewLocalAllocator(plan.GetPreAllocatedSegments().GetBegin(), plan.GetPreAllocatedSegments().GetEnd())
|
||||
return &clusteringCompactionTask{
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
binlogIO: binlogIO,
|
||||
allocator: alloc,
|
||||
logIDAlloc: logIDAlloc,
|
||||
segIDAlloc: segIDAlloc,
|
||||
plan: plan,
|
||||
tr: timerecord.NewTimeRecorder("clustering_compaction"),
|
||||
done: make(chan struct{}, 1),
|
||||
|
@ -295,7 +298,9 @@ func (t *clusteringCompactionTask) getScalarAnalyzeResult(ctx context.Context) e
|
|||
uploadedSegmentStats: make(map[typeutil.UniqueID]storage.SegmentStats, 0),
|
||||
clusteringKeyFieldStats: fieldStats,
|
||||
}
|
||||
t.refreshBufferWriter(buffer)
|
||||
if _, err = t.refreshBufferWriter(buffer); err != nil {
|
||||
return err
|
||||
}
|
||||
t.clusterBuffers = append(t.clusterBuffers, buffer)
|
||||
for _, key := range bucket {
|
||||
scalarToClusterBufferMap[key] = buffer
|
||||
|
@ -346,7 +351,9 @@ func (t *clusteringCompactionTask) getVectorAnalyzeResult(ctx context.Context) e
|
|||
uploadedSegmentStats: make(map[typeutil.UniqueID]storage.SegmentStats, 0),
|
||||
clusteringKeyFieldStats: fieldStats,
|
||||
}
|
||||
t.refreshBufferWriter(clusterBuffer)
|
||||
if _, err = t.refreshBufferWriter(clusterBuffer); err != nil {
|
||||
return err
|
||||
}
|
||||
t.clusterBuffers = append(t.clusterBuffers, clusterBuffer)
|
||||
}
|
||||
t.offsetToBufferFunc = func(offset int64, idMapping []uint32) *ClusterBuffer {
|
||||
|
@ -805,7 +812,7 @@ func (t *clusteringCompactionTask) packBufferToSegment(ctx context.Context, buff
|
|||
for _, fieldBinlog := range buffer.flushedBinlogs {
|
||||
insertLogs = append(insertLogs, fieldBinlog)
|
||||
}
|
||||
statPaths, err := statSerializeWrite(ctx, t.binlogIO, t.allocator, writer, buffer.flushedRowNum.Load())
|
||||
statPaths, err := statSerializeWrite(ctx, t.binlogIO, t.logIDAlloc, writer, buffer.flushedRowNum.Load())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -870,7 +877,7 @@ func (t *clusteringCompactionTask) flushBinlog(ctx context.Context, buffer *Clus
|
|||
}
|
||||
|
||||
start := time.Now()
|
||||
kvs, partialBinlogs, err := serializeWrite(ctx, t.allocator, writer)
|
||||
kvs, partialBinlogs, err := serializeWrite(ctx, t.logIDAlloc, writer)
|
||||
if err != nil {
|
||||
log.Warn("compact wrong, failed to serialize writer", zap.Error(err))
|
||||
return err
|
||||
|
@ -1136,7 +1143,7 @@ func (t *clusteringCompactionTask) refreshBufferWriter(buffer *ClusterBuffer) (b
|
|||
}
|
||||
if buffer.writer == nil || buffer.currentSegmentRowNum.Load()+buffer.writer.GetRowNum() > t.plan.GetMaxSegmentRows() {
|
||||
pack = true
|
||||
segmentID, err = t.allocator.AllocOne()
|
||||
segmentID, err = t.segIDAlloc.AllocOne()
|
||||
if err != nil {
|
||||
return pack, err
|
||||
}
|
||||
|
|
|
@ -25,7 +25,6 @@ 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/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datanode/io"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
|
@ -41,7 +40,6 @@ type ClusteringCompactionTaskSuite struct {
|
|||
suite.Suite
|
||||
|
||||
mockBinlogIO *io.MockBinlogIO
|
||||
mockAlloc *allocator.MockAllocator
|
||||
|
||||
task *clusteringCompactionTask
|
||||
|
||||
|
@ -54,9 +52,8 @@ func (s *ClusteringCompactionTaskSuite) SetupSuite() {
|
|||
|
||||
func (s *ClusteringCompactionTaskSuite) SetupTest() {
|
||||
s.mockBinlogIO = io.NewMockBinlogIO(s.T())
|
||||
s.mockAlloc = allocator.NewMockAllocator(s.T())
|
||||
|
||||
s.task = NewClusteringCompactionTask(context.Background(), s.mockBinlogIO, s.mockAlloc, nil)
|
||||
s.task = NewClusteringCompactionTask(context.Background(), s.mockBinlogIO, nil)
|
||||
|
||||
paramtable.Get().Save(paramtable.Get().CommonCfg.EntityExpirationTTL.Key, "0")
|
||||
|
||||
|
|
|
@ -24,7 +24,7 @@ import (
|
|||
"go.opentelemetry.io/otel"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datanode/io"
|
||||
iter "github.com/milvus-io/milvus/internal/datanode/iterators"
|
||||
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
|
||||
|
@ -130,7 +130,7 @@ func loadDeltaMap(segments []*datapb.CompactionSegmentBinlogs) (map[typeutil.Uni
|
|||
return deltaPaths, allPath, nil
|
||||
}
|
||||
|
||||
func serializeWrite(ctx context.Context, allocator allocator.Allocator, writer *SegmentWriter) (kvs map[string][]byte, fieldBinlogs map[int64]*datapb.FieldBinlog, err error) {
|
||||
func serializeWrite(ctx context.Context, allocator allocator.Interface, writer *SegmentWriter) (kvs map[string][]byte, fieldBinlogs map[int64]*datapb.FieldBinlog, err error) {
|
||||
_, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "serializeWrite")
|
||||
defer span.End()
|
||||
|
||||
|
@ -166,7 +166,7 @@ func serializeWrite(ctx context.Context, allocator allocator.Allocator, writer *
|
|||
return
|
||||
}
|
||||
|
||||
func statSerializeWrite(ctx context.Context, io io.BinlogIO, allocator allocator.Allocator, writer *SegmentWriter, finalRowCount int64) (*datapb.FieldBinlog, error) {
|
||||
func statSerializeWrite(ctx context.Context, io io.BinlogIO, allocator allocator.Interface, writer *SegmentWriter, finalRowCount int64) (*datapb.FieldBinlog, error) {
|
||||
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "statslog serializeWrite")
|
||||
defer span.End()
|
||||
sblob, err := writer.Finish(finalRowCount)
|
||||
|
|
|
@ -27,7 +27,7 @@ import (
|
|||
"go.opentelemetry.io/otel"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datanode/io"
|
||||
"github.com/milvus-io/milvus/internal/datanode/metacache"
|
||||
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
|
||||
|
@ -45,7 +45,7 @@ import (
|
|||
|
||||
type LevelZeroCompactionTask struct {
|
||||
io.BinlogIO
|
||||
allocator allocator.Allocator
|
||||
allocator allocator.Interface
|
||||
cm storage.ChunkManager
|
||||
|
||||
plan *datapb.CompactionPlan
|
||||
|
@ -63,11 +63,11 @@ var _ Compactor = (*LevelZeroCompactionTask)(nil)
|
|||
func NewLevelZeroCompactionTask(
|
||||
ctx context.Context,
|
||||
binlogIO io.BinlogIO,
|
||||
alloc allocator.Allocator,
|
||||
cm storage.ChunkManager,
|
||||
plan *datapb.CompactionPlan,
|
||||
) *LevelZeroCompactionTask {
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
alloc := allocator.NewLocalAllocator(plan.GetBeginLogID(), math.MaxInt64)
|
||||
return &LevelZeroCompactionTask{
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
|
|
|
@ -48,7 +48,6 @@ type LevelZeroCompactionTaskSuite struct {
|
|||
suite.Suite
|
||||
|
||||
mockBinlogIO *io.MockBinlogIO
|
||||
mockAlloc *allocator.MockAllocator
|
||||
task *LevelZeroCompactionTask
|
||||
|
||||
dData *storage.DeleteData
|
||||
|
@ -57,10 +56,9 @@ type LevelZeroCompactionTaskSuite struct {
|
|||
|
||||
func (s *LevelZeroCompactionTaskSuite) SetupTest() {
|
||||
paramtable.Init()
|
||||
s.mockAlloc = allocator.NewMockAllocator(s.T())
|
||||
s.mockBinlogIO = io.NewMockBinlogIO(s.T())
|
||||
// plan of the task is unset
|
||||
s.task = NewLevelZeroCompactionTask(context.Background(), s.mockBinlogIO, s.mockAlloc, nil, nil)
|
||||
s.task = NewLevelZeroCompactionTask(context.Background(), s.mockBinlogIO, nil, nil)
|
||||
|
||||
pk2ts := map[int64]uint64{
|
||||
1: 20000,
|
||||
|
@ -272,6 +270,7 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactLinear() {
|
|||
},
|
||||
},
|
||||
},
|
||||
BeginLogID: 11111,
|
||||
}
|
||||
|
||||
s.task.plan = plan
|
||||
|
@ -289,7 +288,6 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactLinear() {
|
|||
|
||||
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).Return([][]byte{s.dBlob}, nil).Times(1)
|
||||
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Once()
|
||||
s.mockAlloc.EXPECT().AllocOne().Return(19530, nil).Times(2)
|
||||
|
||||
s.Require().Equal(plan.GetPlanID(), s.task.GetPlanID())
|
||||
s.Require().Equal(plan.GetChannel(), s.task.GetChannelName())
|
||||
|
@ -381,6 +379,7 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactBatch() {
|
|||
},
|
||||
},
|
||||
},
|
||||
BeginLogID: 11111,
|
||||
}
|
||||
|
||||
s.task.plan = plan
|
||||
|
@ -396,7 +395,6 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactBatch() {
|
|||
cm.EXPECT().MultiRead(mock.Anything, mock.Anything).Return([][]byte{sw.GetBuffer()}, nil)
|
||||
s.task.cm = cm
|
||||
|
||||
s.mockAlloc.EXPECT().AllocOne().Return(19530, nil).Times(2)
|
||||
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).Return([][]byte{s.dBlob}, nil).Once()
|
||||
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Once()
|
||||
|
||||
|
@ -443,6 +441,7 @@ func (s *LevelZeroCompactionTaskSuite) TestSerializeUpload() {
|
|||
SegmentID: 100,
|
||||
},
|
||||
},
|
||||
BeginLogID: 11111,
|
||||
}
|
||||
|
||||
s.Run("serializeUpload allocator Alloc failed", func() {
|
||||
|
@ -468,7 +467,6 @@ func (s *LevelZeroCompactionTaskSuite) TestSerializeUpload() {
|
|||
writer := NewSegmentDeltaWriter(100, 10, 1)
|
||||
writer.WriteBatch(s.dData.Pks, s.dData.Tss)
|
||||
writers := map[int64]*SegmentDeltaWriter{100: writer}
|
||||
s.mockAlloc.EXPECT().AllocOne().Return(19530, nil)
|
||||
|
||||
results, err := s.task.serializeUpload(ctx, writers)
|
||||
s.Error(err)
|
||||
|
@ -480,7 +478,6 @@ func (s *LevelZeroCompactionTaskSuite) TestSerializeUpload() {
|
|||
s.task.plan = plan
|
||||
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil)
|
||||
|
||||
s.mockAlloc.EXPECT().AllocOne().Return(19530, nil)
|
||||
writer := NewSegmentDeltaWriter(100, 10, 1)
|
||||
writer.WriteBatch(s.dData.Pks, s.dData.Tss)
|
||||
writers := map[int64]*SegmentDeltaWriter{100: writer}
|
||||
|
|
|
@ -20,6 +20,7 @@ import (
|
|||
"context"
|
||||
"fmt"
|
||||
sio "io"
|
||||
"math"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
|
@ -27,7 +28,7 @@ import (
|
|||
"go.opentelemetry.io/otel"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datanode/io"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
|
@ -42,8 +43,8 @@ import (
|
|||
|
||||
// for MixCompaction only
|
||||
type mixCompactionTask struct {
|
||||
binlogIO io.BinlogIO
|
||||
allocator.Allocator
|
||||
binlogIO io.BinlogIO
|
||||
allocator allocator.Interface
|
||||
currentTs typeutil.Timestamp
|
||||
|
||||
plan *datapb.CompactionPlan
|
||||
|
@ -61,15 +62,15 @@ var _ Compactor = (*mixCompactionTask)(nil)
|
|||
func NewMixCompactionTask(
|
||||
ctx context.Context,
|
||||
binlogIO io.BinlogIO,
|
||||
alloc allocator.Allocator,
|
||||
plan *datapb.CompactionPlan,
|
||||
) *mixCompactionTask {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
alloc := allocator.NewLocalAllocator(plan.GetBeginLogID(), math.MaxInt64)
|
||||
return &mixCompactionTask{
|
||||
ctx: ctx1,
|
||||
cancel: cancel,
|
||||
binlogIO: binlogIO,
|
||||
Allocator: alloc,
|
||||
allocator: alloc,
|
||||
plan: plan,
|
||||
tr: timerecord.NewTimeRecorder("mix compaction"),
|
||||
currentTs: tsoutil.GetCurrentTime(),
|
||||
|
@ -198,7 +199,7 @@ func (t *mixCompactionTask) merge(
|
|||
|
||||
if (unflushedRowCount+1)%100 == 0 && writer.FlushAndIsFull() {
|
||||
serWriteStart := time.Now()
|
||||
kvs, partialBinlogs, err := serializeWrite(ctx, t.Allocator, writer)
|
||||
kvs, partialBinlogs, err := serializeWrite(ctx, t.allocator, writer)
|
||||
if err != nil {
|
||||
log.Warn("compact wrong, failed to serialize writer", zap.Error(err))
|
||||
return nil, err
|
||||
|
@ -220,7 +221,7 @@ func (t *mixCompactionTask) merge(
|
|||
|
||||
if !writer.FlushAndIsEmpty() {
|
||||
serWriteStart := time.Now()
|
||||
kvs, partialBinlogs, err := serializeWrite(ctx, t.Allocator, writer)
|
||||
kvs, partialBinlogs, err := serializeWrite(ctx, t.allocator, writer)
|
||||
if err != nil {
|
||||
log.Warn("compact wrong, failed to serialize writer", zap.Error(err))
|
||||
return nil, err
|
||||
|
@ -239,7 +240,7 @@ func (t *mixCompactionTask) merge(
|
|||
}
|
||||
|
||||
serWriteStart := time.Now()
|
||||
sPath, err := statSerializeWrite(ctx, t.binlogIO, t.Allocator, writer, remainingRowCount)
|
||||
sPath, err := statSerializeWrite(ctx, t.binlogIO, t.allocator, writer, remainingRowCount)
|
||||
if err != nil {
|
||||
log.Warn("compact wrong, failed to serialize write segment stats",
|
||||
zap.Int64("remaining row count", remainingRowCount), zap.Error(err))
|
||||
|
@ -309,12 +310,7 @@ func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
|
|||
|
||||
log.Info("compact start")
|
||||
|
||||
targetSegID, err := t.AllocOne()
|
||||
if err != nil {
|
||||
log.Warn("compact wrong, unable to allocate segmentID", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
targetSegID := t.plan.GetPreAllocatedSegments().GetBegin()
|
||||
previousRowCount := t.getNumRows()
|
||||
|
||||
writer, err := NewSegmentWriter(t.plan.GetSchema(), previousRowCount, targetSegID, partitionID, collectionID)
|
||||
|
|
|
@ -18,6 +18,7 @@ package compaction
|
|||
|
||||
import (
|
||||
"context"
|
||||
"github.com/milvus-io/milvus/internal/allocator"
|
||||
"math"
|
||||
"testing"
|
||||
"time"
|
||||
|
@ -29,7 +30,6 @@ 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/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datanode/io"
|
||||
"github.com/milvus-io/milvus/internal/datanode/metacache"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
|
@ -51,7 +51,6 @@ type MixCompactionTaskSuite struct {
|
|||
suite.Suite
|
||||
|
||||
mockBinlogIO *io.MockBinlogIO
|
||||
mockAlloc *allocator.MockAllocator
|
||||
|
||||
meta *etcdpb.CollectionMeta
|
||||
segWriter *SegmentWriter
|
||||
|
@ -66,9 +65,6 @@ func (s *MixCompactionTaskSuite) SetupSuite() {
|
|||
|
||||
func (s *MixCompactionTaskSuite) SetupTest() {
|
||||
s.mockBinlogIO = io.NewMockBinlogIO(s.T())
|
||||
s.mockAlloc = allocator.NewMockAllocator(s.T())
|
||||
|
||||
s.task = NewMixCompactionTask(context.Background(), s.mockBinlogIO, s.mockAlloc, nil)
|
||||
|
||||
s.meta = genTestCollectionMeta()
|
||||
|
||||
|
@ -82,10 +78,14 @@ func (s *MixCompactionTaskSuite) SetupTest() {
|
|||
Field2StatslogPaths: nil,
|
||||
Deltalogs: nil,
|
||||
}},
|
||||
TimeoutInSeconds: 10,
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
Schema: s.meta.GetSchema(),
|
||||
TimeoutInSeconds: 10,
|
||||
Type: datapb.CompactionType_MixCompaction,
|
||||
Schema: s.meta.GetSchema(),
|
||||
BeginLogID: 19530,
|
||||
PreAllocatedSegments: &datapb.IDRange{Begin: 19530},
|
||||
}
|
||||
|
||||
s.task = NewMixCompactionTask(context.Background(), s.mockBinlogIO, s.plan)
|
||||
s.task.plan = s.plan
|
||||
}
|
||||
|
||||
|
@ -104,7 +104,6 @@ func getMilvusBirthday() time.Time {
|
|||
func (s *MixCompactionTaskSuite) TestCompactDupPK() {
|
||||
// Test merge compactions, two segments with the same pk, one deletion pk=1
|
||||
// The merged segment 19530 should remain 3 pk without pk=100
|
||||
s.mockAlloc.EXPECT().AllocOne().Return(int64(19530), nil).Twice()
|
||||
segments := []int64{7, 8, 9}
|
||||
dblobs, err := getInt64DeltaBlobs(
|
||||
1,
|
||||
|
@ -115,8 +114,8 @@ func (s *MixCompactionTaskSuite) TestCompactDupPK() {
|
|||
|
||||
s.mockBinlogIO.EXPECT().Download(mock.Anything, []string{"1"}).
|
||||
Return([][]byte{dblobs.GetValue()}, nil).Times(3)
|
||||
s.mockAlloc.EXPECT().Alloc(mock.Anything).Return(7777777, 8888888, nil)
|
||||
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil)
|
||||
alloc := allocator.NewLocalAllocator(7777777, math.MaxInt64)
|
||||
|
||||
// clear origial segments
|
||||
s.task.plan.SegmentBinlogs = make([]*datapb.CompactionSegmentBinlogs, 0)
|
||||
|
@ -139,7 +138,7 @@ func (s *MixCompactionTaskSuite) TestCompactDupPK() {
|
|||
//}
|
||||
//bfs := metacache.NewBloomFilterSet(statistic)
|
||||
|
||||
kvs, fBinlogs, err := serializeWrite(context.TODO(), s.task.Allocator, s.segWriter)
|
||||
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))
|
||||
|
@ -177,10 +176,8 @@ func (s *MixCompactionTaskSuite) TestCompactDupPK() {
|
|||
}
|
||||
|
||||
func (s *MixCompactionTaskSuite) TestCompactTwoToOne() {
|
||||
s.mockAlloc.EXPECT().AllocOne().Return(int64(19530), nil).Twice()
|
||||
|
||||
segments := []int64{5, 6, 7}
|
||||
s.mockAlloc.EXPECT().Alloc(mock.Anything).Return(7777777, 8888888, nil)
|
||||
alloc := allocator.NewLocalAllocator(7777777, math.MaxInt64)
|
||||
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil)
|
||||
s.task.plan.SegmentBinlogs = make([]*datapb.CompactionSegmentBinlogs, 0)
|
||||
for _, segID := range segments {
|
||||
|
@ -191,7 +188,7 @@ func (s *MixCompactionTaskSuite) TestCompactTwoToOne() {
|
|||
// MaxPK: s.segWriter.pkstats.MaxPk,
|
||||
//}
|
||||
//bfs := metacache.NewBloomFilterSet(statistic)
|
||||
kvs, fBinlogs, err := serializeWrite(context.TODO(), s.task.Allocator, s.segWriter)
|
||||
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))
|
||||
|
@ -251,11 +248,10 @@ func (s *MixCompactionTaskSuite) TestMergeBufferFull() {
|
|||
err := s.segWriter.Write(&v)
|
||||
s.Require().NoError(err)
|
||||
|
||||
s.mockAlloc.EXPECT().Alloc(mock.Anything).Return(888888, 999999, nil).Times(2)
|
||||
kvs, _, err := serializeWrite(context.TODO(), s.task.Allocator, s.segWriter)
|
||||
alloc := allocator.NewLocalAllocator(888888, math.MaxInt64)
|
||||
kvs, _, err := serializeWrite(context.TODO(), alloc, s.segWriter)
|
||||
s.Require().NoError(err)
|
||||
|
||||
s.mockAlloc.EXPECT().AllocOne().Return(888888, nil)
|
||||
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).RunAndReturn(
|
||||
func(ctx context.Context, paths []string) ([][]byte, error) {
|
||||
s.Require().Equal(len(paths), len(kvs))
|
||||
|
@ -279,11 +275,10 @@ func (s *MixCompactionTaskSuite) TestMergeEntityExpired() {
|
|||
currTs := tsoutil.ComposeTSByTime(getMilvusBirthday().Add(time.Second*(time.Duration(collTTL)+1)), 0)
|
||||
s.task.currentTs = currTs
|
||||
s.task.plan.CollectionTtl = int64(collTTL)
|
||||
s.mockAlloc.EXPECT().Alloc(mock.Anything).Return(888888, 999999, nil)
|
||||
alloc := allocator.NewLocalAllocator(888888, math.MaxInt64)
|
||||
|
||||
kvs, _, err := serializeWrite(context.TODO(), s.task.Allocator, s.segWriter)
|
||||
kvs, _, err := serializeWrite(context.TODO(), alloc, s.segWriter)
|
||||
s.Require().NoError(err)
|
||||
s.mockAlloc.EXPECT().AllocOne().Return(888888, nil)
|
||||
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).RunAndReturn(
|
||||
func(ctx context.Context, paths []string) ([][]byte, error) {
|
||||
s.Require().Equal(len(paths), len(kvs))
|
||||
|
@ -313,15 +308,11 @@ func (s *MixCompactionTaskSuite) TestMergeNoExpiration() {
|
|||
{"deleted pk=4", map[interface{}]uint64{int64(4): deleteTs}, 0},
|
||||
}
|
||||
|
||||
s.mockAlloc.EXPECT().Alloc(mock.Anything).Return(888888, 999999, nil)
|
||||
kvs, _, err := serializeWrite(context.TODO(), s.task.Allocator, s.segWriter)
|
||||
alloc := allocator.NewLocalAllocator(888888, math.MaxInt64)
|
||||
kvs, _, err := serializeWrite(context.TODO(), alloc, s.segWriter)
|
||||
s.Require().NoError(err)
|
||||
for _, test := range tests {
|
||||
s.Run(test.description, func() {
|
||||
if test.expectedRowCount > 0 {
|
||||
s.mockAlloc.EXPECT().Alloc(mock.Anything).Return(77777, 99999, nil).Once()
|
||||
}
|
||||
s.mockAlloc.EXPECT().AllocOne().Return(888888, nil)
|
||||
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).RunAndReturn(
|
||||
func(ctx context.Context, paths []string) ([][]byte, error) {
|
||||
s.Require().Equal(len(paths), len(kvs))
|
||||
|
@ -489,12 +480,6 @@ func (s *MixCompactionTaskSuite) TestCompactFail() {
|
|||
_, err := s.task.Compact()
|
||||
s.Error(err)
|
||||
})
|
||||
|
||||
s.Run("Test compact AllocOnce failed", func() {
|
||||
s.mockAlloc.EXPECT().AllocOne().Return(0, errors.New("mock alloc one error")).Once()
|
||||
_, err := s.task.Compact()
|
||||
s.Error(err)
|
||||
})
|
||||
}
|
||||
|
||||
func (s *MixCompactionTaskSuite) TestIsExpiredEntity() {
|
||||
|
|
|
@ -265,7 +265,7 @@ func (s *SchedulerSuite) TestScheduler_Start_Import() {
|
|||
},
|
||||
},
|
||||
Ts: 1000,
|
||||
AutoIDRange: &datapb.AutoIDRange{
|
||||
IDRange: &datapb.IDRange{
|
||||
Begin: 0,
|
||||
End: int64(s.numRows),
|
||||
},
|
||||
|
@ -326,7 +326,7 @@ func (s *SchedulerSuite) TestScheduler_Start_Import_Failed() {
|
|||
},
|
||||
},
|
||||
Ts: 1000,
|
||||
AutoIDRange: &datapb.AutoIDRange{
|
||||
IDRange: &datapb.IDRange{
|
||||
Begin: 0,
|
||||
End: int64(s.numRows),
|
||||
},
|
||||
|
@ -417,7 +417,7 @@ func (s *SchedulerSuite) TestScheduler_ImportFile() {
|
|||
},
|
||||
},
|
||||
Ts: 1000,
|
||||
AutoIDRange: &datapb.AutoIDRange{
|
||||
IDRange: &datapb.IDRange{
|
||||
Begin: 0,
|
||||
End: int64(s.numRows),
|
||||
},
|
||||
|
|
|
@ -66,7 +66,7 @@ func NewImportTask(req *datapb.ImportRequest,
|
|||
UnsetAutoID(req.GetSchema())
|
||||
}
|
||||
// Setting end as math.MaxInt64 to incrementally allocate logID.
|
||||
alloc := allocator.NewLocalAllocator(req.GetAutoIDRange().GetBegin(), math.MaxInt64)
|
||||
alloc := allocator.NewLocalAllocator(req.GetIDRange().GetBegin(), math.MaxInt64)
|
||||
task := &ImportTask{
|
||||
ImportTaskV2: &datapb.ImportTaskV2{
|
||||
JobID: req.GetJobID(),
|
||||
|
|
|
@ -62,7 +62,7 @@ func NewL0ImportTask(req *datapb.ImportRequest,
|
|||
) Task {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
// Setting end as math.MaxInt64 to incrementally allocate logID.
|
||||
alloc := allocator.NewLocalAllocator(req.GetAutoIDRange().GetBegin(), math.MaxInt64)
|
||||
alloc := allocator.NewLocalAllocator(req.GetIDRange().GetBegin(), math.MaxInt64)
|
||||
task := &L0ImportTask{
|
||||
ImportTaskV2: &datapb.ImportTaskV2{
|
||||
JobID: req.GetJobID(),
|
||||
|
|
|
@ -165,7 +165,7 @@ func (s *L0ImportSuite) TestL0Import() {
|
|||
Vchannel: s.channel,
|
||||
},
|
||||
},
|
||||
AutoIDRange: &datapb.AutoIDRange{
|
||||
IDRange: &datapb.IDRange{
|
||||
Begin: 0,
|
||||
End: int64(s.delCnt),
|
||||
},
|
||||
|
|
|
@ -213,6 +213,10 @@ func (node *DataNode) CompactionV2(ctx context.Context, req *datapb.CompactionPl
|
|||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
if req.GetBeginLogID() == 0 {
|
||||
return merr.Status(merr.WrapErrParameterInvalidMsg("invalid beginLogID")), nil
|
||||
}
|
||||
|
||||
/*
|
||||
spanCtx := trace.SpanContextFromContext(ctx)
|
||||
|
||||
|
@ -226,22 +230,25 @@ func (node *DataNode) CompactionV2(ctx context.Context, req *datapb.CompactionPl
|
|||
task = compaction.NewLevelZeroCompactionTask(
|
||||
taskCtx,
|
||||
binlogIO,
|
||||
node.allocator,
|
||||
node.chunkManager,
|
||||
req,
|
||||
)
|
||||
case datapb.CompactionType_MixCompaction:
|
||||
if req.GetPreAllocatedSegments() == nil || req.GetPreAllocatedSegments().GetBegin() == 0 {
|
||||
return merr.Status(merr.WrapErrParameterInvalidMsg("invalid pre-allocated segmentID range")), nil
|
||||
}
|
||||
task = compaction.NewMixCompactionTask(
|
||||
taskCtx,
|
||||
binlogIO,
|
||||
node.allocator,
|
||||
req,
|
||||
)
|
||||
case datapb.CompactionType_ClusteringCompaction:
|
||||
if req.GetPreAllocatedSegments() == nil || req.GetPreAllocatedSegments().GetBegin() == 0 {
|
||||
return merr.Status(merr.WrapErrParameterInvalidMsg("invalid pre-allocated segmentID range")), nil
|
||||
}
|
||||
task = compaction.NewClusteringCompactionTask(
|
||||
taskCtx,
|
||||
binlogIO,
|
||||
node.allocator,
|
||||
req,
|
||||
)
|
||||
default:
|
||||
|
|
|
@ -231,6 +231,7 @@ func (s *DataNodeServicesSuite) TestCompaction() {
|
|||
resp, err := node.CompactionV2(ctx, req)
|
||||
s.NoError(err)
|
||||
s.False(merr.Ok(resp))
|
||||
s.T().Logf("status=%v", resp)
|
||||
})
|
||||
|
||||
s.Run("unknown CompactionType", func() {
|
||||
|
@ -245,11 +246,13 @@ func (s *DataNodeServicesSuite) TestCompaction() {
|
|||
{SegmentID: 102, Level: datapb.SegmentLevel_L0},
|
||||
{SegmentID: 103, Level: datapb.SegmentLevel_L1},
|
||||
},
|
||||
BeginLogID: 100,
|
||||
}
|
||||
|
||||
resp, err := node.CompactionV2(ctx, req)
|
||||
s.NoError(err)
|
||||
s.False(merr.Ok(resp))
|
||||
s.T().Logf("status=%v", resp)
|
||||
})
|
||||
|
||||
s.Run("compact_clustering", func() {
|
||||
|
@ -264,11 +267,60 @@ func (s *DataNodeServicesSuite) TestCompaction() {
|
|||
{SegmentID: 102, Level: datapb.SegmentLevel_L0},
|
||||
{SegmentID: 103, Level: datapb.SegmentLevel_L1},
|
||||
},
|
||||
Type: datapb.CompactionType_ClusteringCompaction,
|
||||
Type: datapb.CompactionType_ClusteringCompaction,
|
||||
BeginLogID: 100,
|
||||
PreAllocatedSegments: &datapb.IDRange{Begin: 100, End: 200},
|
||||
}
|
||||
|
||||
_, err := node.CompactionV2(ctx, req)
|
||||
resp, err := node.CompactionV2(ctx, req)
|
||||
s.NoError(err)
|
||||
s.True(merr.Ok(resp))
|
||||
s.T().Logf("status=%v", resp)
|
||||
})
|
||||
|
||||
s.Run("beginLogID is invalid", func() {
|
||||
node := s.node
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
req := &datapb.CompactionPlan{
|
||||
PlanID: 1000,
|
||||
Channel: dmChannelName,
|
||||
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
|
||||
{SegmentID: 102, Level: datapb.SegmentLevel_L0},
|
||||
{SegmentID: 103, Level: datapb.SegmentLevel_L1},
|
||||
},
|
||||
Type: datapb.CompactionType_ClusteringCompaction,
|
||||
BeginLogID: 0,
|
||||
}
|
||||
|
||||
resp, err := node.CompactionV2(ctx, req)
|
||||
s.NoError(err)
|
||||
s.False(merr.Ok(resp))
|
||||
s.T().Logf("status=%v", resp)
|
||||
})
|
||||
|
||||
s.Run("pre-allocated segmentID range is invalid", func() {
|
||||
node := s.node
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
req := &datapb.CompactionPlan{
|
||||
PlanID: 1000,
|
||||
Channel: dmChannelName,
|
||||
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
|
||||
{SegmentID: 102, Level: datapb.SegmentLevel_L0},
|
||||
{SegmentID: 103, Level: datapb.SegmentLevel_L1},
|
||||
},
|
||||
Type: datapb.CompactionType_ClusteringCompaction,
|
||||
BeginLogID: 100,
|
||||
PreAllocatedSegments: &datapb.IDRange{Begin: 0, End: 0},
|
||||
}
|
||||
|
||||
resp, err := node.CompactionV2(ctx, req)
|
||||
s.NoError(err)
|
||||
s.False(merr.Ok(resp))
|
||||
s.T().Logf("status=%v", resp)
|
||||
})
|
||||
}
|
||||
|
||||
|
|
|
@ -560,6 +560,8 @@ message CompactionPlan {
|
|||
string analyze_result_path = 14;
|
||||
repeated int64 analyze_segment_ids = 15;
|
||||
int32 state = 16;
|
||||
int64 begin_logID = 17;
|
||||
IDRange pre_allocated_segments = 18; // only for clustering compaction
|
||||
}
|
||||
|
||||
message CompactionSegment {
|
||||
|
@ -723,7 +725,7 @@ message PreImportRequest {
|
|||
repeated common.KeyValuePair options = 9;
|
||||
}
|
||||
|
||||
message autoIDRange {
|
||||
message IDRange {
|
||||
int64 begin = 1;
|
||||
int64 end = 2;
|
||||
}
|
||||
|
@ -745,7 +747,7 @@ message ImportRequest {
|
|||
repeated internal.ImportFile files = 8;
|
||||
repeated common.KeyValuePair options = 9;
|
||||
uint64 ts = 10;
|
||||
autoIDRange autoID_range = 11;
|
||||
IDRange ID_range = 11;
|
||||
repeated ImportRequestSegment request_segments = 12;
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue