23821 Commits

Author SHA1 Message Date
junjiejiangjjj
537199fb80
fix: fix voyageai model int8 bug (#46819)
https://github.com/milvus-io/milvus/issues/46815

Signed-off-by: junjie.jiang <junjie.jiang@zilliz.com>
2026-01-07 10:41:29 +08:00
foxspy
53a300db83
enhance: update knowhere version (#45564)
issue: #42937 

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: callers must explicitly close output streams (call
Close()) instead of relying on RemoteOutputStream's destructor to
perform closure.
- Logic removed/simplified: RemoteOutputStream's destructor no longer
closes or asserts on the underlying arrow::io::OutputStream; an explicit
public Close() method was added and closure responsibility moved to that
code path.
- Why this is safe (no data loss/regression): callers now invoke Close()
before reading or destroying streams (e.g.,
DiskFileManagerTest::ReadAndWriteWithStream calls os->Close() before
opening the input stream). Write paths remain unchanged
(RemoteOutputStream::Write -> output_stream_->Write), and Close invokes
output_stream_->Close() with status assertion, ensuring
flush/confirmation via the same API and preserving data integrity;
removing destructor-side asserts prevents unexpected failures during
object destruction without changing write/close semantics.
- Chore: updated third-party pins — internal/core/thirdparty/knowhere
CMakeLists.txt: KNOWHERE_VERSION -> a59816e;
internal/core/thirdparty/milvus-common CMakeLists.txt:
MILVUS-COMMON-VERSION -> b6629f7.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: xianliang.li <xianliang.li@zilliz.com>
2026-01-07 10:39:24 +08:00
wei liu
3242895999
fix: resolve data race in indexMeta (#46763)
issue: #46762

Copy the fieldIndexes map while holding the read lock to prevent data
race. The original code released the lock before iterating over the map,
which could cause concurrent access issues.

Affected methods:
- GetSegmentIndexState
- GetIndexedSegments
- IsUnIndexedSegment
- GetSegmentIndexedFields

Signed-off-by: Wei Liu <wei.liu@zilliz.com>
2026-01-07 10:33:25 +08:00
congqixia
4719cb80ca
fix: Remove extra binlog in flaky unit test (#46841)
Fixes: #46840

The test "failure_returns_partial_file_list" had 3 binlog entries but
only mocked 2 Copy calls, causing flaky behavior. Remove the unmocked
third binlog to make the test deterministic.

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
2026-01-07 10:15:25 +08:00
congqixia
bc8d4c175c
fix: check allow_insert_auto_id flag in RESTful v2 insert API (#46822)
Related to #46818

When a collection has autoID enabled and `allow_insert_auto_id` property
set to true, the RESTful v2 insert API was incorrectly rejecting
requests that included the primary key field. This fix adds proper
checking of the `allow_insert_auto_id` flag in the `anyToColumns`
function.

Changes:
- Read `allow_insert_auto_id` property from collection schema
- Skip PK field only when autoID is enabled AND allow_insert_auto_id is
false
- Allow PK field in insert request when allow_insert_auto_id is true
- Filter out empty PK column when autoID is enabled and user didn't
provide PK

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
2026-01-07 10:13:25 +08:00
congqixia
fbd8a766c5
fix: check field exists in column groups before reading from loon manifest (#46816)
Related to #44956

Before reading field data from loon manifest, check if the field exists
in column groups first. If the field does not exist, return an empty
result instead of proceeding with the read operation.

This is a workaround until loon natively supports returning null for
non-existent fields.

---------

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
2026-01-07 10:09:25 +08:00
cai.zhang
63026e6791
fix: Fill shardClientMgr field for queryTask to prevent panic (#46837)
issue: #46836

Signed-off-by: Cai Zhang <cai.zhang@zilliz.com>
2026-01-06 23:07:24 +08:00
yihao.dai
746c8653cc
fix: Fix missing handling of FlushAllMsg in recovery storage (#46802)
issue: https://github.com/milvus-io/milvus/issues/46799

---------

Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
2026-01-06 22:45:26 +08:00
aoiasd
cc7652327d
enhance: optimize jieba and lindera analyzer clone (#46719)
relate: https://github.com/milvus-io/milvus/issues/46718

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->

## Enhancement: Optimize Jieba and Lindera Analyzer Clone

**Core Invariant**: JiebaTokenizer and LinderaTokenizer must be
efficiently cloneable without lifetime constraints to support analyzer
composition in multi-language detection chains.

**What Logic Was Improved**:
- **JiebaTokenizer**: Replaced `Cow<'a, Jieba>` with
`Arc<jieba_rs::Jieba>` and removed the `<'a>` lifetime parameter. The
global JIEBA instance now wraps in Arc, enabling `#[derive(Clone)]` on
the struct. This eliminates lifetime management complexity while
maintaining zero-copy sharing via atomic reference counting.
- **LinderaTokenizer**: Introduced public `LinderaSegmenter` struct
encapsulating dictionary and mode state, and implemented explicit
`Clone` that properly duplicates the segmenter (cloning Arc-wrapped
dictionary), applies `box_clone()` to each boxed token filter, and
clones the token buffer. Previously, Clone was either unavailable or
incompletely handled trait objects.

**Why Previous Implementation Was Limiting**:
- The `Cow::Borrowed` pattern for JiebaTokenizer created explicit
lifetime dependencies that prevented straightforward `#[derive(Clone)]`.
Switching to Arc eliminates borrow checker constraints while providing
the same reference semantics for immutable shared state.
- LinderaTokenizer's token filters are boxed trait objects, which cannot
be auto-derived. Manual Clone implementation with `box_clone()` calls
correctly handles polymorphic filter duplication.

**No Data Loss or Behavior Regression**:
- Arc cloning is semantically equivalent to `Cow::Borrowed` for
read-only access; both efficiently share the underlying Jieba instance
and Dictionary without data duplication.
- The explicit Clone preserves all tokenizer state: segmenter (with
shared Arc dictionary), all token filters (via individual box_clone),
and the token buffer used during tokenization.
- Token stream behavior unchanged—segmentation and filter application
order remain identical.
- New benchmarks (`bench_jieba_tokenizer_clone`,
`bench_lindera_tokenizer_clone`) measure and validate clone performance
for both tokenizers.

<!-- end of auto-generated comment: release notes by coderabbit.ai -->

---------

Signed-off-by: aoiasd <zhicheng.yue@zilliz.com>
2026-01-06 21:19:25 +08:00
Zhen Ye
c7b5c23ff6
enhance: filter the empty timetick from consuming side (#46541)
issue: #46540

Empty timetick is just used to sync up the time clock between different
component in milvus. So empty timetick can be ignored if we achieve the
lsn/mvcc semantic for timetick. Currently, some components need the
empty timetick to trigger some operation, such as flush/tsafe. So we
only slow down the empty time tick for 5 seconds.

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: with LSN/MVCC semantics consumers only need (a) the
first timetick that advances the latest-required-MVCC to unblock
MVCC-dependent waits and (b) occasional periodic timeticks (~≤5s) for
clock synchronization—therefore frequent non-persisted empty timeticks
can be suppressed without breaking MVCC correctness.
- Logic removed/simplified: per-message dispatch/consumption of frequent
non-persisted empty timeticks is suppressed — an MVCC-aware filter
emptyTimeTickSlowdowner (internal/util/pipeline/consuming_slowdown.go)
short-circuits frequent empty timeticks in the stream pipeline
(internal/util/pipeline/stream_pipeline.go), and the WAL flusher
rate-limits non-persisted timetick dispatch to one emission per ~5s
(internal/streamingnode/server/flusher/flusherimpl/wal_flusher.go); the
delegator exposes GetLatestRequiredMVCCTimeTick to drive the filter
(internal/querynodev2/delegator/delegator.go).
- Why this does NOT introduce data loss or regressions: the slowdowner
always refreshes latestRequiredMVCCTimeTick via
GetLatestRequiredMVCCTimeTick and (1) never filters timeticks <
latestRequiredMVCCTimeTick (so existing tsafe/flush waits stay
unblocked) and (2) always lets the first timetick ≥
latestRequiredMVCCTimeTick pass to notify pending MVCC waits;
separately, WAL flusher suppression applies only to non-persisted
timeticks and still emits when the 5s threshold elapses, preserving
periodic clock-sync messages used by flush/tsafe.
- Enhancement summary (where it takes effect): adds
GetLatestRequiredMVCCTimeTick on ShardDelegator and
LastestMVCCTimeTickGetter, wires emptyTimeTickSlowdowner into
NewPipelineWithStream (internal/util/pipeline), and adds WAL flusher
rate-limiting + metrics
(internal/streamingnode/server/flusher/flusherimpl/wal_flusher.go,
pkg/metrics) to reduce CPU/dispatch overhead while keeping MVCC
correctness and periodic synchronization.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

---------

Signed-off-by: chyezh <chyezh@outlook.com>
2026-01-06 20:53:24 +08:00
wei liu
dc18d2aa8a
fix: Use leaderID for LeaderAction stale check in scheduler (#46785)
issue: #46737

PR #46440 refactored checkStale to use action.Node() for all action
types, which breaks LeaderAction stale checking. For LeaderAction,
Node() returns the worker node where segment resides, but the task is
executed on the leader node (delegator).

When syncing segments from RO worker nodes to a RW delegator, using
action.Node() incorrectly marks the task as stale because the worker is
RO, even though the leader is RW and the task should proceed.

This fix:
- Uses leaderID instead of Node() for LeaderAction stale checking
- Adds detailed comments explaining the distinction
- Adds unit tests covering the RO worker with RW leader scenario

Signed-off-by: Wei Liu <wei.liu@zilliz.com>
2026-01-06 18:27:24 +08:00
yihao.dai
fc2db1a674
enhance: Valid row count result and avoid misleading warn log for sort compaction (#46746)
issue: https://github.com/milvus-io/milvus/issues/46743

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->

## Summary

**Core Invariant:** Sort compaction tasks must not be created
concurrently for the same segment. The system relies on atomic
check-and-set semantics to prevent duplicate task creation.

**What Logic is Improved:** The code now guards sort compaction task
creation with an explicit `CheckAndSetSegmentsCompacting` check before
calling `createSortCompactionTask`. Previously, tasks could be attempted
for segments already undergoing compaction, triggering warning logs that
incorrectly suggested task creation failures. The fix skips task
creation when a segment is already compacting, avoiding these misleading
warnings entirely.

**Why No Data Loss or Regression:** 
- The `CheckAndSetSegmentsCompacting` method atomically checks whether a
segment is already being compacted and only proceeds if it's not; this
is the correct guard pattern for preventing concurrent compactions
- When a segment is already compacting (`isCompacting == true`), the
code correctly increments the done counter and skips to the next
segment, which is the intended behavior (no wasted task creation
attempts)
- The function signature change to `createSortCompactionTask` adds only
an internal parameter (the current task context for logging); no public
APIs are affected
- Logging refactoring maintains semantic equivalence while providing
task-scoped context

**Concrete Fix:** The misleading warning during sort compaction is
eliminated by preventing task creation attempts for already-compacting
segments through the mutex-protected `CheckAndSetSegmentsCompacting`
guard, rather than attempting creation and failing downstream.

<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
2026-01-06 16:33:25 +08:00
aoiasd
ee216877bb
enhance: support compaction with file resource in ref mode (#46399)
Add support for DataNode compaction using file resources in ref mode.
SortCompation and StatsJobs will build text indexes, which may use file
resources.
relate: https://github.com/milvus-io/milvus/issues/43687

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: file resources (analyzer binaries/metadata) are only
fetched, downloaded and used when the node is configured in Ref mode
(fileresource.IsRefMode via CommonCfg.QNFileResourceMode /
DNFileResourceMode); Sync now carries a version and managers track
per-resource versions/resource IDs so newer resource sets win and older
entries are pruned (RefManager/SynchManager resource maps).
- Logic removed / simplified: component-specific FileResourceMode flags
and an indirection through a long-lived BinlogIO wrapper were
consolidated — file-resource mode moved to CommonCfg, Sync/Download APIs
became version- and context-aware, and compaction/index tasks accept a
ChunkManager directly (binlog IO wrapper creation inlined). This
eliminates duplicated config checks and wrapper indirection while
preserving the same chunk/IO semantics.
- Why no data loss or behavior regression: all file-resource code paths
are gated by the configured mode (default remains "sync"); when not in
ref-mode or when no resources exist, compaction and stats flows follow
existing code paths unchanged. Versioned Sync + resourceID maps ensure
newly synced sets replace older ones and RefManager prunes stale files;
GetFileResources returns an error if requested IDs are missing (prevents
silent use of wrong resources). Analyzer naming/parameter changes add
analyzer_extra_info but default-callers pass "" so existing analyzers
and index contents remain unchanged.
- New capability: DataNode compaction and StatsJobs can now build text
indexes using external file resources in Ref mode — DataCoord exposes
GetFileResources and populates CompactionPlan.file_resources;
SortCompaction/StatsTask download resources via fileresource.Manager,
produce an analyzer_extra_info JSON (storage + resource->id map) via
analyzer.BuildExtraResourceInfo, and propagate analyzer_extra_info into
BuildIndexInfo so the tantivy bindings can load custom analyzers during
text index creation.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: aoiasd <zhicheng.yue@zilliz.com>
2026-01-06 16:31:31 +08:00
Chun Han
b7ee93fc52
feat: support query aggregtion(#36380) (#44394)
related: #36380

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: aggregation is centralized and schema-aware — all
aggregate functions are created via the exec Aggregate registry
(milvus::exec::Aggregate) and validated by ValidateAggFieldType, use a
single in-memory accumulator layout (Accumulator/RowContainer) and
grouping primitives (GroupingSet, HashTable, VectorHasher), ensuring
consistent typing, null semantics and offsets across planner → exec →
reducer conversion paths (toAggregateInfo, Aggregate::create,
GroupingSet, AggResult converters).

- Removed / simplified logic: removed ad‑hoc count/group-by and reducer
code (CountNode/PhyCountNode, GroupByNode/PhyGroupByNode, cntReducer and
its tests) and consolidated into a unified AggregationNode →
PhyAggregationNode + GroupingSet + HashTable execution path and
centralized reducers (MilvusAggReducer, InternalAggReducer,
SegcoreAggReducer). AVG now implemented compositionally (SUM + COUNT)
rather than a bespoke operator, eliminating duplicate implementations.

- Why this does NOT cause data loss or regressions: existing data-access
and serialization paths are preserved and explicitly validated —
bulk_subscript / bulk_script_field_data and FieldData creation are used
for output materialization; converters (InternalResult2AggResult ↔
AggResult2internalResult, SegcoreResults2AggResult ↔
AggResult2segcoreResult) enforce shape/type/row-count validation; proxy
and plan-level checks (MatchAggregationExpression,
translateOutputFields, ValidateAggFieldType, translateGroupByFieldIds)
reject unsupported inputs (ARRAY/JSON, unsupported datatypes) early.
Empty-result generation and explicit error returns guard against silent
corruption.

- New capability and scope: end-to-end GROUP BY and aggregation support
added across the stack — proto (plan.proto, RetrieveRequest fields
group_by_field_ids/aggregates), planner nodes (AggregationNode,
ProjectNode, SearchGroupByNode), exec operators (PhyAggregationNode,
PhyProjectNode) and aggregation core (Aggregate implementations:
Sum/Count/Min/Max, SimpleNumericAggregate, RowContainer, GroupingSet,
HashTable) plus proxy/querynode reducers and tests — enabling grouped
and global aggregation (sum, count, min, max, avg via sum+count) with
schema-aware validation and reduction.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: MrPresent-Han <chun.han@gmail.com>
Co-authored-by: MrPresent-Han <chun.han@gmail.com>
2026-01-06 16:29:25 +08:00
Zhen Ye
4c6e33f326
fix: lost tenant/namespace support for pulsar since 2.6 (#46752)
issue: #46748

Signed-off-by: chyezh <chyezh@outlook.com>
2026-01-06 14:33:24 +08:00
congqixia
fa2c3c404c
enhance: Forbid writing V1 format and always use StorageV2 (#46791)
Related to #46595

Remove the EnableStorageV2 config option and enforce StorageV2 format
across all write paths including compaction, import, write buffer, and
streaming segment allocation. V1 format write tests are now skipped as
writing V1 format is no longer supported.

---------

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
2026-01-06 11:55:23 +08:00
Zhen Ye
56e82c78e1
fix: add load config watcher to avoid load config modification lost (#46784)
issue: #46778

Signed-off-by: chyezh <chyezh@outlook.com>
2026-01-06 11:33:24 +08:00
wei liu
5f2e430941
enhance: Add channel-based node blacklist for LB policy retry (#46091)
issue: #46090
This change introduces a global node blacklist mechanism to immediately
cut off query traffic to failed delegators across all concurrent
requests.

Key features:
- Introduce ChannelBlacklist to track failed delegator nodes per channel
- When a query fails, the node is immediately blacklisted and excluded
from ALL subsequent requests (not just retries within the same request)
- Blacklisted nodes are automatically excluded during node selection
- Entries expire after configurable duration (default 30s) to allow
automatic recovery when nodes become healthy again
- Background cleanup loop removes expired entries periodically
- Add proxy.replicaBlacklistDuration and
proxy.replicaBlacklistCleanupInterval configuration parameters
- Blacklist can be disabled by setting duration to 0

Before this change:
- Failed nodes were only excluded within the same request's retry loop
- Concurrent requests would still attempt to query the failed node
- Each request had to experience its own failure before avoiding the
node

After this change:
- Once a node fails, it is immediately excluded from all requests
- New requests arriving during the blacklist period will skip the failed
node without experiencing any failure
- This significantly reduces latency spikes during node failures

Signed-off-by: Wei Liu <wei.liu@zilliz.com>
2026-01-06 11:01:29 +08:00
junjiejiangjjj
f4e459cbf7
fix: Fix function edit interface bug (#46781)
https://github.com/milvus-io/milvus/issues/46780

Signed-off-by: junjie.jiang <junjie.jiang@zilliz.com>
2026-01-06 10:57:24 +08:00
Guanghui Huang
dc66f3fa04
fix: Fix rate limit not work when set collection rate limit properties (#46714)
issue: https://github.com/milvus-io/milvus/issues/46713
/kind bug

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: a Limiter's hasUpdated flag controls whether its
non-Infinite limit is exported to proxies via toRequestLimiter(); only
limiters with hasUpdated==true and non-Inf rates produce rate updates
delivered to proxies (pkg/util/ratelimitutil/limiter.go: HasUpdated /
toRequestLimiter behavior unchanged).

- Exact bug and fix (issue #46713): collection-level limiters created
from configured collection/partition/database properties were
constructed with correct limits but left hasUpdated==false, so they were
skipped by the existing !HasUpdated() check and never sent to proxies.
Fix: add Limiter.SetHasUpdated(updated bool) and call a new
updateLimiterHasUpdated helper immediately after creating limiter nodes
during initialization/reset (internal/rootcoord/quota_center.go) to mark
non-Inf newly-created limiters as updated so they are included in
toRequestLimiter exports.

- Logic simplified / redundancy removed: initialization now explicitly
sets limiter initialization state (hasUpdated) for newly-created
non-Infinite limiters instead of relying on implicit later side-effects
to toggle the flag; this removes the implicit gap between creation and
the expectation that a configured limiter should be published.

- No data-loss or behavior regression: the change only mutates the
in-memory hasUpdated flag for freshly created limiter instances
(pkg/util/ratelimitutil/limiter.go: SetHasUpdated) and sets it in the
limiter initialization path (internal/rootcoord/quota_center.go). It
does not alter token accounting (advance, AllowN, Cancel), rate
computation, SetLimit semantics, persistence, or proxy filtering
logic—only ensures intended collection-level rates are delivered to
proxies—so no persisted data or runtime rate behavior is removed or
degraded.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: guanghuihuang <guanghuihuang@didiglobal.com>
Co-authored-by: guanghuihuang <guanghuihuang@didiglobal.com>
2026-01-06 10:21:24 +08:00
wei liu
975c91df16
feat: Add comprehensive snapshot functionality for collections (#44361)
issue: #44358

Implement complete snapshot management system including creation,
deletion, listing, description, and restoration capabilities across all
system components.

Key features:
- Create snapshots for entire collections
- Drop snapshots by name with proper cleanup
- List snapshots with collection filtering
- Describe snapshot details and metadata

Components added/modified:
- Client SDK with full snapshot API support and options
- DataCoord snapshot service with metadata management
- Proxy layer with task-based snapshot operations
- Protocol buffer definitions for snapshot RPCs
- Comprehensive unit tests with mockey framework
- Integration tests for end-to-end validation

Technical implementation:
- Snapshot metadata storage in etcd with proper indexing
- File-based snapshot data persistence in object storage
- Garbage collection integration for snapshot cleanup
- Error handling and validation across all operations
- Thread-safe operations with proper locking mechanisms

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant/assumption: snapshots are immutable point‑in‑time
captures identified by (collection, snapshot name/ID); etcd snapshot
metadata is authoritative for lifecycle (PENDING → COMMITTED → DELETING)
and per‑segment manifests live in object storage (Avro / StorageV2). GC
and restore logic must see snapshotRefIndex loaded
(snapshotMeta.IsRefIndexLoaded) before reclaiming or relying on
segment/index files.

- New capability added: full end‑to‑end snapshot subsystem — client SDK
APIs (Create/Drop/List/Describe/Restore + restore job queries),
DataCoord SnapshotWriter/Reader (Avro + StorageV2 manifests),
snapshotMeta in meta, SnapshotManager orchestration
(create/drop/describe/list/restore), copy‑segment restore
tasks/inspector/checker, proxy & RPC surface, GC integration, and
docs/tests — enabling point‑in‑time collection snapshots persisted to
object storage and restorations orchestrated across components.

- Logic removed/simplified and why: duplicated recursive
compaction/delta‑log traversal and ad‑hoc lookup code were consolidated
behind two focused APIs/owners (Handler.GetDeltaLogFromCompactTo for
delta traversal and SnapshotManager/SnapshotReader for snapshot I/O).
MixCoord/coordinator broker paths were converted to thin RPC proxies.
This eliminates multiple implementations of the same traversal/lookup,
reducing divergence and simplifying responsibility boundaries.

- Why this does NOT introduce data loss or regressions: snapshot
create/drop use explicit two‑phase semantics (PENDING → COMMIT/DELETING)
with SnapshotWriter writing manifests and metadata before commit; GC
uses snapshotRefIndex guards and
IsRefIndexLoaded/GetSnapshotBySegment/GetSnapshotByIndex checks to avoid
removing referenced files; restore flow pre‑allocates job IDs, validates
resources (partitions/indexes), performs rollback on failure
(rollbackRestoreSnapshot), and converts/updates segment/index metadata
only after successful copy tasks. Extensive unit and integration tests
exercise pending/deleting/GC/restore/error paths to ensure idempotence
and protection against premature deletion.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

---------

Signed-off-by: Wei Liu <wei.liu@zilliz.com>
2026-01-06 10:15:24 +08:00
yanliang567
9996e8d1ce
test: Update error msg for search by ids tests (#46792)
related issue: #46789

Signed-off-by: yanliang567 <yanliang.qiao@zilliz.com>
2026-01-05 20:09:24 +08:00
congqixia
01da5010f5
enhance: Add MAP_POPULATE flag for mmap to reduce page faults (#46761)
Add configurable MAP_POPULATE flag support for mmap operations to reduce
page faults and improve first read performance.

Key changes:
- Add `queryNode.mmap.populate` config (default: true) to control
MAP_POPULATE flag usage
- Add `mmap_populate` parameter to MmapChunkTarget, ChunkTranslator,
GroupChunkTranslator, and ManifestGroupTranslator
- Apply MAP_POPULATE to both MmapChunkTarget and MemChunkTarget
- Propagate mmap_populate setting through chunk creation pipeline

When enabled, MAP_POPULATE pre-faults the mapped pages into memory,
eliminating page faults during subsequent access and improving query
performance for the first read operations.

issue: #46760

---------

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
2026-01-05 18:57:24 +08:00
jiamingli-maker
d0e6a624a7
test: Skip HNSW_PRQ test cases (#46771)
/kind improvement
Skipping HNSW_PRQ tests due to index creation timeout.

Signed-off-by: zilliz <jiaming.li@zilliz.com>
2026-01-05 18:55:24 +08:00
tianhang
4f8a8dd7ef
fix: Register all sealed segments’ BM25 stats in IDFOracle (#46726)
If QueryNode loads multiple sealed segments with BM25 enabled, BM25
stats registration into IDFOracle could stop after the first segment due
to an early-terminating ConcurrentMap.Range callback. This change:
Register BM25 stats for all sealed segments by continuing iteration
(return true) during sealed-segment load
Prevent repeated warnings like idf oracle lack some sealed segment
Ensure IDF/BM25 statistics are not silently incomplete (improving BM25
ranking correctness)
issue: #46725 
Core invariant: for any BM25-enabled collection, every loaded sealed
segment with available BM25 stats must be registered into IDFOracle, so
SyncDistribution can always find the sealed segments present in the
distribution snapshot.
Bug fix: ConcurrentMap.Range respects the callback’s boolean return;
returning false stops iteration. The sealed BM25 stats registration
callback previously returned false, which could register only the first
sealed segment and leave the rest missing—causing IDFOracle to warn idf
oracle lack some sealed segment and potentially compute IDF from
incomplete stats. Fixed by returning true to continue iterating and
registering all segments.
No behavior regression: the change only affects the sealed-segment BM25
stats registration loop; it does not alter segment loading, distribution
snapshot generation, or non-BM25 codepaths. For collections without BM25
(or when BM25 stats are nil), behavior remains unchanged.

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: For any BM25-enabled collection, every loaded sealed
segment with available BM25 stats must be registered into IDFOracle so
SyncDistribution can discover them in distribution snapshots.

- Bug fix (links to #46725): The BM25 stats registration callback used
with bm25Stats.Range() in loadStreamDelete() returned false, which
prematurely stopped iteration after the first sealed segment and left
subsequent sealed segments unregistered. The fix changes the callback to
return true so the Range loop completes and registers BM25 stats for all
sealed segments.

- Logic simplified/removed: The early-return (false) in the
ConcurrentMap.Range callback that aborted further registrations has been
removed (replaced by returning true). That early abort was redundant and
incorrect because registration must proceed for every entry; allowing
Range to continue restores the intended one-to-many registration
behavior.

- No data loss or regression: The change is narrowly scoped to the
sealed-segment BM25 stats registration loop in
internal/querynodev2/delegator/delegator_data.go and does not modify
segment loading, distribution snapshot generation, growing-segment
handling, or non-BM25 codepaths. Returning true only permits full
iteration and registration; it does not delete or alter existing data
structures or load state, so IDF/BM25 statistics become complete without
changing other behaviors.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: thangTang <tangtianhang099@gmail.com>
2026-01-05 17:41:24 +08:00
XuanYang-cn
52a05831b5
enhance: Tidy build index logs (#46531)
Signed-off-by: yangxuan <xuan.yang@zilliz.com>
2026-01-05 15:27:24 +08:00
lif
65cca5d046
fix: correct typo CredentialSeperator to CredentialSeparator (#46631)
issue: #46635

## Summary
- Fix spelling error in constant name: `CredentialSeperator` ->
`CredentialSeparator`
- Updated all usages across the codebase to use the correct spelling

## Changes
- `pkg/util/constant.go`: Renamed the constant
- `pkg/util/contextutil/context_util.go`: Updated usage
- `pkg/util/contextutil/context_util_test.go`: Updated usage
- `internal/proxy/authentication_interceptor.go`: Updated usage
- `internal/proxy/util.go`: Updated usage
- `internal/proxy/util_test.go`: Updated usage
- `internal/proxy/trace_log_interceptor_test.go`: Updated usage
- `internal/proxy/accesslog/info/util.go`: Updated usage
- `internal/distributed/proxy/service.go`: Updated usage
- `internal/distributed/proxy/httpserver/utils.go`: Updated usage

## Test Plan
- [x] All references updated consistently
- [x] No functional changes - only constant name spelling correction

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: the separator character for credentials remains ":"
everywhere — only the exported identifier was renamed from
CredentialSeperator → CredentialSeparator; the constant value and
split/join semantics are unchanged.
- Change (bug fix): corrected the misspelled exported constant in
pkg/util/constant.go and updated all references across the codebase
(parsing, token construction, header handling and tests) to use the new
identifier; this is an identifier rename that removes an inconsistent
symbol and prevents compile-time/reference errors.
- Logic simplified/redundant work removed: no runtime logic was removed;
the simplification is purely maintenance-focused — eliminating a
misspelled exported name that could cause developers to introduce
duplicate or incorrect constants.
- No data loss or behavior regression: runtime code paths are unchanged
— e.g., GetAuthInfoFromContext, ParseUsernamePassword,
AuthenticationInterceptor, proxy service token construction and
access-log extraction still use ":" to split/join credentials; updated
and added unit tests (parsing and metadata extraction) exercise these
paths and validate identical semantics.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

---------

Signed-off-by: majiayu000 <1835304752@qq.com>
Signed-off-by: lif <1835304752@qq.com>
Co-authored-by: Claude Opus 4.5 <noreply@anthropic.com>
2026-01-05 14:37:24 +08:00
Li Liu
69083617b6
enhance: improve search by PK with duplicate check and auto anns_field inference (#46744)
issue: #46740

  - Add duplicate ID check before query execution (fail fast)
  - Auto infer anns_field when only one vector field exists in schema

Signed-off-by: Li Liu <li.liu@zilliz.com>
2026-01-05 14:07:23 +08:00
yanliang567
7018151c7d
test: Add tests for search by ids (#46756)
related issue: #46755

Signed-off-by: yanliang567 <yanliang.qiao@zilliz.com>
2026-01-05 13:25:23 +08:00
sijie-ni-0214
941c6eaed7
fix: text log loading failure during cluster upgrade (#46697)
issue: https://github.com/milvus-io/milvus/issues/46678

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: Text index log keys are canonicalized at KV
(serialization) boundaries — etcd stores compressed filename-only
entries, while in-memory and runtime consumers must receive full
object-storage keys so Datanode/QueryNode can load text indexes
directly.

- Logic removed/simplified: ad-hoc reconstruction of full text-log paths
scattered across components (garbage_collector.getTextLogs,
querynodev2.LoadTextIndex, compactor/index task code) was removed;
consumers now use TextIndexStats.Files as-provided (full keys). Path
compression/decompression was centralized into KV marshal/unmarshal
utilities (metautil.ExtractTextLogFilenames in marshalSegmentInfo and
metautil.BuildTextLogPaths in kv_catalog.listSegments), eliminating
redundant, inconsistent prefix-rebuilding logic that broke during
rolling upgrades.

- Why this does NOT cause data loss or regressions: before persist,
marshalSegmentInfo compresses TextStatsLogs.Files to filenames
(metautil.ExtractTextLogFilenames) so stored KV remains compact; on
load, kv_catalog.listSegments calls metautil.BuildTextLogPaths to
restore full paths and includes compatibility logic that leaves
already-full keys unchanged. Thus every persisted filename is
recoverable to a valid full key and consumers receive correct full paths
(see marshalSegmentInfo → KV write path and kv_catalog.listSegments →
reload path), preventing dropped or malformed keys.

- Bug fix (refs #46678): resolves text-log loading failures during
cluster upgrades by centralizing path handling at KV encode/decode and
removing per-component path reconstruction — the immediate fix is
changing consumers to read TextIndexStats.Files directly and relying on
marshal/unmarshal to perform compression/expansion, preventing
mixed-format failures during rolling upgrades.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

---------

Signed-off-by: sijie-ni-0214 <sijie.ni@zilliz.com>
2026-01-05 11:19:24 +08:00
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
jiamingli-maker
c10cf53b4b
test: Add HNSW_PRQ test cases and fix HNSW_PQ (#46680)
/kind improvement

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: index parameter validation and test expectations for
the HNSW-family must be explicit, consistent, and deterministic — this
PR enforces that by adding exhaustive parameter matrices for HNSW_PRQ
(tests/python_client/testcases/indexes/{idx_hnsw_prq.py,
test_hnsw_prq.py}) and normalizing expectations in idx_hnsw_pq.py via a
shared success variable.
- Logic removed / simplified: brittle, ad-hoc string expectations were
consolidated — literal "success" occurrences were replaced with a single
success variable and ambiguous short error messages were replaced by the
canonical descriptive error text; this reduces duplicated assertion
logic in tests and removes dependence on fragile, truncated messages.
- Bug fix (tests): corrected HNSW_PQ test expectations to assert the
full, authoritative error for invalid PQ m ("The dimension of the vector
(dim) should be a multiple of the number of subquantizers (m).") and
aligned HNSW_PRQ test matrices (idx_hnsw_prq.py) to the same explicit
expectations — the change targets test assertions only and fixes false
negatives caused by mismatched messages.
- No data loss or behavior regression: only test code is added/modified
(tests/python_client/testcases/indexes/*). Production code paths remain
unmodified — collection creation, insert/flush, client.create_index,
wait_for_index_ready, load_collection, search, and client.describe_index
are invoked by tests but not changed; therefore persisted data, index
artifacts, and runtime behavior are unaffected.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: zilliz <jiaming.li@zilliz.com>
2026-01-04 18:57:22 +08:00
XuanYang-cn
146f173eea
enhance: tidy code in import plugin context (#46374)
Signed-off-by: yangxuan <xuan.yang@zilliz.com>
2026-01-04 16:51:23 +08:00
Zhen Ye
27525d57cc
enhance: add glog sink to transfer cgo log into zap (#46721)
issue: #45640

- After async logging, the C log and go log has no order promise,
meanwhile the C log format is not consistent with Go Log; so we close
the output of glog, just forward the log result operation into Go side
which will be handled by the async zap logger.
- Use CGO to filter all cgo logging and promise the order between c log
and go log.
- Also fix the metric name, add new metric to count the logging.
- TODO: after woodpecker use the logger of milvus, we can add bigger
buffer for logging.

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: all C (glog) and Go logs must be routed through the
same zap async pipeline so ordering and formatting are preserved; this
PR ensures every glog emission is captured and forwarded to zap before
any async buffering diverges the outputs.

- Logic removed/simplified: direct glog outputs and hard
stdout/stderr/log_dir settings are disabled (configs/glog.conf and flags
in internal/core/src/config/ConfigKnowhere.cpp) because they are
redundant once a single zap sink handles all logs; logging metrics were
simplified from per-length/volatile gauges to totalized counters
(pkg/metrics/logging_metrics.go & pkg/log/*), removing duplicate
length-tracking and making accounting consistent.

- No data loss or behavior regression (concrete code paths): Google
logging now adds a GoZapSink (internal/core/src/common/logging_c.h,
logging_c.cpp) that calls the exported CGO bridge goZapLogExt
(internal/util/cgo/logging/logging.go). Go side uses
C.GoStringN/C.GoString to capture full message and file, maps glog
severities to zapcore levels, preserves caller info, and writes via the
existing zap async core (same write path used by Go logs). The C++
send() trims glog's trailing newline and forwards exact buffers/lengths,
so message content, file, line, and severity are preserved and
serialized through the same async writer—no log entries are dropped or
reordered relative to Go logs.

- Capability added (where it takes effect): a CGO bridge that forwards
glog into zap—new Go-exported function goZapLogExt
(internal/util/cgo/logging/logging.go), a GoZapSink in C++ that forwards
glog sends (internal/core/src/common/logging_c.h/.cpp), and blank
imports of the cgo initializer across multiple packages (various
internal/* files) to ensure the bridge is registered early so all C logs
are captured.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: chyezh <chyezh@outlook.com>
2026-01-04 14:45:23 +08:00
sre-ci-robot
c585fbac72
[automated] Bump milvus version to v2.6.8 (#46732)
Bump milvus version to v2.6.8
Signed-off-by: sre-ci-robot sre-ci-robot@users.noreply.github.com

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->

**Core Invariant**: This PR assumes the Milvus v2.6.8 Docker image is a
stable, compatible release that can transparently replace v2.6.7 in
standalone embed configurations without breaking backward compatibility.

**What Changed**: Updated the Milvus Docker image tag from `v2.6.7` to
`v2.6.8` in two standalone embedding configuration scripts:
- `scripts/standalone_embed.bat` (line 83)
- `scripts/standalone_embed.sh` (line 62)

**Why This Is Safe**: These scripts only specify the container image
version and pass through pre-existing configuration files
(`embedEtcd.yaml`, `user.yaml`) to the container. No local logic, data
schemas, or API contracts are modified—the container startup behavior
remains identical, just pulling a newer upstream image tag. Version
increments within the same major.minor series (v2.6.x) follow semantic
versioning conventions ensuring no breaking changes.

**Impact**: Users pulling or running these standalone embed scripts will
automatically use the newer v2.6.8 Milvus release, receiving bug fixes
and enhancements from the patch version bump while maintaining
compatible behavior with existing configurations.

<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Co-authored-by: github-actions[bot] <41898282+github-actions[bot]@users.noreply.github.com>
2026-01-04 12:01:22 +08:00
sre-ci-robot
e669ca91c4
[automated] Bump milvus version to v2.6.8 (#46722)
Bump milvus version to v2.6.8
Signed-off-by: sre-ci-robot sre-ci-robot@users.noreply.github.com

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->

## Version Bump Summary: Milvus v2.6.7 → v2.6.8

**Core Invariant**: This PR assumes v2.6.8 is a drop-in replacement for
v2.6.7, maintaining API compatibility and deployment configuration
compatibility across all environments (standalone, GPU, and cluster
deployments).

**Scope of Changes**: Updates Docker image version references from
`milvusdb/milvus:v2.6.7` to `milvusdb/milvus:v2.6.8` across all
deployment documentation and configuration files:
- Binary deployment README
- Standalone docker-compose.yml (CPU variant)
- GPU standalone docker-compose.yml
- Cluster distributed deployment inventory.ini

**No Behavior Regression Risk**: Since this modifies only external
artifact references (Docker image tags in deployment configs and
documentation examples), not any runtime logic or data schemas, there is
zero risk of data loss or operational regression. The semantic
versioning convention (patch-level bump: v2.6.7 → v2.6.8) indicates this
is a maintenance release with backward compatibility preserved.

**Automation Context**: This is an automated version bump by
sre-ci-robot, indicating a routine dependency update process rather than
manual configuration changes.

<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Co-authored-by: github-actions[bot] <41898282+github-actions[bot]@users.noreply.github.com>
2026-01-04 10:37:28 +08:00
congqixia
92c0c38e24
fix: validate collection TTL property to prevent compaction stuck (#46717)
If collection TTL property is malformed (e.g., non-numeric value),
compaction tasks would fail silently and get stuck. This change:

- Add centralized GetCollectionTTL/GetCollectionTTLFromMap functions in
pkg/common to handle TTL parsing with proper error handling
- Validate TTL property in createCollectionTask and alterCollectionTask
PreExecute to reject invalid values early
- Refactor datacoord compaction policies to use the new common functions
- Remove duplicated getCollectionTTL from datacoord/util.go

issue: #46716

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: collection.ttl.seconds must be a parseable int64 and
validated at collection creation/alter time so malformed TTLs never
reach compaction/execution codepaths.
- Bug fix (resolves #46716): malformed/non-numeric TTLs could silently
cause compaction tasks to fail/stall; fixed by adding centralized
parsing helpers pkg/common.GetCollectionTTL and GetCollectionTTLFromMap
and validating TTL in createCollectionTask.PreExecute and
alterCollectionTask.PreExecute (calls with default -1 and return
parameter-invalid errors on parse failure).
- Simplification / removed redundancy: eliminated duplicated
getCollectionTTL in internal/datacoord/util.go and replaced ad-hoc TTL
parsing across datacoord (compaction policies, import_util, compaction
triggers) and proxy util with the common helpers, centralizing error
handling and defaulting logic.
- No data loss or behavior regression: valid TTL parsing semantics
unchanged (helpers use identical int64 parsing and default fallback from
paramtable/CommonCfg); validation occurs in PreExecute so existing valid
collections proceed unchanged while malformed values are rejected
early—compaction codepaths now receive only validated TTL values (or
explicit defaults), preventing silent skips without altering valid
execution flows.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

---------

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
2026-01-01 08:13:22 +08:00
Zhen Ye
bb913dd837
fix: simplify go ut (#46606)
issue: #46500

- simplify the run_go_codecov.sh to make sure the set -e to protect any
sub command failure.
- remove all embed etcd in test to make full test can be run at local.

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->

## PR Summary: Simplify Go Unit Tests by Removing Embedded etcd and
Async Startup Scaffolding

**Core Invariant:**
This PR assumes that unit tests can be simplified by running without
embedded etcd servers (delegating to environment-based or external etcd
instances via `kvfactory.GetEtcdAndPath()` or `ETCD_ENDPOINTS`) and by
removing goroutine-based async startup scaffolding in favor of
synchronous component initialization. Tests remain functionally
equivalent while becoming simpler to run and debug locally.

**What is Removed or Simplified:**

1. **Embedded etcd test infrastructure deleted**: Removes
`EmbedEtcdUtil` type and its public methods (SetupEtcd,
TearDownEmbedEtcd) from `pkg/util/testutils/embed_etcd.go`, removes the
`StartTestEmbedEtcdServer()` helper from `pkg/util/etcd/etcd_util.go`,
and removes etcd embedding from test suites (e.g., `TaskSuite`,
`EtcdSourceSuite`, `mixcoord/client_test.go`). Tests now either skip
etcd-dependent tests (via `MILVUS_UT_WITHOUT_KAFKA=1` environment flag
in `kafka_test.go`) or source etcd from external configuration (via
`kvfactory.GetEtcdAndPath()` in `task_test.go`, or `ETCD_ENDPOINTS`
environment variable in `etcd_source_test.go`). This eliminates the
overhead of spinning up temporary etcd servers for unit tests.

2. **Async startup scaffolding replaced with synchronous
initialization**: In `internal/proxy/proxy_test.go` and
`proxy_rpc_test.go`, the `startGrpc()` method signature removes the
`sync.WaitGroup` parameter; components are now created, prepared, and
run synchronously in-place rather than in goroutines (e.g., `go
testServer.startGrpc(ctx, &p)` becomes `testServer.startGrpc(ctx, &p)`
running synchronously). Readiness checks (e.g., `waitForGrpcReady()`)
remain in place to ensure startup safety without concurrency constructs.
This simplifies control flow and reduces debugging complexity.

3. **Shell script orchestration unified with proper error handling**: In
`scripts/run_go_codecov.sh` and `scripts/run_intergration_test.sh`,
per-package inline test invocations are consolidated into a single
`test_cmd()` function with unified `TEST_CMD_WITH_ARGS` array containing
race, coverage, verbose, and other flags. The problematic `set -ex` is
replaced with `set -e` alone (removing debug output noise while
preserving strict error semantics), ensuring the scripts fail fast on
any command failure.

**Why No Regression:**
- Test assertions and code paths remain unchanged; only deployment
source of etcd (embedded → external) and startup orchestration (async →
sync) change.
- Readiness verification (e.g., `waitForGrpcReady()`) is retained,
ensuring components are initialized before test execution.
- Test flags (race detection, coverage, verbosity) are uniformly applied
across all packages via unified `TEST_CMD_WITH_ARGS`, preserving test
coverage and quality.
- `set -e` alone is sufficient for strict failure detection without the
`-x` flag's verbose output.

<!-- end of auto-generated comment: release notes by coderabbit.ai -->

---------

Signed-off-by: chyezh <chyezh@outlook.com>
2025-12-31 16:07:22 +08:00
aoiasd
d261034af6
enhance: fix unstable config util unit test (#46702)
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: config refresh events must reliably propagate updated
values and evict cached entries within a bounded time window; tests must
observe this deterministically without relying on fixed sleeps.
- Logic simplified: brittle fixed time.Sleep delays and separate error
assertions were replaced by assert.Eventually polling blocks that
combine value checks and cache-eviction verification, and consolidated
checks to reduce redundant assertions.
- Why no data loss / no behavior regression: only test synchronization
and assertions were changed—production config manager code paths (value
propagation, KV puts, cache eviction) are untouched; tests now wait for
the same outcomes more robustly, so no mutation of runtime behavior or
storage occurs.
- Enhancement scope: this is a test-stability improvement (no new
runtime capability); it fixes flaky unit tests (root cause: timing
assumptions) by replacing fixed waits with bounded polling and by using
t.Context for KV puts to align test context usage.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: aoiasd <zhicheng.yue@zilliz.com>
2025-12-31 15:59:21 +08:00
Zhen Ye
ca8740c7c0
fix: remove redundant log (#46695)
issue: #45841

- CPP log make the multi log line in one debug, remove the "\n\t".
- remove some log that make no sense.
- slow down some log like ChannelDistManager.

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: logging is purely observational — this PR only
reduces, consolidates, or reformats diagnostic output (removing
per-item/noise logs, consolidating batched logs, and converting
multi-line log strings) while preserving all control flow, return
values, and state mutations across affected code paths.

- Removed / simplified logic: deleted low-value per-operation debug/info
logs (e.g., ListIndexes, GetRecoveryInfo, GcConfirm,
push-to-reorder-buffer, several streaming/wal/debug traces), replaced
per-item inline logs with single batched deferred logs in
querynodev2/delegator (logExcludeInfo) and CleanInvalid, changed C++
PlanNode ToString() multi-line output to compact single-line bracketed
format (removed "\n\t"), and added thresholded interceptor logging
(InterceptorMetrics.ShouldBeLogged) and message-type-driven log levels
to avoid verbose entries.

- Why this does NOT cause data loss or behavioral regression: no
function signatures, branching, state updates, persistence calls, or
return values were changed — examples: ListIndexes still returns the
same Status/IndexInfos; GcConfirm still constructs and returns
resp.GetGcFinished(); Insert and CleanInvalid still perform the same
insert/removal operations (only their per-item logging was aggregated);
PlanNode ToString changes only affect emitted debug strings. All error
handling and control flow paths remain intact.

- Enhancement intent: reduce log volume and improve signal-to-noise for
debugging by removing redundant, noisy logs and emitting concise,
rate-/threshold-limited summaries while preserving necessary diagnostics
and original program behavior.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

---------

Signed-off-by: chyezh <chyezh@outlook.com>
2025-12-31 15:35:21 +08:00
wei liu
c2677967ad
fix: prevent empty segment list when partial result is enabled (#46670)
issue: #46669
When partial result is enabled (PartialResultRequiredDataRatio < 1.0),
the Serviceable() method would return true even if syncedByCoord is
false (by bypassing viewReady check). However, PinReadableSegments uses
GetLoadedRatio() == 1.0 to decide whether to filter segments by target
version.

This causes a problem: when loadedRatio == 1.0 but syncedByCoord ==
false, segments are filtered by an incorrect target version, resulting
in an empty segment list during search.

This change:
- Replace GetLoadedRatio() == 1.0 with Serviceable() check to ensure
target version filtering only happens after coord sync completes
- Remove partial result bypass in Serviceable() to keep the check
consistent

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->

## Bug Fix Summary

**Core Invariant**: `Serviceable()` must enforce a strict requirement
that both data loading AND coordinator synchronization are complete
before allowing full search operations. This prevents using stale or
uninitialized target versions.

**Logic Removed/Simplified**: 
- Removed the partial-result bypass from `Serviceable()` that previously
allowed it to return `true` even when `syncedByCoord == false`
- Replaced `GetLoadedRatio() == 1.0` checks in `PinReadableSegments`
with `Serviceable()` calls to ensure target-version filtering only
occurs after coord sync completes
- Simplified the serviceability condition from parameterized
partial-result logic to a direct conjunction: `loadedRatio >= 1.0 AND
syncedByCoord == true`

**No Data Loss or Regression**: The change is safe because:
- When `Serviceable()` returns `true` (both loadedRatio ≥ 1.0 AND
syncedByCoord ≥ true), segments are filtered by the current valid target
version—this is the full-result path
- When `Serviceable()` returns `false` but `loadedRatio >=
requiredLoadRatio` (partial result case), segments are filtered against
the query view's segment lists rather than target version, ensuring
non-empty results as validated by
`TestPinReadableSegments_PartialResultNotEmpty`
- The test explicitly demonstrates that even with `loadedRatio == 1.0`
and `syncedByCoord == false`, calling `PinReadableSegments(0.8,
partition)` returns segments (partial result) instead of an empty list,
which was the bug root cause

**Root Cause Fix**: Previously, segments could be filtered with
`unreadableTargetVersion` when `loadedRatio == 1.0` but the querycoord
hadn't yet synced the target, causing empty segment lists. Now the sync
state is checked before deciding the filtering strategy, preventing this
race condition.

<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: Wei Liu <wei.liu@zilliz.com>
2025-12-31 15:03:22 +08:00
wei liu
5b6697c5cb
fix: Fix RBAC etcd prefix matching to prevent data leakage (#46707)
issue: #46676

This change fixes a bug where etcd prefix queries in RBAC could
incorrectly match entries with similar prefixes. For example, when
querying roles for user "admin", it could mistakenly return roles
belonging to "admin2".

The fix adds explicit "/" suffix to prefix keys before LoadWithPrefix
calls in three locations:
- getRolesByUsername: user role mapping queries
- ListGrant (appendGrantEntity): grantee ID queries
- ListGrant (role query): role grant queries

Also updates related unit tests to match the new prefix format and adds
TestRBACPrefixMatch to verify the fix.

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->

## Bug Fix: RBAC Etcd Prefix Matching Data Leakage

**Core Invariant:**
Etcd prefix queries must use explicit "/" delimiters between key
segments to enforce strict hierarchical boundaries; without them,
string-prefix matching returns all keys with similar starting characters
(e.g., prefix "admin" matches both "admin" and "admin2").

**Root Cause & Fix:**
The bug occurred in three RBAC query functions where prefix-based
lookups lacked trailing "/" separators. For example,
`getRolesByUsername(ctx, tenant, "admin")` would construct prefix
`"RoleMappingPrefix/tenant/admin"` and query `LoadWithPrefix(ctx,
prefix)`, unintentionally matching roles assigned to both "admin" and
"admin2" users. The fix appends "/" to the prefix before querying (e.g.,
`prefix + "/"`), making queries strictly match the intended
user/role/grantee entry only.

**Why No Data Loss or Regression:**
The fix modifies only how keys are *queried*, not how they are *stored*.
Etcd keys remain unchanged (still formatted as
`"RoleMappingPrefix/tenant/username/rolename"`). The corresponding
parsing logic using `typeutil.AfterN(key, k, "/")` correctly extracts
role names since the prefix `k` now ends with "/" (eliminating the need
to append "/" in the delimiter argument). All three affected code
paths—`getRolesByUsername`, `ListGrant` grantee ID queries, and
`ListGrant` role grant queries—consistently apply the same pattern,
ensuring backward-compatible behavior while fixing the unintended
cross-user/role leakage.

**Verification:**
New test suite `TestRBACPrefixMatch` confirms that querying user "user1"
no longer returns user "user10"'s roles, and similarly for role/grantee
ID prefixes, validating the fix resolves the reported data isolation
issue.

<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: Wei Liu <wei.liu@zilliz.com>
2025-12-31 13:47:20 +08:00
zhagnlu
635bead131
fix:fix incorrect rootPath for local storage mode (#46692)
#45959

Signed-off-by: luzhang <luzhang@zilliz.com>
Co-authored-by: luzhang <luzhang@zilliz.com>
2025-12-31 12:23:22 +08:00
Buqian Zheng
724598d231
fix: handle mixed int64/float types in BinaryRangeExpr for JSON fields (#46681)
test: add unit tests for mixed int64/float types in BinaryRangeExpr

When processing binary range expressions (e.g., `x > 499 && x <= 512.0`)
on JSON/dynamic fields with expression templates, the lower and upper
bounds could have different numeric types (int64 vs float64). This
caused an assertion failure in GetValueFromProto when the template type
didn't match the actual proto value type.

Fixes:
1. Go side (fill_expression_value.go): Normalize numeric types for JSON
fields - if either bound is float and the other is int, convert the int
to float.

2. C++ side (BinaryRangeExpr.cpp):
   - Check both lower_val and upper_val types when dispatching
   - Use double template when either bound is float
- Use GetValueWithCastNumber instead of GetValueFromProto to safely
handle int64->double conversion

issue: #46588

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: JSON field binary-range expressions must present
numeric bounds to the evaluator with a consistent numeric type; if
either bound is floating-point, both bounds must be treated as double to
avoid proto-type mismatches during template instantiation.
- Bug fix (issue #46588 & concrete change): mixed int64/float bounds
could dispatch the wrong template (e.g.,
ExecRangeVisitorImplForJson<int64_t>) and trigger assertions in
GetValueFromProto. Fixes: (1) Go parser (FillBinaryRangeExpressionValue
in fill_expression_value.go) normalizes mixed JSON numeric bounds by
promoting the int bound to float; (2) C++ evaluator
(PhyBinaryRangeFilterExpr::Eval in BinaryRangeExpr.cpp) inspects both
lower_type and upper_type, sets use_double when either is float, selects
ExecRangeVisitorImplForJson<double> for mixed numeric cases, and
replaces GetValueFromProto with GetValueWithCastNumber so int64→double
conversions are handled safely.
- Removed / simplified logic: the previous evaluator branched on only
the lower bound's proto type and had separate index/non-index handling
for int64 vs float; that per-bound branching is replaced by unified
numeric handling (convert to double when needed) and a single numeric
path for index use — eliminating redundant, error-prone branches that
assumed homogeneous bound types.
- No data loss or regression: changes only promote int→double for
JSON-range comparisons when the other bound is float; integer-only and
float-only paths remain unchanged. Promotion uses IEEE double (C++
double and Go float64) and only affects template dispatch and
value-extraction paths; GetValueWithCastNumber safely converts int64 to
double and index/non-index code paths both normalize consistently,
preserving semantics for comparisons and avoiding assertion failures.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

---------

Signed-off-by: Buqian Zheng <zhengbuqian@gmail.com>
2025-12-31 11:52:24 +08:00
junjiejiangjjj
1100d8f7e2
feat: Add semantic highlight (#46189)
https://github.com/milvus-io/milvus/issues/42589

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->

## Semantic Highlighting Feature

**Core Invariant**: Semantic highlighting operates on a per-field basis
with independent text processing through an external Zilliz highlight
provider. The implementation maintains field ID to field name mapping
and correlates highlight results back to original field outputs.

**What is Added**: This PR introduces semantic highlighting capability
for search results alongside the existing lexical highlighting. The
feature consists of:
- New `SemanticHighlight` orchestrator that validates queries/input
fields against collection schema, instantiates a Zilliz-based provider,
and batches text processing across multiple queries
- New `SemanticHighlighter` proxy wrapper implementing the `Highlighter`
interface for search pipeline integration
- New `semanticHighlightOperator` that processes search results by
delegating per-field text processing to the provider and attaching
correlated `HighlightResult` data to search outputs
- New gRPC service definition (`HighlightService`) and
`ZillizClient.Highlight()` method for external provider communication

**No Data Loss or Regression**: The change is purely additive without
modifying existing logic:
- Lexical highlighting path remains unchanged (separate switch case in
`createHighlightTask`)
- New `HighlightResults` field is only populated when semantic
highlighting is explicitly requested via `HighlightType_Semantic` enum
value
- Gracefully handles missing fields by returning explicit errors rather
than silent failures
- Pipeline operator integration follows existing patterns and only
processes when semantic highlighter is instantiated

**Why This Design**: Semantic highlighting is routed through the same
pipeline operator pattern as lexical highlighting, ensuring consistent
integration into search workflows. The per-field model allows flexible
highlighting across different text columns and batch processing ensures
efficient handling of multiple queries with configurable provider
constraints.

<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: junjie.jiang <junjie.jiang@zilliz.com>
2025-12-31 11:41:22 +08:00
foxspy
49939f5f2b
fix: enable diskann option by default (#46584)
issue: #46481 


<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: DiskANN requires OS support for asynchronous I/O
(AIO); the Makefile now encodes this by defaulting disk_index=OFF on
Darwin (macOS) and disk_index=ON on other OSes where AIO is available.
- Simplified logic: the build-time default was inverted for non-Darwin
platforms so DiskANN is enabled by default; redundant conditional
handling that previously forced OFF everywhere has been removed in favor
of an OS-based default while preserving the single manual override
variable.
- No data loss or regression: this is a compile-time change only — it
toggles inclusion of DiskANN code paths at build time and does not
modify runtime persistence or existing index files. macOS builds still
skip AIO-dependent DiskANN code paths, and Linux/other builds merely
compile support by default; no migration or runtime data-path changes
are introduced.
- Backward compatibility / fix for issue #46481: addresses the reported
need to enable DiskANN by default (issue #46481) while keeping explicit
disk_index overrides intact for CI and developer workflows, so existing
build scripts and deployments that pass disk_index continue to behave
unchanged.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: xianliang.li <xianliang.li@zilliz.com>
2025-12-31 11:07:22 +08:00
yanliang567
15ce8aedd8
test: Add some tests for group by search support json and dynamic field (#46630)
related issue: #46616


<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: these tests assume the v2 group-by search
implementation (TestMilvusClientV2Base and pymilvus v2 APIs such as
AnnSearchRequest/WeightedRanker) is functionally correct; the PR extends
coverage to validate group-by semantics when using JSON fields and
dynamic fields (see
tests/python_client/milvus_client_v2/test_milvus_client_search_group_by.py
— TestGroupSearch.setup_class and parametrized group_by_field cases).
- Logic removed/simplified: legacy v1 test scaffolding and duplicated
parametrized fixtures/test permutations were consolidated into
v2-focused suites (TestGroupSearch now inherits TestMilvusClientV2Base;
old TestGroupSearch/TestcaseBase patterns and large blocks in
test_mix_scenes were removed) to avoid redundant fixture permutations
and duplicate assertions while reusing shared helpers in common_func
(e.g., gen_scalar_field, gen_row_data_by_schema) and common_type
constants.
- Why this does NOT introduce data loss or behavior regression: only
test code, test helpers, and test imports were changed — no
production/server code altered. Test helper changes are
backward-compatible (gen_scalar_field forces primary key nullable=False
and only affects test data generation paths in
tests/python_client/common/common_func.py; get_field_dtype_by_field_name
now accepts schema dicts/ORM schemas and is used only by tests to choose
vector generation) and collection creation/insertion in tests use the
same CollectionSchema/FieldSchema paths, so production
storage/serialization logic is untouched.
- New capability (test addition): adds v2 test coverage for group-by
search over JSON and dynamic fields plus related scenarios — pagination,
strict/non-strict group_size, min/max group constraints, multi-field
group-bys and binary vector cases — implemented in
tests/python_client/milvus_client_v2/test_milvus_client_search_group_by.py
to address issue #46616.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

---------

Signed-off-by: yanliang567 <yanliang.qiao@zilliz.com>
2025-12-31 11:03:21 +08:00
zhagnlu
031acf5711
enhance: convert jsonstats translator to bson_index translator (#45036)
issue: #42533

Signed-off-by: luzhang <luzhang@zilliz.com>
Co-authored-by: luzhang <luzhang@zilliz.com>
2025-12-31 10:39:21 +08:00
nico
e75ad275aa
test: update tets cases (#46699)
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->

## Pull Request Summary: Test Case Updates for API Behavior Changes

**Core Invariant**: These test case updates reflect backend API
improvements to error messaging and schema information returned by
collection operations. The changes maintain backward compatibility—no
public signatures change, and all modifications are test expectation
updates.

**Updated Error Messages for Better Diagnostics**:
- `test_add_field_feature.py`: Updated expected error when adding a
vector field without dimension specification from a generic "not support
to add vector field" to the more descriptive "vector field must have
dimension specified, field name = {field_name}: invalid parameter". This
change is non-breaking for clients that only check error codes; it
improves developer experience with clearer error context.

**Schema Information Extension**:
- `test_milvus_client_collection.py`: Added `enable_namespace: False` to
the expected `describe_collection()` output. This is a new boolean field
in the collection metadata that defaults to False, representing an
opt-in feature. Existing code querying describe_collection continues to
work; the new field is simply an additional property in the response
dictionary.

**Dynamic Error Message Construction**:
- `test_milvus_client_search_invalid.py`: Replaced hardcoded error
message with conditional logic that generates the appropriate error
based on input state (None vectors vs invalid vector data). This
prevents test brittle failure if multiple error conditions exist, and
correctly validates the API's behavior handles both "missing data" and
"malformed data" cases distinctly.

**No Regression Risk**: All changes update test expectations to match
improved backend behavior. The changes are additive (new field in
schema) or clarifying (better error messages), with no modifications to
existing response structures or behavior for valid inputs.

<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: nico <cheng.yuan@zilliz.com>
2025-12-31 10:17:22 +08:00
cai.zhang
b13aac5164
fix: Include fieldID in raw data cleanup to prevent delete other fields (#46688)
issue: #46687 

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: raw-data cleanup must be scoped to (segment_id,
field_id) so deleting temporary raw files for one field never removes
raw files for other fields in the same segment (prevents cross-field
deletion during index builds).
- Root cause and fix (bug): VectorDiskIndex::Build() and
BuildWithDataset() called RemoveDir on the segment-level path; this
removed rawdata/{segment_id}/. The fix changes both calls to remove
storage::GenFieldRawDataPathPrefix(local_chunk_manager, segment_id,
field_id) instead, limiting cleanup to rawdata/{segment_id}_{field_id}/
(field-scoped).
- Logic removed/simplified: the old helper GetSegmentRawDataPathPrefix
was removed and callers were switched to GenFieldRawDataPathPrefix;
cleanup logic is simplified from segment-level to field-level path
generation and removal, eliminating redundant broad deletions.
- Why this does NOT cause data loss or regress behavior: the change
narrows RemoveDir() to the exact field path used when caching raw data
and offsets earlier in Build (offsets_path and CacheRawDataToDisk
produce field-scoped local paths). Build still writes/reads offsets and
raw data from GenFieldRawDataPathPrefix(...) and then removes that same
prefix after successful index.Build(); therefore only temporary files
for the built field are deleted and other fields’ raw files under the
same segment are preserved. This fixes issue #46687 by preventing
accidental deletion of other fields’ raw data.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: Cai Zhang <cai.zhang@zilliz.com>
2025-12-30 21:13:21 +08:00