cai.zhang a16d04f5d1
feat: Support ttl field for entity level expiration (#46342)
issue: #46033

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
## Pull Request Summary: Entity-Level TTL Field Support

### Core Invariant and Design
This PR introduces **per-entity TTL (time-to-live) expiration** via a
dedicated TIMESTAMPTZ field as a fine-grained alternative to
collection-level TTL. The key invariant is **mutual exclusivity**:
collection-level TTL and entity-level TTL field cannot coexist on the
same collection. Validation is enforced at the proxy layer during
collection creation/alteration (`validateTTL()` prevents both being set
simultaneously).

### What Is Removed and Why
- **Global `EntityExpirationTTL` parameter** removed from config
(`configs/milvus.yaml`, `pkg/util/paramtable/component_param.go`). This
was the only mechanism for collection-level expiration. The removal is
safe because:
- The collection-level TTL path (`isEntityExpired(ts)` check) remains
intact in the codebase for backward compatibility
- TTL field check (`isEntityExpiredByTTLField()`) is a secondary path
invoked only when a TTL field is configured
- Existing deployments using collection TTL can continue without
modification
  
The global parameter was removed specifically because entity-level TTL
makes per-entity control redundant with a collection-wide setting, and
the PR chooses one mechanism per collection rather than layering both.

### No Data Loss or Behavior Regression
**TTL filtering logic is additive and safe:**
1. **Collection-level TTL unaffected**: The `isEntityExpired(ts)` check
still applies when no TTL field is configured; callers of
`EntityFilter.Filtered()` pass `-1` as the TTL expiration timestamp when
no field exists, causing `isEntityExpiredByTTLField()` to return false
immediately
2. **Null/invalid TTL values treated safely**: Rows with null TTL or TTL
≤ 0 are marked as "never expire" (using sentinel value `int64(^uint64(0)
>> 1)`) and are preserved across compactions; percentile calculations
only include positive TTL values
3. **Query-time filtering automatic**: TTL filtering is transparently
added to expression compilation via `AddTTLFieldFilterExpressions()`,
which appends `(ttl_field IS NULL OR ttl_field > current_time)` to the
filter pipeline. Entities with null TTL always pass the filter
4. **Compaction triggering granular**: Percentile-based expiration (20%,
40%, 60%, 80%, 100%) allows configurable compaction thresholds via
`SingleCompactionRatioThreshold`, preventing premature data deletion

### Capability Added: Per-Entity Expiration with Data Distribution
Awareness
Users can now specify a TIMESTAMPTZ collection property `ttl_field`
naming a schema field. During data writes, TTL values are collected per
segment and percentile quantiles (5-value array) are computed and stored
in segment metadata. At query time, the TTL field is automatically
filtered. At compaction time, segment-level percentiles drive
expiration-based compaction decisions, enabling intelligent compaction
of segments where a configurable fraction of data has expired (e.g.,
compact when 40% of rows are expired, controlled by threshold ratio).
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

---------

Signed-off-by: Cai Zhang <cai.zhang@zilliz.com>
2026-01-05 10:27:24 +08:00

182 lines
5.8 KiB
Go

package compactor
import (
"context"
"fmt"
"time"
"github.com/apache/arrow/go/v17/arrow/array"
"go.opentelemetry.io/otel"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/compaction"
"github.com/milvus-io/milvus/internal/flushcommon/io"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/metrics"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/util/timerecord"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
func mergeSortMultipleSegments(ctx context.Context,
plan *datapb.CompactionPlan,
collectionID, partitionID, maxRows int64,
binlogIO io.BinlogIO,
binlogs []*datapb.CompactionSegmentBinlogs,
tr *timerecord.TimeRecorder,
currentTime time.Time,
collectionTtl int64,
compactionParams compaction.Params,
sortByFields []int64,
) ([]*datapb.CompactionSegment, error) {
_ = tr.RecordSpan()
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "mergeSortMultipleSegments")
defer span.End()
log := log.With(zap.Int64("planID", plan.GetPlanID()))
segIDAlloc := allocator.NewLocalAllocator(plan.GetPreAllocatedSegmentIDs().GetBegin(), plan.GetPreAllocatedSegmentIDs().GetEnd())
logIDAlloc := allocator.NewLocalAllocator(plan.GetPreAllocatedLogIDs().GetBegin(), plan.GetPreAllocatedLogIDs().GetEnd())
compAlloc := NewCompactionAllocator(segIDAlloc, logIDAlloc)
writer, err := NewMultiSegmentWriter(ctx, binlogIO, compAlloc, plan.GetMaxSize(), plan.GetSchema(), compactionParams, maxRows, partitionID, collectionID, plan.GetChannel(), 4096,
storage.WithStorageConfig(compactionParams.StorageConfig),
storage.WithUseLoonFFI(compactionParams.UseLoonFFI),
)
if err != nil {
return nil, err
}
pkField, err := typeutil.GetPrimaryFieldSchema(plan.GetSchema())
if err != nil {
log.Warn("failed to get pk field from schema")
return nil, err
}
ttlFieldID := getTTLFieldID(plan.GetSchema())
hasTTLField := ttlFieldID >= common.StartOfUserFieldID
segmentReaders := make([]storage.RecordReader, len(binlogs))
segmentFilters := make([]compaction.EntityFilter, len(binlogs))
for i, s := range binlogs {
var reader storage.RecordReader
if s.GetManifest() != "" {
reader, err = storage.NewManifestRecordReader(ctx,
s.GetManifest(),
plan.GetSchema(),
storage.WithCollectionID(collectionID),
storage.WithDownloader(binlogIO.Download),
storage.WithVersion(s.StorageVersion),
storage.WithStorageConfig(compactionParams.StorageConfig),
)
} else {
reader, err = storage.NewBinlogRecordReader(ctx,
s.GetFieldBinlogs(),
plan.GetSchema(),
storage.WithCollectionID(collectionID),
storage.WithDownloader(binlogIO.Download),
storage.WithVersion(s.StorageVersion),
storage.WithStorageConfig(compactionParams.StorageConfig),
)
}
if err != nil {
return nil, err
}
segmentReaders[i] = reader
deltalogPaths := make([]string, 0)
for _, d := range s.GetDeltalogs() {
for _, l := range d.GetBinlogs() {
deltalogPaths = append(deltalogPaths, l.GetLogPath())
}
}
delta, err := compaction.ComposeDeleteFromDeltalogs(ctx, binlogIO, deltalogPaths)
if err != nil {
return nil, err
}
segmentFilters[i] = compaction.NewEntityFilter(delta, collectionTtl, currentTime)
}
defer func() {
for _, r := range segmentReaders {
r.Close()
}
}()
var predicate func(r storage.Record, ri, i int) bool
switch pkField.DataType {
case schemapb.DataType_Int64:
predicate = func(r storage.Record, ri, i int) bool {
pk := r.Column(pkField.FieldID).(*array.Int64).Value(i)
ts := r.Column(common.TimeStampField).(*array.Int64).Value(i)
expireTs := int64(-1)
if hasTTLField {
col := r.Column(ttlFieldID).(*array.Int64)
if col.IsValid(i) {
expireTs = col.Value(i)
}
}
return !segmentFilters[ri].Filtered(pk, uint64(ts), expireTs)
}
case schemapb.DataType_VarChar:
predicate = func(r storage.Record, ri, i int) bool {
pk := r.Column(pkField.FieldID).(*array.String).Value(i)
ts := r.Column(common.TimeStampField).(*array.Int64).Value(i)
expireTs := int64(-1)
if hasTTLField {
col := r.Column(ttlFieldID).(*array.Int64)
if col.IsValid(i) {
expireTs = col.Value(i)
}
}
return !segmentFilters[ri].Filtered(pk, uint64(ts), expireTs)
}
default:
log.Warn("compaction only support int64 and varchar pk field")
}
if _, err = storage.MergeSort(compactionParams.BinLogMaxSize, plan.GetSchema(), segmentReaders, writer, predicate, sortByFields); err != nil {
writer.Close()
return nil, err
}
if err := writer.Close(); err != nil {
log.Warn("compact wrong, failed to finish writer", zap.Error(err))
return nil, err
}
res := writer.GetCompactionSegments()
for _, seg := range res {
seg.IsSorted = true
}
var (
deletedRowCount int
expiredRowCount int
missingDeleteCount int
deltalogDeleteEntriesCount int
)
for _, filter := range segmentFilters {
deletedRowCount += filter.GetDeletedCount()
expiredRowCount += filter.GetExpiredCount()
missingDeleteCount += filter.GetMissingDeleteCount()
deltalogDeleteEntriesCount += filter.GetDeltalogDeleteCount()
}
totalElapse := tr.RecordSpan()
log.Info("compact mergeSortMultipleSegments end",
zap.Int("deleted row count", deletedRowCount),
zap.Int("expired entities", expiredRowCount),
zap.Int("missing deletes", missingDeleteCount),
zap.Duration("total elapse", totalElapse))
metrics.DataNodeCompactionDeleteCount.WithLabelValues(fmt.Sprint(collectionID)).Add(float64(deltalogDeleteEntriesCount))
metrics.DataNodeCompactionMissingDeleteCount.WithLabelValues(fmt.Sprint(collectionID)).Add(float64(missingDeleteCount))
return res, nil
}