milvus/internal/datanode/compaction/compactor_common.go

297 lines
8.4 KiB
Go

// 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 compaction
import (
"context"
sio "io"
"strconv"
"time"
"go.opentelemetry.io/otel"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/allocator"
"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/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/util/tsoutil"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
const compactionBatchSize = 100
type EntityFilter struct {
deletedPkTs map[interface{}]typeutil.Timestamp // pk2ts
ttl int64 // nanoseconds
currentTime time.Time
expiredCount int
deletedCount int
}
func newEntityFilter(deletedPkTs map[interface{}]typeutil.Timestamp, ttl int64, currTime time.Time) *EntityFilter {
if deletedPkTs == nil {
deletedPkTs = make(map[interface{}]typeutil.Timestamp)
}
return &EntityFilter{
deletedPkTs: deletedPkTs,
ttl: ttl,
currentTime: currTime,
}
}
func (filter *EntityFilter) Filtered(pk any, ts typeutil.Timestamp) bool {
if filter.isEntityDeleted(pk, ts) {
filter.deletedCount++
return true
}
// Filtering expired entity
if filter.isEntityExpired(ts) {
filter.expiredCount++
return true
}
return false
}
func (filter *EntityFilter) GetExpiredCount() int {
return filter.expiredCount
}
func (filter *EntityFilter) GetDeletedCount() int {
return filter.deletedCount
}
func (filter *EntityFilter) GetDeltalogDeleteCount() int {
return len(filter.deletedPkTs)
}
func (filter *EntityFilter) GetMissingDeleteCount() int {
diff := filter.GetDeltalogDeleteCount() - filter.GetDeletedCount()
if diff <= 0 {
diff = 0
}
return diff
}
func (filter *EntityFilter) isEntityDeleted(pk interface{}, pkTs typeutil.Timestamp) bool {
if deleteTs, ok := filter.deletedPkTs[pk]; ok {
// 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 pkTs < deleteTs {
return true
}
}
return false
}
func (filter *EntityFilter) isEntityExpired(entityTs typeutil.Timestamp) bool {
// entity expire is not enabled if duration <= 0
if filter.ttl <= 0 {
return false
}
entityTime, _ := tsoutil.ParseTS(entityTs)
// this dur can represents 292 million years before or after 1970, enough for milvus
// ttl calculation
dur := filter.currentTime.UnixMilli() - entityTime.UnixMilli()
// filter.ttl is nanoseconds
return filter.ttl/int64(time.Millisecond) <= dur
}
func mergeDeltalogs(ctx context.Context, io io.BinlogIO, paths []string) (map[interface{}]typeutil.Timestamp, error) {
pk2Ts := make(map[interface{}]typeutil.Timestamp)
log := log.Ctx(ctx)
if len(paths) == 0 {
log.Debug("compact with no deltalogs, skip merge deltalogs")
return pk2Ts, nil
}
blobs := make([]*storage.Blob, 0)
binaries, err := io.Download(ctx, paths)
if err != nil {
log.Warn("compact wrong, fail to download deltalogs",
zap.Strings("path", paths),
zap.Error(err))
return nil, err
}
for i := range binaries {
blobs = append(blobs, &storage.Blob{Value: binaries[i]})
}
reader, err := storage.CreateDeltalogReader(blobs)
if err != nil {
log.Error("malformed delta file", zap.Error(err))
return nil, err
}
defer reader.Close()
for {
err := reader.Next()
if err != nil {
if err == sio.EOF {
break
}
log.Error("compact wrong, fail to read deltalogs", zap.Error(err))
return nil, err
}
dl := reader.Value()
if ts, ok := pk2Ts[dl.Pk.GetValue()]; ok && ts > dl.Ts {
continue
}
pk2Ts[dl.Pk.GetValue()] = dl.Ts
}
log.Info("compact mergeDeltalogs end", zap.Int("delete entries counts", len(pk2Ts)))
return pk2Ts, nil
}
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()
blobs, tr, err := writer.SerializeYield()
startID, _, err := allocator.Alloc(uint32(len(blobs)))
if err != nil {
return nil, nil, err
}
kvs = make(map[string][]byte)
fieldBinlogs = make(map[int64]*datapb.FieldBinlog)
for i := range blobs {
// Blob Key is generated by Serialize from int64 fieldID in collection schema, which won't raise error in ParseInt
fID, _ := strconv.ParseInt(blobs[i].GetKey(), 10, 64)
key, _ := binlog.BuildLogPath(storage.InsertBinlog, writer.GetCollectionID(), writer.GetPartitionID(), writer.GetSegmentID(), fID, startID+int64(i))
kvs[key] = blobs[i].GetValue()
fieldBinlogs[fID] = &datapb.FieldBinlog{
FieldID: fID,
Binlogs: []*datapb.Binlog{
{
LogSize: int64(len(blobs[i].GetValue())),
MemorySize: blobs[i].GetMemorySize(),
LogPath: key,
EntriesNum: blobs[i].RowNum,
TimestampFrom: tr.GetMinTimestamp(),
TimestampTo: tr.GetMaxTimestamp(),
},
},
}
}
return
}
func statSerializeWrite(ctx context.Context, io io.BinlogIO, allocator allocator.Interface, writer *SegmentWriter) (*datapb.FieldBinlog, error) {
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "statslog serializeWrite")
defer span.End()
sblob, err := writer.Finish()
if err != nil {
return nil, err
}
return uploadStatsBlobs(ctx, writer.GetCollectionID(), writer.GetPartitionID(), writer.GetSegmentID(), writer.GetPkID(), writer.GetRowNum(), io, allocator, sblob)
}
func uploadStatsBlobs(ctx context.Context, collectionID, partitionID, segmentID, pkID, numRows int64,
io io.BinlogIO, allocator allocator.Interface, blob *storage.Blob,
) (*datapb.FieldBinlog, error) {
logID, err := allocator.AllocOne()
if err != nil {
return nil, err
}
key, _ := binlog.BuildLogPath(storage.StatsBinlog, collectionID, partitionID, segmentID, pkID, logID)
kvs := map[string][]byte{key: blob.GetValue()}
statFieldLog := &datapb.FieldBinlog{
FieldID: pkID,
Binlogs: []*datapb.Binlog{
{
LogSize: int64(len(blob.GetValue())),
MemorySize: int64(len(blob.GetValue())),
LogPath: key,
EntriesNum: numRows,
},
},
}
if err := io.Upload(ctx, kvs); err != nil {
log.Warn("failed to upload stats log", zap.Error(err))
return nil, err
}
return statFieldLog, nil
}
func mergeFieldBinlogs(base, paths map[typeutil.UniqueID]*datapb.FieldBinlog) {
for fID, fpath := range paths {
if _, ok := base[fID]; !ok {
base[fID] = &datapb.FieldBinlog{FieldID: fID, Binlogs: make([]*datapb.Binlog, 0)}
}
base[fID].Binlogs = append(base[fID].Binlogs, fpath.GetBinlogs()...)
}
}
func bm25SerializeWrite(ctx context.Context, io io.BinlogIO, allocator allocator.Interface, writer *SegmentWriter) ([]*datapb.FieldBinlog, error) {
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "bm25 stats log serializeWrite")
defer span.End()
stats, err := writer.GetBm25StatsBlob()
if err != nil {
return nil, err
}
logID, _, err := allocator.Alloc(uint32(len(stats)))
if err != nil {
return nil, err
}
kvs := make(map[string][]byte)
binlogs := []*datapb.FieldBinlog{}
for fieldID, blob := range stats {
key, _ := binlog.BuildLogPath(storage.BM25Binlog, writer.GetCollectionID(), writer.GetPartitionID(), writer.GetSegmentID(), fieldID, logID)
kvs[key] = blob.GetValue()
fieldLog := &datapb.FieldBinlog{
FieldID: fieldID,
Binlogs: []*datapb.Binlog{
{
LogSize: int64(len(blob.GetValue())),
MemorySize: int64(len(blob.GetValue())),
LogPath: key,
EntriesNum: writer.GetRowNum(),
},
},
}
binlogs = append(binlogs, fieldLog)
}
if err := io.Upload(ctx, kvs); err != nil {
log.Warn("failed to upload bm25 log", zap.Error(err))
return nil, err
}
return binlogs, nil
}