2021-11-08 19:49:07 +08:00
|
|
|
// Licensed to the LF AI & Data foundation under one
|
|
|
|
// or more contributor license agreements. See the NOTICE file
|
|
|
|
// distributed with this work for additional information
|
|
|
|
// regarding copyright ownership. The ASF licenses this file
|
|
|
|
// to you under the Apache License, Version 2.0 (the
|
|
|
|
// "License"); you may not use this file except in compliance
|
|
|
|
// with the License. You may obtain a copy of the License at
|
|
|
|
//
|
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
//
|
|
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
// See the License for the specific language governing permissions and
|
|
|
|
// limitations under the License.
|
|
|
|
|
|
|
|
package datanode
|
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
|
|
|
"fmt"
|
2024-03-08 18:25:02 +08:00
|
|
|
sio "io"
|
2024-02-18 14:08:49 +08:00
|
|
|
"sync"
|
2021-11-08 19:49:07 +08:00
|
|
|
"time"
|
|
|
|
|
2023-02-26 11:31:49 +08:00
|
|
|
"github.com/cockroachdb/errors"
|
2024-02-01 14:25:04 +08:00
|
|
|
"github.com/samber/lo"
|
2024-01-23 10:37:00 +08:00
|
|
|
"go.opentelemetry.io/otel"
|
2023-09-21 09:45:27 +08:00
|
|
|
"go.uber.org/zap"
|
|
|
|
|
2023-11-14 15:56:19 +08:00
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
2023-06-09 01:28:37 +08:00
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
2023-03-23 19:43:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
2024-02-20 14:38:51 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/datanode/io"
|
2023-11-15 15:24:18 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/datanode/metacache"
|
|
|
|
"github.com/milvus-io/milvus/internal/datanode/syncmgr"
|
2024-01-18 22:06:31 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
|
2021-11-08 19:49:07 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/etcdpb"
|
2021-12-20 14:38:25 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/storage"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/log"
|
|
|
|
"github.com/milvus-io/milvus/pkg/metrics"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/funcutil"
|
2023-09-19 18:05:22 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/merr"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
2023-09-19 18:05:22 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/tsoutil"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
2021-11-08 19:49:07 +08:00
|
|
|
)
|
|
|
|
|
|
|
|
var (
|
2024-02-01 14:25:04 +08:00
|
|
|
errIllegalCompactionPlan = errors.New("compaction plan illegal")
|
|
|
|
errTransferType = errors.New("transfer intferface to type wrong")
|
|
|
|
errUnknownDataType = errors.New("unknown shema DataType")
|
|
|
|
errContext = errors.New("context done or timeout")
|
2021-11-08 19:49:07 +08:00
|
|
|
)
|
|
|
|
|
|
|
|
type iterator = storage.Iterator
|
|
|
|
|
|
|
|
type compactor interface {
|
2022-01-18 17:49:39 +08:00
|
|
|
complete()
|
2023-11-14 15:56:19 +08:00
|
|
|
compact() (*datapb.CompactionPlanResult, error)
|
2023-11-29 10:50:29 +08:00
|
|
|
injectDone()
|
2021-11-11 20:56:49 +08:00
|
|
|
stop()
|
2021-11-08 19:49:07 +08:00
|
|
|
getPlanID() UniqueID
|
2021-11-11 20:56:49 +08:00
|
|
|
getCollection() UniqueID
|
2021-12-02 16:39:33 +08:00
|
|
|
getChannelName() string
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
2021-11-10 19:16:40 +08:00
|
|
|
// make sure compactionTask implements compactor interface
|
|
|
|
var _ compactor = (*compactionTask)(nil)
|
|
|
|
|
2024-02-01 14:25:04 +08:00
|
|
|
// for MixCompaction only
|
2021-11-08 19:49:07 +08:00
|
|
|
type compactionTask struct {
|
2024-02-20 14:38:51 +08:00
|
|
|
binlogIO io.BinlogIO
|
2021-11-08 19:49:07 +08:00
|
|
|
compactor
|
2023-11-15 15:24:18 +08:00
|
|
|
metaCache metacache.MetaCache
|
|
|
|
syncMgr syncmgr.SyncManager
|
2023-03-23 19:43:57 +08:00
|
|
|
allocator.Allocator
|
2021-11-08 19:49:07 +08:00
|
|
|
|
|
|
|
plan *datapb.CompactionPlan
|
2021-11-11 20:56:49 +08:00
|
|
|
|
|
|
|
ctx context.Context
|
|
|
|
cancel context.CancelFunc
|
2021-12-02 16:39:33 +08:00
|
|
|
|
2024-02-18 14:08:49 +08:00
|
|
|
injectDoneOnce sync.Once
|
|
|
|
done chan struct{}
|
|
|
|
tr *timerecord.TimeRecorder
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func newCompactionTask(
|
2021-11-11 20:56:49 +08:00
|
|
|
ctx context.Context,
|
2024-02-20 14:38:51 +08:00
|
|
|
binlogIO io.BinlogIO,
|
2023-11-15 15:24:18 +08:00
|
|
|
metaCache metacache.MetaCache,
|
|
|
|
syncMgr syncmgr.SyncManager,
|
2023-03-23 19:43:57 +08:00
|
|
|
alloc allocator.Allocator,
|
2022-11-17 20:37:10 +08:00
|
|
|
plan *datapb.CompactionPlan,
|
2023-09-21 09:45:27 +08:00
|
|
|
) *compactionTask {
|
2021-11-11 20:56:49 +08:00
|
|
|
ctx1, cancel := context.WithCancel(ctx)
|
2021-11-08 19:49:07 +08:00
|
|
|
return &compactionTask{
|
2024-02-20 14:38:51 +08:00
|
|
|
ctx: ctx1,
|
|
|
|
cancel: cancel,
|
|
|
|
binlogIO: binlogIO,
|
|
|
|
syncMgr: syncMgr,
|
|
|
|
metaCache: metaCache,
|
|
|
|
Allocator: alloc,
|
|
|
|
plan: plan,
|
|
|
|
tr: timerecord.NewTimeRecorder("levelone compaction"),
|
|
|
|
done: make(chan struct{}, 1),
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-01-18 17:49:39 +08:00
|
|
|
func (t *compactionTask) complete() {
|
2023-02-21 16:10:26 +08:00
|
|
|
t.done <- struct{}{}
|
2022-01-18 17:49:39 +08:00
|
|
|
}
|
|
|
|
|
2021-11-11 20:56:49 +08:00
|
|
|
func (t *compactionTask) stop() {
|
|
|
|
t.cancel()
|
2023-02-21 16:10:26 +08:00
|
|
|
<-t.done
|
2023-11-29 10:50:29 +08:00
|
|
|
t.injectDone()
|
2021-11-11 20:56:49 +08:00
|
|
|
}
|
|
|
|
|
2021-11-08 19:49:07 +08:00
|
|
|
func (t *compactionTask) getPlanID() UniqueID {
|
|
|
|
return t.plan.GetPlanID()
|
|
|
|
}
|
|
|
|
|
2021-12-02 16:39:33 +08:00
|
|
|
func (t *compactionTask) getChannelName() string {
|
|
|
|
return t.plan.GetChannel()
|
|
|
|
}
|
|
|
|
|
2023-05-29 10:21:28 +08:00
|
|
|
// return num rows of all segment compaction from
|
|
|
|
func (t *compactionTask) getNumRows() (int64, error) {
|
|
|
|
numRows := int64(0)
|
|
|
|
for _, binlog := range t.plan.SegmentBinlogs {
|
2023-11-16 00:22:20 +08:00
|
|
|
seg, ok := t.metaCache.GetSegmentByID(binlog.GetSegmentID())
|
2023-11-15 15:24:18 +08:00
|
|
|
if !ok {
|
2023-05-29 10:21:28 +08:00
|
|
|
return 0, merr.WrapErrSegmentNotFound(binlog.GetSegmentID(), "get compaction segments num rows failed")
|
|
|
|
}
|
2023-11-15 15:24:18 +08:00
|
|
|
|
|
|
|
numRows += seg.NumOfRows()
|
2023-05-29 10:21:28 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
return numRows, nil
|
|
|
|
}
|
|
|
|
|
2023-09-04 17:41:48 +08:00
|
|
|
func (t *compactionTask) mergeDeltalogs(dBlobs map[UniqueID][]*Blob) (map[interface{}]Timestamp, error) {
|
2022-09-02 14:31:06 +08:00
|
|
|
log := log.With(zap.Int64("planID", t.getPlanID()))
|
2022-02-08 14:11:45 +08:00
|
|
|
mergeStart := time.Now()
|
2021-11-08 19:49:07 +08:00
|
|
|
dCodec := storage.NewDeleteCodec()
|
|
|
|
|
2023-09-21 09:45:27 +08:00
|
|
|
pk2ts := make(map[interface{}]Timestamp)
|
2021-11-08 19:49:07 +08:00
|
|
|
|
|
|
|
for _, blobs := range dBlobs {
|
|
|
|
_, _, dData, err := dCodec.Deserialize(blobs)
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("merge deltalogs wrong", zap.Error(err))
|
2023-09-04 17:41:48 +08:00
|
|
|
return nil, err
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
2021-11-09 15:01:17 +08:00
|
|
|
for i := int64(0); i < dData.RowCount; i++ {
|
|
|
|
pk := dData.Pks[i]
|
|
|
|
ts := dData.Tss[i]
|
2024-01-14 10:26:52 +08:00
|
|
|
if lastTS, ok := pk2ts[pk.GetValue()]; ok && lastTS > ts {
|
|
|
|
ts = lastTS
|
|
|
|
}
|
2023-09-04 17:41:48 +08:00
|
|
|
pk2ts[pk.GetValue()] = ts
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-10-25 19:31:30 +08:00
|
|
|
log.Info("mergeDeltalogs end",
|
2023-06-19 14:18:41 +08:00
|
|
|
zap.Int("number of deleted pks to compact in insert logs", len(pk2ts)),
|
2023-08-31 17:35:03 +08:00
|
|
|
zap.Duration("elapse", time.Since(mergeStart)))
|
2021-11-08 19:49:07 +08:00
|
|
|
|
2023-09-04 17:41:48 +08:00
|
|
|
return pk2ts, nil
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
2023-05-29 10:21:28 +08:00
|
|
|
func (t *compactionTask) uploadRemainLog(
|
2022-09-25 20:12:52 +08:00
|
|
|
ctxTimeout context.Context,
|
|
|
|
targetSegID UniqueID,
|
|
|
|
partID UniqueID,
|
|
|
|
meta *etcdpb.CollectionMeta,
|
2023-05-29 10:21:28 +08:00
|
|
|
stats *storage.PrimaryKeyStats,
|
|
|
|
totRows int64,
|
2024-01-13 01:32:52 +08:00
|
|
|
writeBuffer *storage.InsertData,
|
2023-09-21 09:45:27 +08:00
|
|
|
) (map[UniqueID]*datapb.FieldBinlog, map[UniqueID]*datapb.FieldBinlog, error) {
|
2024-02-20 14:38:51 +08:00
|
|
|
iCodec := storage.NewInsertCodecWithSchema(meta)
|
|
|
|
inPaths := make(map[int64]*datapb.FieldBinlog, 0)
|
|
|
|
var err error
|
|
|
|
if !writeBuffer.IsEmpty() {
|
|
|
|
inPaths, err = uploadInsertLog(ctxTimeout, t.binlogIO, t.Allocator, meta.GetID(), partID, targetSegID, writeBuffer, iCodec)
|
|
|
|
if err != nil {
|
|
|
|
return nil, nil, err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
statPaths, err := uploadStatsLog(ctxTimeout, t.binlogIO, t.Allocator, meta.GetID(), partID, targetSegID, stats, totRows, iCodec)
|
2023-05-29 10:21:28 +08:00
|
|
|
if err != nil {
|
|
|
|
return nil, nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
return inPaths, statPaths, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (t *compactionTask) uploadSingleInsertLog(
|
|
|
|
ctxTimeout context.Context,
|
|
|
|
targetSegID UniqueID,
|
|
|
|
partID UniqueID,
|
|
|
|
meta *etcdpb.CollectionMeta,
|
2024-01-13 01:32:52 +08:00
|
|
|
writeBuffer *storage.InsertData,
|
2023-09-21 09:45:27 +08:00
|
|
|
) (map[UniqueID]*datapb.FieldBinlog, error) {
|
2024-02-20 14:38:51 +08:00
|
|
|
iCodec := storage.NewInsertCodecWithSchema(meta)
|
|
|
|
|
|
|
|
inPaths, err := uploadInsertLog(ctxTimeout, t.binlogIO, t.Allocator, meta.GetID(), partID, targetSegID, writeBuffer, iCodec)
|
2022-09-25 20:12:52 +08:00
|
|
|
if err != nil {
|
2023-05-29 10:21:28 +08:00
|
|
|
return nil, err
|
2022-09-25 20:12:52 +08:00
|
|
|
}
|
|
|
|
|
2023-05-29 10:21:28 +08:00
|
|
|
return inPaths, nil
|
2022-09-25 20:12:52 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (t *compactionTask) merge(
|
2024-01-23 10:37:00 +08:00
|
|
|
ctx context.Context,
|
2022-09-25 20:12:52 +08:00
|
|
|
unMergedInsertlogs [][]string,
|
|
|
|
targetSegID UniqueID,
|
|
|
|
partID UniqueID,
|
|
|
|
meta *etcdpb.CollectionMeta,
|
2023-09-21 09:45:27 +08:00
|
|
|
delta map[interface{}]Timestamp,
|
|
|
|
) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, int64, error) {
|
2024-01-23 10:37:00 +08:00
|
|
|
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, fmt.Sprintf("CompactMerge-%d", t.getPlanID()))
|
|
|
|
defer span.End()
|
2022-09-02 14:31:06 +08:00
|
|
|
log := log.With(zap.Int64("planID", t.getPlanID()))
|
2022-02-08 14:11:45 +08:00
|
|
|
mergeStart := time.Now()
|
2021-11-08 19:49:07 +08:00
|
|
|
|
|
|
|
var (
|
2024-01-13 01:32:52 +08:00
|
|
|
numBinlogs int // binlog number
|
|
|
|
numRows int64 // the number of rows uploaded
|
|
|
|
expired int64 // the number of expired entity
|
2021-11-08 19:49:07 +08:00
|
|
|
|
2022-09-25 20:12:52 +08:00
|
|
|
insertField2Path = make(map[UniqueID]*datapb.FieldBinlog)
|
|
|
|
insertPaths = make([]*datapb.FieldBinlog, 0)
|
2021-11-08 19:49:07 +08:00
|
|
|
|
2022-10-31 17:41:34 +08:00
|
|
|
statField2Path = make(map[UniqueID]*datapb.FieldBinlog)
|
|
|
|
statPaths = make([]*datapb.FieldBinlog, 0)
|
|
|
|
)
|
2024-01-13 01:32:52 +08:00
|
|
|
writeBuffer, err := storage.NewInsertData(meta.GetSchema())
|
|
|
|
if err != nil {
|
|
|
|
return nil, nil, -1, err
|
|
|
|
}
|
2022-09-27 16:02:53 +08:00
|
|
|
|
2022-04-02 17:43:29 +08:00
|
|
|
isDeletedValue := func(v *storage.Value) bool {
|
2022-07-28 14:52:31 +08:00
|
|
|
ts, ok := delta[v.PK.GetValue()]
|
2023-03-14 19:11:54 +08:00
|
|
|
// insert task and delete task has the same ts when upsert
|
|
|
|
// here should be < instead of <=
|
|
|
|
// to avoid the upsert data to be deleted after compact
|
|
|
|
if ok && uint64(v.Timestamp) < ts {
|
2022-07-28 14:52:31 +08:00
|
|
|
return true
|
2022-04-02 17:43:29 +08:00
|
|
|
}
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
2023-08-08 21:17:21 +08:00
|
|
|
addInsertFieldPath := func(inPaths map[UniqueID]*datapb.FieldBinlog, timestampFrom, timestampTo int64) {
|
2022-09-25 20:12:52 +08:00
|
|
|
for fID, path := range inPaths {
|
2023-08-08 21:17:21 +08:00
|
|
|
for _, binlog := range path.GetBinlogs() {
|
|
|
|
binlog.TimestampTo = uint64(timestampTo)
|
|
|
|
binlog.TimestampFrom = uint64(timestampFrom)
|
|
|
|
}
|
2022-09-25 20:12:52 +08:00
|
|
|
tmpBinlog, ok := insertField2Path[fID]
|
|
|
|
if !ok {
|
|
|
|
tmpBinlog = path
|
|
|
|
} else {
|
|
|
|
tmpBinlog.Binlogs = append(tmpBinlog.Binlogs, path.GetBinlogs()...)
|
|
|
|
}
|
|
|
|
insertField2Path[fID] = tmpBinlog
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-10-31 17:41:34 +08:00
|
|
|
addStatFieldPath := func(statPaths map[UniqueID]*datapb.FieldBinlog) {
|
|
|
|
for fID, path := range statPaths {
|
|
|
|
tmpBinlog, ok := statField2Path[fID]
|
|
|
|
if !ok {
|
|
|
|
tmpBinlog = path
|
|
|
|
} else {
|
|
|
|
tmpBinlog.Binlogs = append(tmpBinlog.Binlogs, path.GetBinlogs()...)
|
|
|
|
}
|
|
|
|
statField2Path[fID] = tmpBinlog
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-09-25 20:12:52 +08:00
|
|
|
// get pkID, pkType, dim
|
2023-05-29 10:21:28 +08:00
|
|
|
var pkField *schemapb.FieldSchema
|
2022-09-25 20:12:52 +08:00
|
|
|
for _, fs := range meta.GetSchema().GetFields() {
|
|
|
|
if fs.GetIsPrimaryKey() && fs.GetFieldID() >= 100 && typeutil.IsPrimaryFieldType(fs.GetDataType()) {
|
2023-05-29 10:21:28 +08:00
|
|
|
pkField = fs
|
2022-09-02 10:34:59 +08:00
|
|
|
}
|
2022-12-12 10:33:26 +08:00
|
|
|
}
|
|
|
|
|
2023-05-29 10:21:28 +08:00
|
|
|
if pkField == nil {
|
2023-06-21 14:00:42 +08:00
|
|
|
log.Warn("failed to get pk field from schema")
|
2023-05-29 10:21:28 +08:00
|
|
|
return nil, nil, 0, fmt.Errorf("no pk field in schema")
|
|
|
|
}
|
|
|
|
|
|
|
|
pkID := pkField.GetFieldID()
|
|
|
|
pkType := pkField.GetDataType()
|
|
|
|
|
2022-02-07 22:45:46 +08:00
|
|
|
expired = 0
|
2022-09-25 20:12:52 +08:00
|
|
|
numRows = 0
|
|
|
|
numBinlogs = 0
|
|
|
|
currentTs := t.GetCurrentTime()
|
|
|
|
currentRows := 0
|
|
|
|
downloadTimeCost := time.Duration(0)
|
|
|
|
uploadInsertTimeCost := time.Duration(0)
|
2022-10-31 17:41:34 +08:00
|
|
|
|
2023-05-29 10:21:28 +08:00
|
|
|
oldRowNums, err := t.getNumRows()
|
|
|
|
if err != nil {
|
|
|
|
return nil, nil, 0, err
|
|
|
|
}
|
|
|
|
|
2023-11-21 10:28:21 +08:00
|
|
|
stats, err := storage.NewPrimaryKeyStats(pkID, int64(pkType), oldRowNums)
|
|
|
|
if err != nil {
|
|
|
|
return nil, nil, 0, err
|
|
|
|
}
|
2023-08-08 21:17:21 +08:00
|
|
|
// initial timestampFrom, timestampTo = -1, -1 is an illegal value, only to mark initial state
|
|
|
|
var (
|
|
|
|
timestampTo int64 = -1
|
|
|
|
timestampFrom int64 = -1
|
|
|
|
)
|
2023-05-29 10:21:28 +08:00
|
|
|
|
2022-09-25 20:12:52 +08:00
|
|
|
for _, path := range unMergedInsertlogs {
|
|
|
|
downloadStart := time.Now()
|
2024-02-20 14:38:51 +08:00
|
|
|
data, err := downloadBlobs(ctx, t.binlogIO, path)
|
2022-09-25 20:12:52 +08:00
|
|
|
if err != nil {
|
2023-06-19 14:18:41 +08:00
|
|
|
log.Warn("download insertlogs wrong", zap.Strings("path", path), zap.Error(err))
|
2022-10-31 17:41:34 +08:00
|
|
|
return nil, nil, 0, err
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
2022-09-25 20:12:52 +08:00
|
|
|
downloadTimeCost += time.Since(downloadStart)
|
2021-11-08 19:49:07 +08:00
|
|
|
|
2024-03-08 18:25:02 +08:00
|
|
|
iter, err := storage.NewBinlogDeserializeReader(data, pkID)
|
2022-09-25 20:12:52 +08:00
|
|
|
if err != nil {
|
2024-03-08 18:25:02 +08:00
|
|
|
log.Warn("new insert binlogs reader wrong", zap.Strings("path", path), zap.Error(err))
|
2022-10-31 17:41:34 +08:00
|
|
|
return nil, nil, 0, err
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
2023-08-08 21:17:21 +08:00
|
|
|
|
2024-03-08 18:25:02 +08:00
|
|
|
for {
|
|
|
|
err := iter.Next()
|
|
|
|
if err != nil {
|
|
|
|
if err == sio.EOF {
|
|
|
|
break
|
|
|
|
} else {
|
|
|
|
log.Warn("transfer interface to Value wrong", zap.Strings("path", path))
|
|
|
|
return nil, nil, 0, errors.New("unexpected error")
|
|
|
|
}
|
2022-09-25 20:12:52 +08:00
|
|
|
}
|
2024-03-08 18:25:02 +08:00
|
|
|
v := iter.Value()
|
2022-09-25 20:12:52 +08:00
|
|
|
if isDeletedValue(v) {
|
|
|
|
continue
|
|
|
|
}
|
2022-02-07 22:45:46 +08:00
|
|
|
|
2022-09-25 20:12:52 +08:00
|
|
|
ts := Timestamp(v.Timestamp)
|
|
|
|
// Filtering expired entity
|
|
|
|
if t.isExpiredEntity(ts, currentTs) {
|
|
|
|
expired++
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2023-08-08 21:17:21 +08:00
|
|
|
// Update timestampFrom, timestampTo
|
|
|
|
if v.Timestamp < timestampFrom || timestampFrom == -1 {
|
|
|
|
timestampFrom = v.Timestamp
|
|
|
|
}
|
|
|
|
if v.Timestamp > timestampTo || timestampFrom == -1 {
|
|
|
|
timestampTo = v.Timestamp
|
|
|
|
}
|
|
|
|
|
2022-09-25 20:12:52 +08:00
|
|
|
row, ok := v.Value.(map[UniqueID]interface{})
|
|
|
|
if !ok {
|
2023-06-19 14:18:41 +08:00
|
|
|
log.Warn("transfer interface to map wrong", zap.Strings("path", path))
|
2022-10-31 17:41:34 +08:00
|
|
|
return nil, nil, 0, errors.New("unexpected error")
|
2022-09-25 20:12:52 +08:00
|
|
|
}
|
|
|
|
|
2024-01-13 01:32:52 +08:00
|
|
|
err = writeBuffer.Append(row)
|
|
|
|
if err != nil {
|
|
|
|
return nil, nil, 0, err
|
2022-09-25 20:12:52 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
currentRows++
|
2024-01-13 01:32:52 +08:00
|
|
|
stats.Update(v.PK)
|
|
|
|
|
|
|
|
// check size every 100 rows in case of too many `GetMemorySize` call
|
|
|
|
if (currentRows+1)%100 == 0 && writeBuffer.GetMemorySize() > paramtable.Get().DataNodeCfg.BinLogMaxSize.GetAsInt() {
|
|
|
|
numRows += int64(writeBuffer.GetRowNum())
|
2022-09-25 20:12:52 +08:00
|
|
|
uploadInsertStart := time.Now()
|
2024-02-20 14:38:51 +08:00
|
|
|
inPaths, err := t.uploadSingleInsertLog(ctx, targetSegID, partID, meta, writeBuffer)
|
2022-09-25 20:12:52 +08:00
|
|
|
if err != nil {
|
2022-12-12 10:33:26 +08:00
|
|
|
log.Warn("failed to upload single insert log", zap.Error(err))
|
2022-10-31 17:41:34 +08:00
|
|
|
return nil, nil, 0, err
|
2022-09-25 20:12:52 +08:00
|
|
|
}
|
|
|
|
uploadInsertTimeCost += time.Since(uploadInsertStart)
|
2023-08-08 21:17:21 +08:00
|
|
|
addInsertFieldPath(inPaths, timestampFrom, timestampTo)
|
|
|
|
timestampFrom = -1
|
|
|
|
timestampTo = -1
|
2021-11-08 19:49:07 +08:00
|
|
|
|
2024-01-13 01:32:52 +08:00
|
|
|
writeBuffer, _ = storage.NewInsertData(meta.GetSchema())
|
2022-09-25 20:12:52 +08:00
|
|
|
currentRows = 0
|
|
|
|
numBinlogs++
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2023-05-29 10:21:28 +08:00
|
|
|
|
|
|
|
// upload stats log and remain insert rows
|
2024-01-13 01:32:52 +08:00
|
|
|
if writeBuffer.GetRowNum() > 0 || numRows > 0 {
|
|
|
|
numRows += int64(writeBuffer.GetRowNum())
|
2023-05-29 10:21:28 +08:00
|
|
|
uploadStart := time.Now()
|
2024-01-23 10:37:00 +08:00
|
|
|
inPaths, statsPaths, err := t.uploadRemainLog(ctx, targetSegID, partID, meta,
|
2024-02-20 14:38:51 +08:00
|
|
|
stats, numRows+int64(currentRows), writeBuffer)
|
2022-09-25 20:12:52 +08:00
|
|
|
if err != nil {
|
2022-10-31 17:41:34 +08:00
|
|
|
return nil, nil, 0, err
|
2022-09-25 20:12:52 +08:00
|
|
|
}
|
2021-11-08 19:49:07 +08:00
|
|
|
|
2023-05-29 10:21:28 +08:00
|
|
|
uploadInsertTimeCost += time.Since(uploadStart)
|
2023-08-08 21:17:21 +08:00
|
|
|
addInsertFieldPath(inPaths, timestampFrom, timestampTo)
|
2022-10-31 17:41:34 +08:00
|
|
|
addStatFieldPath(statsPaths)
|
2023-05-29 10:21:28 +08:00
|
|
|
numBinlogs += len(inPaths)
|
2022-03-08 16:34:01 +08:00
|
|
|
}
|
2021-11-08 19:49:07 +08:00
|
|
|
|
2022-09-25 20:12:52 +08:00
|
|
|
for _, path := range insertField2Path {
|
|
|
|
insertPaths = append(insertPaths, path)
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
2022-10-31 17:41:34 +08:00
|
|
|
for _, path := range statField2Path {
|
|
|
|
statPaths = append(statPaths, path)
|
2022-09-02 10:34:59 +08:00
|
|
|
}
|
2021-11-08 19:49:07 +08:00
|
|
|
|
2023-08-31 17:35:03 +08:00
|
|
|
log.Info("compact merge end",
|
|
|
|
zap.Int64("remaining insert numRows", numRows),
|
|
|
|
zap.Int64("expired entities", expired),
|
|
|
|
zap.Int("binlog file number", numBinlogs),
|
|
|
|
zap.Duration("download insert log elapse", downloadTimeCost),
|
|
|
|
zap.Duration("upload insert log elapse", uploadInsertTimeCost),
|
|
|
|
zap.Duration("merge elapse", time.Since(mergeStart)))
|
2022-09-25 20:12:52 +08:00
|
|
|
|
2022-10-31 17:41:34 +08:00
|
|
|
return insertPaths, statPaths, numRows, nil
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
2023-11-14 15:56:19 +08:00
|
|
|
func (t *compactionTask) compact() (*datapb.CompactionPlanResult, error) {
|
2024-01-23 10:37:00 +08:00
|
|
|
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(t.ctx, fmt.Sprintf("Compact-%d", t.getPlanID()))
|
|
|
|
defer span.End()
|
2024-02-01 14:25:04 +08:00
|
|
|
|
|
|
|
log := log.Ctx(ctx).With(zap.Int64("planID", t.plan.GetPlanID()), zap.Int32("timeout in seconds", t.plan.GetTimeoutInSeconds()))
|
2024-01-23 10:37:00 +08:00
|
|
|
if ok := funcutil.CheckCtxValid(ctx); !ok {
|
2022-12-01 20:07:15 +08:00
|
|
|
log.Warn("compact wrong, task context done or timeout")
|
2022-08-23 15:50:52 +08:00
|
|
|
return nil, errContext
|
2022-01-18 17:49:39 +08:00
|
|
|
}
|
|
|
|
|
2024-01-23 10:37:00 +08:00
|
|
|
ctxTimeout, cancelAll := context.WithTimeout(ctx, time.Duration(t.plan.GetTimeoutInSeconds())*time.Second)
|
2021-11-08 19:49:07 +08:00
|
|
|
defer cancelAll()
|
|
|
|
|
2024-02-01 14:25:04 +08:00
|
|
|
compactStart := time.Now()
|
|
|
|
durInQueue := t.tr.RecordSpan()
|
|
|
|
log.Info("compact start")
|
|
|
|
if len(t.plan.GetSegmentBinlogs()) < 1 {
|
2022-12-01 20:07:15 +08:00
|
|
|
log.Warn("compact wrong, there's no segments in segment binlogs")
|
2022-08-23 15:50:52 +08:00
|
|
|
return nil, errIllegalCompactionPlan
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
2024-02-01 14:25:04 +08:00
|
|
|
targetSegID, err := t.AllocOne()
|
2024-01-18 22:06:31 +08:00
|
|
|
if err != nil {
|
2024-02-01 14:25:04 +08:00
|
|
|
log.Warn("compact wrong, unable to allocate segmentID", zap.Error(err))
|
2024-01-18 22:06:31 +08:00
|
|
|
return nil, err
|
|
|
|
}
|
2021-11-08 19:49:07 +08:00
|
|
|
|
2024-02-01 14:25:04 +08:00
|
|
|
segIDs := lo.Map(t.plan.GetSegmentBinlogs(), func(binlogs *datapb.CompactionSegmentBinlogs, _ int) int64 {
|
|
|
|
return binlogs.GetSegmentID()
|
|
|
|
})
|
2021-11-08 19:49:07 +08:00
|
|
|
|
|
|
|
// Inject to stop flush
|
2024-02-01 14:25:04 +08:00
|
|
|
// when compaction failed, these segments need to be Unblocked by injectDone in compaction_executor
|
|
|
|
// when compaction succeeded, these segments will be Unblocked by SyncSegments from DataCoord.
|
2023-11-15 15:24:18 +08:00
|
|
|
for _, segID := range segIDs {
|
|
|
|
t.syncMgr.Block(segID)
|
|
|
|
}
|
2024-02-01 14:25:04 +08:00
|
|
|
log.Info("compact finsh injection", zap.Duration("elapse", t.tr.RecordSpan()))
|
|
|
|
|
|
|
|
if err := binlog.DecompressCompactionBinlogs(t.plan.GetSegmentBinlogs()); err != nil {
|
|
|
|
log.Warn("compact wrong, fail to decompress compaction binlogs", zap.Error(err))
|
|
|
|
return nil, err
|
|
|
|
}
|
2021-11-08 19:49:07 +08:00
|
|
|
|
2023-09-21 09:45:27 +08:00
|
|
|
dblobs := make(map[UniqueID][]*Blob)
|
2023-06-19 14:18:41 +08:00
|
|
|
allPath := make([][]string, 0)
|
2021-11-08 19:49:07 +08:00
|
|
|
for _, s := range t.plan.GetSegmentBinlogs() {
|
2022-02-18 18:47:51 +08:00
|
|
|
// Get the number of field binlog files from non-empty segment
|
|
|
|
var binlogNum int
|
|
|
|
for _, b := range s.GetFieldBinlogs() {
|
|
|
|
if b != nil {
|
|
|
|
binlogNum = len(b.GetBinlogs())
|
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
// Unable to deal with all empty segments cases, so return error
|
|
|
|
if binlogNum == 0 {
|
2023-08-31 17:35:03 +08:00
|
|
|
log.Warn("compact wrong, all segments' binlogs are empty")
|
2022-08-23 15:50:52 +08:00
|
|
|
return nil, errIllegalCompactionPlan
|
2022-02-18 18:47:51 +08:00
|
|
|
}
|
2021-11-08 19:49:07 +08:00
|
|
|
|
2022-02-18 18:47:51 +08:00
|
|
|
for idx := 0; idx < binlogNum; idx++ {
|
|
|
|
var ps []string
|
2021-11-08 19:49:07 +08:00
|
|
|
for _, f := range s.GetFieldBinlogs() {
|
2021-12-19 20:00:42 +08:00
|
|
|
ps = append(ps, f.GetBinlogs()[idx].GetLogPath())
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
2023-06-19 14:18:41 +08:00
|
|
|
allPath = append(allPath, ps)
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
segID := s.GetSegmentID()
|
2023-09-19 18:05:22 +08:00
|
|
|
paths := make([]string, 0)
|
2021-11-08 19:49:07 +08:00
|
|
|
for _, d := range s.GetDeltalogs() {
|
2021-12-19 20:00:42 +08:00
|
|
|
for _, l := range d.GetBinlogs() {
|
|
|
|
path := l.GetLogPath()
|
2023-09-19 18:05:22 +08:00
|
|
|
paths = append(paths, path)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(paths) != 0 {
|
2024-02-20 14:38:51 +08:00
|
|
|
bs, err := downloadBlobs(ctxTimeout, t.binlogIO, paths)
|
2023-09-19 18:05:22 +08:00
|
|
|
if err != nil {
|
2024-02-01 14:25:04 +08:00
|
|
|
log.Warn("compact wrong, fail to download deltalogs", zap.Int64("segment", segID), zap.Strings("path", paths), zap.Error(err))
|
2023-09-19 18:05:22 +08:00
|
|
|
return nil, err
|
2021-12-19 20:00:42 +08:00
|
|
|
}
|
2023-09-19 18:05:22 +08:00
|
|
|
dblobs[segID] = append(dblobs[segID], bs...)
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
}
|
2024-02-01 14:25:04 +08:00
|
|
|
log.Info("compact download deltalogs done", zap.Duration("elapse", t.tr.RecordSpan()))
|
2021-11-08 19:49:07 +08:00
|
|
|
|
2023-09-04 17:41:48 +08:00
|
|
|
deltaPk2Ts, err := t.mergeDeltalogs(dblobs)
|
2021-11-08 19:49:07 +08:00
|
|
|
if err != nil {
|
2024-02-01 14:25:04 +08:00
|
|
|
log.Warn("compact wrong, fail to merge deltalogs", zap.Error(err))
|
2022-08-23 15:50:52 +08:00
|
|
|
return nil, err
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
2024-02-01 14:25:04 +08:00
|
|
|
segmentBinlog := t.plan.GetSegmentBinlogs()[0]
|
|
|
|
partID := segmentBinlog.GetPartitionID()
|
|
|
|
meta := &etcdpb.CollectionMeta{ID: t.metaCache.Collection(), Schema: t.metaCache.Schema()}
|
|
|
|
|
2023-06-19 14:18:41 +08:00
|
|
|
inPaths, statsPaths, numRows, err := t.merge(ctxTimeout, allPath, targetSegID, partID, meta, deltaPk2Ts)
|
2021-11-08 19:49:07 +08:00
|
|
|
if err != nil {
|
2024-02-01 14:25:04 +08:00
|
|
|
log.Warn("compact wrong, fail to merge", zap.Error(err))
|
2022-08-23 15:50:52 +08:00
|
|
|
return nil, err
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
2023-11-14 15:56:19 +08:00
|
|
|
pack := &datapb.CompactionSegment{
|
2021-11-08 19:49:07 +08:00
|
|
|
SegmentID: targetSegID,
|
2022-09-25 20:12:52 +08:00
|
|
|
InsertLogs: inPaths,
|
|
|
|
Field2StatslogPaths: statsPaths,
|
2021-11-08 19:49:07 +08:00
|
|
|
NumOfRows: numRows,
|
2022-10-20 16:39:29 +08:00
|
|
|
Channel: t.plan.GetChannel(),
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
2023-08-31 17:35:03 +08:00
|
|
|
log.Info("compact done",
|
2022-07-04 11:10:20 +08:00
|
|
|
zap.Int64("targetSegmentID", targetSegID),
|
2022-10-18 15:33:26 +08:00
|
|
|
zap.Int64s("compactedFrom", segIDs),
|
2022-09-25 20:12:52 +08:00
|
|
|
zap.Int("num of binlog paths", len(inPaths)),
|
|
|
|
zap.Int("num of stats paths", len(statsPaths)),
|
2023-09-04 17:41:48 +08:00
|
|
|
zap.Int("num of delta paths", len(pack.GetDeltalogs())),
|
2024-02-01 14:25:04 +08:00
|
|
|
zap.Duration("elapse", time.Since(compactStart)),
|
2021-11-17 17:57:11 +08:00
|
|
|
)
|
2022-02-08 19:15:45 +08:00
|
|
|
|
2023-12-27 15:56:48 +08:00
|
|
|
metrics.DataNodeCompactionLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), t.plan.GetType().String()).Observe(float64(t.tr.ElapseSpan().Milliseconds()))
|
2023-06-26 17:52:44 +08:00
|
|
|
metrics.DataNodeCompactionLatencyInQueue.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(durInQueue.Milliseconds()))
|
2022-02-28 19:11:55 +08:00
|
|
|
|
2023-11-14 15:56:19 +08:00
|
|
|
planResult := &datapb.CompactionPlanResult{
|
|
|
|
State: commonpb.CompactionState_Completed,
|
|
|
|
PlanID: t.getPlanID(),
|
2024-02-01 14:25:04 +08:00
|
|
|
Channel: t.plan.GetChannel(),
|
2023-11-14 15:56:19 +08:00
|
|
|
Segments: []*datapb.CompactionSegment{pack},
|
2024-01-19 11:34:54 +08:00
|
|
|
Type: t.plan.GetType(),
|
2023-11-14 15:56:19 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
return planResult, nil
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
2023-11-29 10:50:29 +08:00
|
|
|
func (t *compactionTask) injectDone() {
|
2024-02-18 14:08:49 +08:00
|
|
|
t.injectDoneOnce.Do(func() {
|
|
|
|
for _, binlog := range t.plan.SegmentBinlogs {
|
|
|
|
t.syncMgr.Unblock(binlog.SegmentID)
|
|
|
|
}
|
|
|
|
})
|
2023-03-17 17:27:56 +08:00
|
|
|
}
|
|
|
|
|
2021-11-08 19:49:07 +08:00
|
|
|
// TODO copy maybe expensive, but this seems to be the only convinent way.
|
|
|
|
func interface2FieldData(schemaDataType schemapb.DataType, content []interface{}, numRows int64) (storage.FieldData, error) {
|
|
|
|
var rst storage.FieldData
|
|
|
|
switch schemaDataType {
|
|
|
|
case schemapb.DataType_Bool:
|
2023-09-21 09:45:27 +08:00
|
|
|
data := &storage.BoolFieldData{
|
2023-01-28 11:09:52 +08:00
|
|
|
Data: make([]bool, 0, len(content)),
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
for _, c := range content {
|
|
|
|
r, ok := c.(bool)
|
|
|
|
if !ok {
|
|
|
|
return nil, errTransferType
|
|
|
|
}
|
|
|
|
data.Data = append(data.Data, r)
|
|
|
|
}
|
|
|
|
rst = data
|
|
|
|
|
|
|
|
case schemapb.DataType_Int8:
|
2023-09-21 09:45:27 +08:00
|
|
|
data := &storage.Int8FieldData{
|
2023-01-28 11:09:52 +08:00
|
|
|
Data: make([]int8, 0, len(content)),
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
for _, c := range content {
|
|
|
|
r, ok := c.(int8)
|
|
|
|
if !ok {
|
|
|
|
return nil, errTransferType
|
|
|
|
}
|
|
|
|
data.Data = append(data.Data, r)
|
|
|
|
}
|
|
|
|
rst = data
|
|
|
|
|
|
|
|
case schemapb.DataType_Int16:
|
2023-09-21 09:45:27 +08:00
|
|
|
data := &storage.Int16FieldData{
|
2023-01-28 11:09:52 +08:00
|
|
|
Data: make([]int16, 0, len(content)),
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
for _, c := range content {
|
|
|
|
r, ok := c.(int16)
|
|
|
|
if !ok {
|
|
|
|
return nil, errTransferType
|
|
|
|
}
|
|
|
|
data.Data = append(data.Data, r)
|
|
|
|
}
|
|
|
|
rst = data
|
|
|
|
|
|
|
|
case schemapb.DataType_Int32:
|
2023-09-21 09:45:27 +08:00
|
|
|
data := &storage.Int32FieldData{
|
2023-01-28 11:09:52 +08:00
|
|
|
Data: make([]int32, 0, len(content)),
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
for _, c := range content {
|
|
|
|
r, ok := c.(int32)
|
|
|
|
if !ok {
|
|
|
|
return nil, errTransferType
|
|
|
|
}
|
|
|
|
data.Data = append(data.Data, r)
|
|
|
|
}
|
|
|
|
rst = data
|
|
|
|
|
|
|
|
case schemapb.DataType_Int64:
|
2023-09-21 09:45:27 +08:00
|
|
|
data := &storage.Int64FieldData{
|
2023-01-28 11:09:52 +08:00
|
|
|
Data: make([]int64, 0, len(content)),
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
for _, c := range content {
|
|
|
|
r, ok := c.(int64)
|
|
|
|
if !ok {
|
|
|
|
return nil, errTransferType
|
|
|
|
}
|
|
|
|
data.Data = append(data.Data, r)
|
|
|
|
}
|
|
|
|
rst = data
|
|
|
|
|
|
|
|
case schemapb.DataType_Float:
|
2023-09-21 09:45:27 +08:00
|
|
|
data := &storage.FloatFieldData{
|
2023-01-28 11:09:52 +08:00
|
|
|
Data: make([]float32, 0, len(content)),
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
for _, c := range content {
|
|
|
|
r, ok := c.(float32)
|
|
|
|
if !ok {
|
|
|
|
return nil, errTransferType
|
|
|
|
}
|
|
|
|
data.Data = append(data.Data, r)
|
|
|
|
}
|
|
|
|
rst = data
|
|
|
|
|
|
|
|
case schemapb.DataType_Double:
|
2023-09-21 09:45:27 +08:00
|
|
|
data := &storage.DoubleFieldData{
|
2023-01-28 11:09:52 +08:00
|
|
|
Data: make([]float64, 0, len(content)),
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
for _, c := range content {
|
|
|
|
r, ok := c.(float64)
|
|
|
|
if !ok {
|
|
|
|
return nil, errTransferType
|
|
|
|
}
|
|
|
|
data.Data = append(data.Data, r)
|
|
|
|
}
|
|
|
|
rst = data
|
|
|
|
|
2022-03-25 14:27:25 +08:00
|
|
|
case schemapb.DataType_String, schemapb.DataType_VarChar:
|
2023-09-21 09:45:27 +08:00
|
|
|
data := &storage.StringFieldData{
|
2023-01-28 11:09:52 +08:00
|
|
|
Data: make([]string, 0, len(content)),
|
2022-03-25 14:27:25 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
for _, c := range content {
|
|
|
|
r, ok := c.(string)
|
|
|
|
if !ok {
|
|
|
|
return nil, errTransferType
|
|
|
|
}
|
|
|
|
data.Data = append(data.Data, r)
|
|
|
|
}
|
|
|
|
rst = data
|
|
|
|
|
2023-05-17 19:09:22 +08:00
|
|
|
case schemapb.DataType_JSON:
|
2023-09-21 09:45:27 +08:00
|
|
|
data := &storage.JSONFieldData{
|
2023-05-17 19:09:22 +08:00
|
|
|
Data: make([][]byte, 0, len(content)),
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, c := range content {
|
|
|
|
r, ok := c.([]byte)
|
|
|
|
if !ok {
|
2023-12-28 15:42:48 +08:00
|
|
|
return nil, errTransferType
|
|
|
|
}
|
|
|
|
data.Data = append(data.Data, r)
|
|
|
|
}
|
|
|
|
rst = data
|
|
|
|
|
|
|
|
case schemapb.DataType_Array:
|
|
|
|
data := &storage.ArrayFieldData{
|
|
|
|
Data: make([]*schemapb.ScalarField, 0, len(content)),
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(content) > 0 {
|
|
|
|
data.ElementType = content[0].(*schemapb.ScalarField).GetArrayData().GetElementType()
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, c := range content {
|
|
|
|
r, ok := c.(*schemapb.ScalarField)
|
|
|
|
if !ok {
|
2023-05-17 19:09:22 +08:00
|
|
|
return nil, errTransferType
|
|
|
|
}
|
|
|
|
data.Data = append(data.Data, r)
|
|
|
|
}
|
|
|
|
rst = data
|
|
|
|
|
2021-11-08 19:49:07 +08:00
|
|
|
case schemapb.DataType_FloatVector:
|
2023-09-21 09:45:27 +08:00
|
|
|
data := &storage.FloatVectorFieldData{
|
2023-01-28 11:09:52 +08:00
|
|
|
Data: []float32{},
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
for _, c := range content {
|
2021-11-23 17:09:15 +08:00
|
|
|
r, ok := c.([]float32)
|
2021-11-08 19:49:07 +08:00
|
|
|
if !ok {
|
|
|
|
return nil, errTransferType
|
|
|
|
}
|
2021-11-23 17:09:15 +08:00
|
|
|
data.Data = append(data.Data, r...)
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
data.Dim = len(data.Data) / int(numRows)
|
|
|
|
rst = data
|
|
|
|
|
2023-09-08 10:03:16 +08:00
|
|
|
case schemapb.DataType_Float16Vector:
|
2023-09-21 09:45:27 +08:00
|
|
|
data := &storage.Float16VectorFieldData{
|
2023-09-08 10:03:16 +08:00
|
|
|
Data: []byte{},
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, c := range content {
|
|
|
|
r, ok := c.([]byte)
|
|
|
|
if !ok {
|
|
|
|
return nil, errTransferType
|
|
|
|
}
|
|
|
|
data.Data = append(data.Data, r...)
|
|
|
|
}
|
|
|
|
|
|
|
|
data.Dim = len(data.Data) / 2 / int(numRows)
|
|
|
|
rst = data
|
|
|
|
|
2024-01-11 15:48:51 +08:00
|
|
|
case schemapb.DataType_BFloat16Vector:
|
|
|
|
data := &storage.BFloat16VectorFieldData{
|
|
|
|
Data: []byte{},
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, c := range content {
|
|
|
|
r, ok := c.([]byte)
|
|
|
|
if !ok {
|
|
|
|
return nil, errTransferType
|
|
|
|
}
|
|
|
|
data.Data = append(data.Data, r...)
|
|
|
|
}
|
|
|
|
|
|
|
|
data.Dim = len(data.Data) / 2 / int(numRows)
|
|
|
|
rst = data
|
|
|
|
|
2021-11-08 19:49:07 +08:00
|
|
|
case schemapb.DataType_BinaryVector:
|
2023-09-21 09:45:27 +08:00
|
|
|
data := &storage.BinaryVectorFieldData{
|
2023-01-28 11:09:52 +08:00
|
|
|
Data: []byte{},
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
for _, c := range content {
|
2021-11-23 17:09:15 +08:00
|
|
|
r, ok := c.([]byte)
|
2021-11-08 19:49:07 +08:00
|
|
|
if !ok {
|
|
|
|
return nil, errTransferType
|
|
|
|
}
|
2021-11-23 17:09:15 +08:00
|
|
|
data.Data = append(data.Data, r...)
|
2021-11-08 19:49:07 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
data.Dim = len(data.Data) * 8 / int(numRows)
|
|
|
|
rst = data
|
|
|
|
|
|
|
|
default:
|
|
|
|
return nil, errUnknownDataType
|
|
|
|
}
|
|
|
|
|
|
|
|
return rst, nil
|
|
|
|
}
|
|
|
|
|
2021-11-11 20:56:49 +08:00
|
|
|
func (t *compactionTask) getCollection() UniqueID {
|
2023-11-15 15:24:18 +08:00
|
|
|
return t.metaCache.Collection()
|
2021-11-11 20:56:49 +08:00
|
|
|
}
|
2022-02-07 22:45:46 +08:00
|
|
|
|
|
|
|
func (t *compactionTask) GetCurrentTime() typeutil.Timestamp {
|
|
|
|
return tsoutil.GetCurrentTime()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (t *compactionTask) isExpiredEntity(ts, now Timestamp) bool {
|
2022-04-28 11:51:47 +08:00
|
|
|
// entity expire is not enabled if duration <= 0
|
2022-10-10 20:31:22 +08:00
|
|
|
if t.plan.GetCollectionTtl() <= 0 {
|
2022-02-07 22:45:46 +08:00
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
|
|
|
pts, _ := tsoutil.ParseTS(ts)
|
|
|
|
pnow, _ := tsoutil.ParseTS(now)
|
2022-10-10 20:31:22 +08:00
|
|
|
expireTime := pts.Add(time.Duration(t.plan.GetCollectionTtl()))
|
2022-02-07 22:45:46 +08:00
|
|
|
return expireTime.Before(pnow)
|
|
|
|
}
|