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>
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>
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>
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>
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>
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>
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>
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>
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>
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>
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>
<!-- 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>
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>
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>
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: TestWait must deterministically verify that
FixedSizeAllocator.Wait() is notified when virtual resources are
released, so an allocation blocked due to exhausted virtual capacity
eventually succeeds after explicit deallocations.
- Removed/simplified logic: replaced the previous flaky pattern that
spawned 100 concurrent goroutines performing Reallocate with an explicit
channel-synchronized release goroutine that performs 100 sequential
negative Reallocate calls only after the test blocks on allocation. This
eliminates timing-dependent concurrency and the nondeterministic i-based
assertion.
- Why no data loss or behavior regression: only the test changed —
allocator implementation (Allocate/Reallocate/Release/Wait/notify) and
public APIs are unchanged. The test now exercises the same code paths
(Allocate fails, Wait blocks on cond, Reallocate/Release call notify),
but in a deterministic order, so the allocator semantics and resource
accounting (used, allocs map) remain unaffected.
- Change type and intent: Enhancement/refactor of unit test stability —
it tightens test synchronization to remove race-dependent assertions and
ensure the Wait/notify mechanism is reliably exercised without modifying
allocator logic.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
Signed-off-by: aoiasd <zhicheng.yue@zilliz.com>
issue: https://github.com/milvus-io/milvus/issues/44123
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: legacy in-cluster CDC/replication plumbing
(ReplicateMsg types, ReplicateID-based guards and flags) is obsolete —
the system relies on standard msgstream positions, subPos/end-ts
semantics and timetick ordering as the single source of truth for
message ordering and skipping, so replication-specific
channels/types/guards can be removed safely.
- Removed/simplified logic (what and why): removed replication feature
flags and params (ReplicateMsgChannel, TTMsgEnabled,
CollectionReplicateEnable), ReplicateMsg type and its tests, ReplicateID
constants/helpers and MergeProperties hooks, ReplicateConfig and its
propagation (streamPipeline, StreamConfig, dispatcher, target),
replicate-aware dispatcher/pipeline branches, and replicate-mode
pre-checks/timestamp-allocation in proxy tasks — these implemented a
redundant alternate “replicate-mode” pathway that duplicated
position/end-ts and timetick logic.
- Why this does NOT cause data loss or regression (concrete code paths):
no persistence or core write paths were removed — proxy PreExecute flows
(internal/proxy/task_*.go) still perform the same schema/ID/size
validations and then follow the normal non-replicate execution path;
dispatcher and pipeline continue to use position/subPos and
pullback/end-ts in Seek/grouping (pkg/mq/msgdispatcher/dispatcher.go,
internal/util/pipeline/stream_pipeline.go), so skipping and ordering
behavior remains unchanged; timetick emission in rootcoord
(sendMinDdlTsAsTt) is now ungated (no silent suppression), preserving or
increasing timetick delivery rather than removing it.
- PR type and net effect: Enhancement/Refactor — removes deprecated
replication API surface (types, helpers, config, tests) and replication
branches, simplifies public APIs and constructor signatures, and reduces
surface area for future maintenance while keeping DML/DDL persistence,
ordering, and seek semantics intact.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
---------
Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
relate: https://github.com/milvus-io/milvus/issues/46498
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: text fields configured with multi_analyzer_params must
include a "by_field" string that names another field containing per-row
analyzer choices; schemaInfo.GetMultiAnalyzerNameFieldID caches and
returns the dependent field ID (or 0 if none) and relies on that mapping
to make per-row analyzer names available to the highlighter.
- What changed / simplified: the highlighter is now schema-aware —
addTaskWithSearchText accepts *schemaInfo and uses
GetMultiAnalyzerNameFieldID to resolve the analyzer-name field;
resolution and caching moved into schemaInfo.multiAnalyzerFieldMap
(meta_cache.go), eliminating ad-hoc/typeutil-only lookups and duplicated
logic; GetMultiAnalyzerParams now gates on EnableAnalyzer(),
centralizing analyzer enablement checks.
- Why this fixes the bug (root cause): fixes#46498 — previously the
highlighter failed when the analyzer-by-field was not in output_fields.
The change (1) populates task.AnalyzerNames (defaulting missing names to
"default") when multi-analyzer is configured and (2) appends the
analyzer-name field ID to LexicalHighlighter.extraFields so FieldIDs
includes it; the operator then requests the analyzer-name column at
search time, ensuring per-row analyzer selection is available for
highlighting.
- No data-loss or regression: when no multi-analyzer is configured
GetMultiAnalyzerNameFieldID returns 0 and behavior is unchanged; the
patch only adds the analyzer-name field to requested output IDs (no
mutation of stored data). Error handling on malformed params is
preserved (errors are returned instead of silently changing data), and
single-analyzer behavior remains untouched.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
Signed-off-by: aoiasd <zhicheng.yue@zilliz.com>
issue: #45881
Add persistent task management for external collections with automatic
detection of external_source and external_spec changes. When source
changes, the system aborts running tasks and creates new ones, ensuring
only one active task per collection. Tasks validate their source on
completion to prevent superseded tasks from committing results.
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: at most one active UpdateExternalCollection task
exists per collection — tasks are serialized by collectionID
(collection-level locking) and any change to external_source or
external_spec aborts superseded tasks and causes a new task creation
(externalCollectionManager + external_collection_task_meta
collection-based locks enforce this).
- What was simplified/removed: per-task fine-grained locking and
concurrent multi-task acceptance per collection were replaced by
collection-level synchronization (external_collection_task_meta.go) and
a single persistent task lifecycle in DataCoord/Index task code;
redundant double-concurrent update paths were removed by checking
existing task presence in AddTask/LoadOrStore and aborting/overwriting
via Drop/Cancel flows.
- Why this does NOT cause data loss or regress behavior: task state
transitions and commit are validated against the current external
source/spec before applying changes — UpdateStateWithMeta and SetJobInfo
verify task metadata and persist via catalog only under matching
collection-state; DataNode externalCollectionManager persists task
results to in-memory manager and exposes Query/Drop flows (services.go)
without modifying existing segment data unless a task successfully
finishes and SetJobInfo atomically updates segments via meta/catalog
calls, preventing superseded tasks from committing stale results.
- New capability added: end-to-end external collection update workflow —
DataCoord Index task + Cluster RPC helpers + DataNode external task
runner and ExternalCollectionManager enable creating, querying,
cancelling, and applying external collection updates
(fragment-to-segment balancing, kept/updated segment handling, allocator
integration); accompanying unit tests cover success, failure,
cancellation, allocator errors, and balancing logic.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
---------
Signed-off-by: sunby <sunbingyi1992@gmail.com>
issue: #46550
- Add CatchUpStreamingDataTsLag parameter to control tolerable lag
threshold for delegator to be considered caught up
- Add catchingUpStreamingData field in delegator to track whether
delegator has caught up with streaming data
- Add catching_up_streaming_data field in LeaderViewStatus proto
- Check catching up status in CheckDelegatorDataReady, return not
ready when delegator is still catching up streaming data
- Add unit tests for the new functionality
When tsafe lag exceeds the threshold, the distribution will not be
considered serviceable, preventing queries from timing out in waitTSafe.
This is useful when streaming message queue consumption is slow.
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: a delegator must not be considered serviceable while
its tsafe lags behind the latest committed timestamp beyond a
configurable tolerance; a delegator is "caught-up" only when
(latestTsafe - delegator.GetTSafe()) < CatchUpStreamingDataTsLag
(configured by queryNode.delegator.catchUpStreamingDataTsLag, default
1s).
- New capability and where it takes effect: adds streaming-catchup
tracking to QueryNode/QueryCoord — an atomic catchingUpStreamingData
flag on shardDelegator (internal/querynodev2/delegator/delegator.go), a
new param CatchUpStreamingDataTsLag
(pkg/util/paramtable/component_param.go), and a
LeaderViewStatus.catching_up_streaming_data field in the proto
(pkg/proto/query_coord.proto). The flag is exposed in
GetDataDistribution (internal/querynodev2/services.go) and used by
QueryCoord readiness checks
(internal/querycoordv2/utils/util.go::CheckDelegatorDataReady) to reject
leaders that are still catching up.
- What logic is simplified/added (not removed): instead of relying
solely on segment distribution/worker heartbeats, the PR adds an
explicit readiness gate that returns "not available" when the delegator
reports catching-up-streaming-data. This is strictly additive — no
existing checks are removed; the new precondition runs before segment
availability validation to prevent premature routing to slow-consuming
delegators.
- Why this does NOT cause data loss or regress behavior: the change only
controls serviceability visibility and routing — it never drops or
mutates data. Concretely: shardDelegator starts with
catchingUpStreamingData=true and flips to false in UpdateTSafe once the
sampled lag falls below the configured threshold
(internal/querynodev2/delegator/delegator.go::UpdateTSafe). QueryCoord
will short-circuit in CheckDelegatorDataReady when
leader.Status.GetCatchingUpStreamingData() is true
(internal/querycoordv2/utils/util.go), returning a channel-not-available
error before any segment checks; when the flag clears, existing
segment-distribution checks (same code paths) resume. Tests added cover
both catching-up and caught-up paths
(internal/querynodev2/delegator/delegator_test.go,
internal/querycoordv2/utils/util_test.go,
internal/querynodev2/services_test.go), demonstrating convergence
without changed data flows or deletion of data.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
---------
Signed-off-by: Wei Liu <wei.liu@zilliz.com>
issue: https://github.com/milvus-io/milvus/issues/46517
ref: https://github.com/milvus-io/milvus/issues/42148
This PR supports match operator family with struct array and brute force
search only.
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: match operators only target struct-array element-level
predicates and assume callers provide a correct row_start so element
indices form a contiguous range; IArrayOffsets implementations convert
row-level bitmaps/rows (starting at row_start) into element-level
bitmaps or a contiguous element-offset vector used by brute-force
evaluation.
- New capability added: end-to-end support for MATCH_* semantics
(match_any, match_all, match_least, match_most, match_exact) — parser
(grammar + proto), planner (ParseMatchExprs), expr model
(expr::MatchExpr), compilation (Expr→PhyMatchFilterExpr), execution
(PhyMatchFilterExpr::Eval uses element offsets/bitmaps), and unit tests
(MatchExprTest + parser tests). Implementation currently works for
struct-array inputs and uses brute-force element counting via
RowBitsetToElementOffsets/RowBitsetToElementBitset.
- Logic removed or simplified and why: removed the ad-hoc
DocBitsetToElementOffsets helper and consolidated offset/bitset
derivation into IArrayOffsets::RowBitsetToElementOffsets and a
row_start-aware RowBitsetToElementBitset, and removed EvalCtx overloads
that embedded ExprSet (now EvalCtx(exec_ctx, offset_input)). This
centralizes array-layout logic in ArrayOffsets and removes duplicated
offset conversion and EvalCtx variants that were redundant for
element-level evaluation.
- No data loss / no behavior regression: persistent formats are
unchanged (no proto storage or on-disk layout changed); callers were
updated to supply row_start and now route through the centralized
ArrayOffsets APIs which still use the authoritative
row_to_element_start_ mapping, preserving exact element index mappings.
Eval logic changes are limited to in-memory plumbing (how
offsets/bitmaps are produced and how EvalCtx is constructed); expression
evaluation still invokes exprs_->Eval where needed, so existing behavior
and stored data remain intact.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
---------
Signed-off-by: SpadeA <tangchenjie1210@gmail.com>
Signed-off-by: SpadeA-Tang <tangchenjie1210@gmail.com>
issue: #44452
## Summary
Reduce test combinations in `TestSparsePlaceholderGroupSize` to decrease
test execution time:
- `nqs`: from `[1, 10, 100, 1000, 10000]` to `[1, 100, 10000]`
- `averageNNZs`: from `[1, 2, 4, 8, 16, 32, 64, 128, 256, 512, 1024,
2048]` to `[1, 4, 16, 64, 256, 1024]`
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
## TestSparsePlaceholderGroupSize Test Reduction
**Core Invariant:** The sparse vector NNZ estimation algorithm
(`EstimateSparseVectorNNZFromPlaceholderGroup`) must maintain accuracy
within bounded error thresholds—individual cases < 10% error and no more
than 2% of cases exceeding 5% error—across representative parameter
ranges.
**Test Coverage Optimized, Not Removed:** Test combinations reduced from
60 to 18 by pruning redundant parameter points while retaining critical
coverage: nqs now tests [1, 100, 10000] (min, mid, max) and averageNNZs
tests [1, 4, 16, 64, 256, 1024] (exponential spacing). Variant
generation logic (powers of 2 scaling) remains unchanged, ensuring error
scenarios are still exercised.
**No Behavioral Regression:** The algorithm under test is untouched;
only test case frequency decreases. The same assertions validate error
bounds are satisfied—individual assertions (`assert.Less(errorRatio,
10.0)`) and statistical assertions (`assert.Less(largeErrorRatio, 2.0)`)
remain identical, confirming that estimation quality is still verified.
**Why Safe:** Exponential spacing of removed parameters (e.g., nqs: 10,
1000 removed; averageNNZs: 2, 8, 32, 128, 512, 2048 removed) addresses
diminishing returns—intermediate values provide no new error scenarios
beyond what surrounding powers-of-2 values expose, while keeping test
execution time proportional to coverage value gained.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
Signed-off-by: Buqian Zheng <zhengbuqian@gmail.com>
issue: https://github.com/milvus-io/milvus/issues/46410
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: etcd metadata and in-memory Segment/TextIndex records
must store only compact filenames for text-index files; full object keys
are deterministically reconstructed at use-sites from a stable root +
common.TextIndexPath + IDs via metautil.BuildTextLogPaths.
- Bug & fix (issue #46410): the etcd RPC size overflow was caused by
persisting full upload keys in segment/TextIndex metadata. Fix: at
upload/creation sites (internal/datanode/compactor/sort_compaction.go
and internal/datanode/index/task_stats.go) store only filenames using
metautil.ExtractTextLogFilenames; at consumption/use sites
(internal/datacoord/garbage_collector.go,
internal/querynodev2/segments/segment.go, and other GC/loader code)
reconstruct full paths with metautil.BuildTextLogPaths before accessing
object storage.
- Simplified/removed logic: removed the redundant practice of carrying
full object keys through metadata and in-memory structures; callers now
persist compact filenames and perform on-demand path reconstruction.
This eliminates large payloads in etcd and reduces memory pressure while
preserving the same runtime control flow and error handling.
- No data loss / no regression: filename extraction is a deterministic
suffix operation (metautil.ExtractTextLogFilenames) and reloadFromKV
performs backward compatibility (internal/datacoord/meta.go converts
existing full-path entries to filenames before caching). All read paths
reconstruct full paths at runtime (garbage_collector.getTextLogs,
LocalSegment.LoadTextIndex, GC/loader), so no files are modified/deleted
and access semantics remain identical.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
Signed-off-by: sijie-ni-0214 <sijie.ni@zilliz.com>
Support crate analyzer with file resource info, and return used file
resource ids when validate analyzer.
Save the related resource ids in collection schema.
relate: https://github.com/milvus-io/milvus/issues/43687
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: analyzer file-resource resolution is deterministic and
traceable by threading a FileResourcePathHelper (collecting used
resource IDs in a HashSet) through all tokenizer/analyzer construction
and validation paths; validate_analyzer(params, extra_info) returns the
collected Vec<i64) which is propagated through C/Rust/Go layers to
callers (CValidateResult → RustResult::from_vec_i64 → Go []int64 →
querypb.ValidateAnalyzerResponse.ResourceIds →
CollectionSchema.FileResourceIds).
- Logic removed/simplified: ad‑hoc, scattered resource-path lookups and
per-filter file helpers (e.g., read_synonyms_file and other inline
file-reading logic) were consolidated into ResourceInfo +
FileResourcePathHelper and a centralized get_resource_path(helper, ...)
API; filter/tokenizer builder APIs now accept &mut
FileResourcePathHelper so all file path resolution and ID collection use
the same path and bookkeeping logic (redundant duplicated lookups
removed).
- Why no data loss or behavior regression: changes are additive and
default-preserving — existing call sites pass extra_info = "" so
analyzer creation/validation behavior and error paths remain unchanged;
new Collection.FileResourceIds is populated from resp.ResourceIds in
validateSchema and round‑tripped through marshal/unmarshal
(model.Collection ↔ schemapb.CollectionSchema) so schema persistence
uses the new list without overwriting other schema fields; proto change
adds a repeated field (resource_ids) which is wire‑compatible (older
clients ignore extra field). Concrete code paths: analyzer creation
still uses create_analyzer (now with extra_info ""), tokenizer
validation still returns errors as before but now also returns IDs via
CValidateResult/RustResult, and rootcoord.validateSchema assigns
resp.ResourceIds → schema.FileResourceIds.
- New capability added: end‑to‑end discovery, return, and persistence of
file resource IDs used by analyzers — validate flows now return resource
IDs and the system stores them in collection schema (affects tantivy
analyzer binding, canalyzer C bindings, internal/util analyzer APIs,
querynode ValidateAnalyzer response, and rootcoord/create_collection
flow).
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
Signed-off-by: aoiasd <zhicheng.yue@zilliz.com>
### **Description**
- Add `-buildvcs=false` flag to Go test commands in coverage script
- Increase default session TTL from 10s to 15s
- Update SessionTTL parameter default value from 30 to 15
Signed-off-by: Cai Zhang <cai.zhang@zilliz.com>
Co-authored-by: bigsheeper <yihao.dai@zilliz.com>
Co-authored-by: chyezh <chyezh@outlook.com>
Co-authored-by: czs007 <zhenshan.cao@zilliz.com>
Related to #46133
Move jemalloc_stats.go and its test file from pkg/util/hardware to
internal/util/segcore. This is a more appropriate location because:
- jemalloc_stats depends on milvus_core C++ library via cgo
- The pkg directory should remain independent of internal C++
dependencies
- segcore is the natural home for core memory allocator utilities
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
## Summary by CodeRabbit
* **Refactor**
* Improved internal code organization by reorganizing memory statistics
collection infrastructure for better maintainability and modularity. No
impact on end-user functionality or behavior.
<sub>✏️ Tip: You can customize this high-level summary in your review
settings.</sub>
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
Introduce a ScannerStartupDelay configuration to enable WAL write-only
recovery, allowing fence messages to be persisted during
primary–secondary switchover when the StreamingNode is trapped in crash
loops.
issue: https://github.com/milvus-io/milvus/issues/46368
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
## Summary by CodeRabbit
* **New Features**
* Added a configurable WAL scanner pause/resume and a consumer request
flag to optionally ignore pause signals.
* **Metrics**
* Added a scanner pause gauge and pause-duration tracking for WAL
scanning.
* **Tests**
* Added coverage for pause-consumption behavior and cleanup in stream
client tests.
* **Chores**
* Consolidated flush-all logging into a single field and added a helper
for bulk message conversion.
<sub>✏️ Tip: You can customize this high-level summary in your review
settings.</sub>
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
---------
Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
related: #45993
This commit extends nullable vector support to the proxy layer,
querynode,
and adds comprehensive validation, search reduce, and field data
handling
for nullable vectors with sparse storage.
Proxy layer changes:
- Update validate_util.go checkAligned() with getExpectedVectorRows()
helper
to validate nullable vector field alignment using valid data count
- Update checkFloatVectorFieldData/checkSparseFloatVectorFieldData for
nullable vector validation with proper row count expectations
- Add FieldDataIdxComputer in typeutil/schema.go for logical-to-physical
index translation during search reduce operations
- Update search_reduce_util.go reduceSearchResultData to use
idxComputers
for correct field data indexing with nullable vectors
- Update task.go, task_query.go, task_upsert.go for nullable vector
handling
- Update msg_pack.go with nullable vector field data processing
QueryNode layer changes:
- Update segments/result.go for nullable vector result handling
- Update segments/search_reduce.go with nullable vector offset
translation
Storage and index changes:
- Update data_codec.go and utils.go for nullable vector serialization
- Update indexcgowrapper/dataset.go and index.go for nullable vector
indexing
Utility changes:
- Add FieldDataIdxComputer struct with Compute() method for efficient
logical-to-physical index mapping across multiple field data
- Update EstimateEntitySize() and AppendFieldData() with fieldIdxs
parameter
- Update funcutil.go with nullable vector support functions
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
## Summary by CodeRabbit
* **New Features**
* Full support for nullable vector fields (float, binary, float16,
bfloat16, int8, sparse) across ingest, storage, indexing, search and
retrieval; logical↔physical offset mapping preserves row semantics.
* Client: compaction control and compaction-state APIs.
* **Bug Fixes**
* Improved validation for adding vector fields (nullable + dimension
checks) and corrected search/query behavior for nullable vectors.
* **Chores**
* Persisted validity maps with indexes and on-disk formats.
* **Tests**
* Extensive new and updated end-to-end nullable-vector tests.
<sub>✏️ Tip: You can customize this high-level summary in your review
settings.</sub>
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
---------
Signed-off-by: marcelo-cjl <marcelo.chen@zilliz.com>
relate: https://github.com/milvus-io/milvus/issues/42589
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
## Summary by CodeRabbit
## New Features
- Added `concurrency_per_cpu_core` configuration parameter for the
analyzer component, enabling customizable per-CPU concurrency tuning
(default: 8).
## Tests
- Added test coverage for batch analysis operations.
<sub>✏️ Tip: You can customize this high-level summary in your review
settings.</sub>
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
Signed-off-by: aoiasd <zhicheng.yue@zilliz.com>
This commit refines L0 compaction to ensure data consistency by properly
setting the delete position boundary for L1 segment selection.
Key Changes:
1. L0 View Trigger Sets latestDeletePos for L1 Selection
2. Filter L0 Segments by Growing Segment Position in policy, not in
views
3. Renamed LevelZeroSegmentsView to LevelZeroCompactionView
4. Renamed fields for semantic clarity: * segments -> l0Segments *
earliestGrowingSegmentPos -> latestDeletePos
5. Update Default Compaction Prioritizer to level
See also: #46434
---------
Signed-off-by: yangxuan <xuan.yang@zilliz.com>
This PR improves the robustness of object storage operations by retrying
both explicit throttling errors (e.g. HTTP 429, SlowDown, ServerBusy).
These errors commonly occur under high concurrency and are typically
recoverable with bounded retries.
issue: https://github.com/milvus-io/milvus/issues/44772
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
## Summary by CodeRabbit
* **New Features**
* Configurable retry support for reads from object storage and improved
mapping of transient/rate-limit errors.
* Added a retryable reader wrapper used by CSV/JSON/Parquet/Numpy import
paths.
* **Configuration**
* New parameter to control storage read retry attempts.
* **Tests**
* Expanded unit tests covering error mapping and retry behaviors across
storage backends.
* Standardized mock readers and test initialization to simplify test
setups.
<sub>✏️ Tip: You can customize this high-level summary in your review
settings.</sub>
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
---------
Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
### **User description**
issue: #46507
we use the assign/unassign api to manage the consumer manually, the
commit operation will generate a new consumer group which is not what we
want. so we disable the auto commit to avoid it, also see:
https://github.com/confluentinc/confluent-kafka-python/issues/250#issuecomment-331377925
___
### **PR Type**
Bug fix
___
### **Description**
- Disable auto-commit in Kafka consumer configuration
- Prevents unwanted consumer group creation from manual offset
management
- Clarifies offset reset behavior with explanatory comments
___
### Diagram Walkthrough
```mermaid
flowchart LR
A["Kafka Consumer Config"] --> B["Set enable.auto.commit to false"]
B --> C["Prevent auto consumer group creation"]
A --> D["Set auto.offset.reset to earliest"]
D --> E["Handle deleted offsets gracefully"]
```
<details><summary><h3>File Walkthrough</h3></summary>
<table><thead><tr><th></th><th align="left">Relevant
files</th></tr></thead><tbody><tr><td><strong>Bug
fix</strong></td><td><table>
<tr>
<td>
<details>
<summary><strong>builder.go</strong><dd><code>Disable auto-commit and
add configuration comments</code>
</dd></summary>
<hr>
pkg/streaming/walimpls/impls/kafka/builder.go
<ul><li>Added <code>enable.auto.commit</code> configuration set to
<code>false</code> to prevent <br>automatic consumer group creation<br>
<li> Added explanatory comments for both <code>auto.offset.reset</code>
and <br><code>enable.auto.commit</code> settings<br> <li> Clarifies that
manual assign/unassign API is used for consumer <br>management</ul>
</details>
</td>
<td><a
href="https://github.com/milvus-io/milvus/pull/46508/files#diff-4b5635821fdc8b585d16c02d8a3b59079d8e667b2be43a073265112d72701add">+7/-0</a>
</td>
</tr>
</table></td></tr></tbody></table>
</details>
___
<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
## Summary by CodeRabbit
## Bug Fixes
* Kafka consumer now reads from the earliest available messages and
auto-commit has been disabled to support manual offset management.
<sub>✏️ Tip: You can customize this high-level summary in your review
settings.</sub>
<!-- end of auto-generated comment: release notes by coderabbit.ai -->
Signed-off-by: chyezh <chyezh@outlook.com>
issue: https://github.com/milvus-io/milvus/issues/45890
ComputePhraseMatchSlop accepts three pararms:
1. A string: query text
2. Some trings: data texts
3. Analyzer params,
Slop will be calculated for the query text with each data text in the
context of phrase match where they are tokenized with tokenizer with
analyzer params.
So two array will be returned:
1. is_match: is phrase match can sucess
2. slop: the related slop if phrase match can sucess, or -1 is cannot.
---------
Signed-off-by: SpadeA <tangchenjie1210@gmail.com>
issue: #46443
Add `Forbidden: true` to all tiered storage related parameters to
prevent runtime configuration changes via etcd. These parameters are
marked as refreshable:"false" but that tag was only documentation - the
actual prevention requires the Forbidden field.
Without this fix, if tiered storage parameters are modified at runtime:
- Go side would read the new values dynamically
- C++ caching layer would still use the old values (set at InitQueryNode
time)
- This mismatch could cause resource tracking issues and anomalies
Signed-off-by: Shawn Wang <shawn.wang@zilliz.com>
Related to #44614
Previous PR: #44666
Bump etcd version in pkg/go.mod to 3.5.23 and update test code
accordingly
Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
Related to #46358
Add segment reopen mechanism in QueryCoord to handle segment data
updates when the manifest path changes. This enables QueryNode to reload
segment data without full segment reload, supporting storage v2
incremental updates.
Changes:
- Add ActionTypeReopen action type and LoadScope_Reopen in protobuf
- Track ManifestPath in segment distribution metadata
- Add CheckSegmentDataReady utility to verify segment data matches
target
- Extend getSealedSegmentDiff to detect segments needing reopen
- Create segment reopen tasks when manifest path differs from target
- Block target update until segment data is ready
---------
Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
issue: #43897
also for issue: #46166
add ack_sync_up flag into broadcast message header, which indicates that
whether the broadcast operation is need to be synced up between the
streaming node and the coordinator.
If the ack_sync_up is false, the broadcast operation will be acked once
the recovery storage see the message at current vchannel, the fast ack
operation can be applied to speed up the broadcast operation.
If the ack_sync_up is true, the broadcast operation will be acked after
the checkpoint of current vchannel reach current message.
The fast ack operation can not be applied to speed up the broadcast
operation, because the ack operation need to be synced up with streaming
node.
e.g. if truncate collection operation want to call ack once callback
after the all segment are flushed at current vchannel, it should set the
ack_sync_up to be true.
TODO: current implementation doesn't promise the ack sync up semantic,
it only promise FastAck operation will not be applied, wait for 3.0 to
implement the ack sync up semantic. only for truncate api now.
---------
Signed-off-by: chyezh <chyezh@outlook.com>
Related to #44956
This change propagates the useLoonFFI configuration through the import
pipeline to enable LOON FFI usage during data import operations.
Key changes:
- Add use_loon_ffi field to ImportRequest protobuf message
- Add manifest_path field to ImportSegmentInfo for tracking manifest
- Initialize manifest path when creating segments (both import and
growing)
- Pass useLoonFFI flag through NewSyncTask in import tasks
- Simplify pack_writer_v2 by removing GetManifestInfo method and relying
on pre-initialized manifest path from segment creation
- Update segment meta with manifest path after import completion
This allows the import workflow to use the LOON FFI based packed writer
when the common.useLoonFFI configuration is enabled.
---------
Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
issue: #46358
This PR implements segment reopening functionality on query nodes,
enabling the application of data or schema changes to already-loaded
segments without requiring a full reload.
### Core (C++)
**New SegmentLoadInfo class**
(`internal/core/src/segcore/SegmentLoadInfo.h/cpp`):
- Encapsulates segment load configuration with structured access
- Implements `ComputeDiff()` to calculate differences between old and
new load states
- Tracks indexes, binlogs, and column groups that need to be loaded or
dropped
- Provides `ConvertFieldIndexInfoToLoadIndexInfo()` for index loading
**ChunkedSegmentSealedImpl modifications**:
- Added `Reopen(const SegmentLoadInfo&)` method to apply incremental
changes based on computed diff
- Refactored `LoadColumnGroups()` and `LoadColumnGroup()` to support
selective loading via field ID map
- Extracted `LoadBatchIndexes()` and `LoadBatchFieldData()` for reusable
batch loading logic
- Added `LoadManifest()` for manifest-based loading path
- Updated all methods to use `SegmentLoadInfo` wrapper instead of direct
proto access
**SegmentGrowingImpl modifications**:
- Added `Reopen()` stub method for interface compliance
**C API additions** (`segment_c.h/cpp`):
- Added `ReopenSegment()` function exposing reopen to Go layer
### Go Side
**QueryNode handlers** (`internal/querynodev2/`):
- Added `HandleReopen()` in handlers.go
- Added `ReopenSegments()` RPC in services.go
**Segment interface** (`internal/querynodev2/segments/`):
- Extended `Segment` interface with `Reopen()` method
- Implemented `Reopen()` in LocalSegment
- Added `Reopen()` to segment loader
**Segcore wrapper** (`internal/util/segcore/`):
- Added `Reopen()` method in segment.go
- Added `ReopenSegmentRequest` in requests.go
### Proto
- Added new fields to support reopen in `query_coord.proto`
---------
Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
Related #44956
Add manifest_path field to CreateStatsRequest and propagate it through
the stats task pipeline. This enables stats tasks and text index
building to access segment manifest for storage v2 format operations.
- Add manifest_path field to CreateStatsRequest proto
- Set ManifestPath from segment metadata in DataCoord
- Pass manifest to BuildIndexInfo in stats task builder
- Include manifest in compaction text index creation
Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
issue: https://github.com/milvus-io/milvus/issues/42148
For a vector field inside a STRUCT, since a STRUCT can only appear as
the element type of an ARRAY field, the vector field in STRUCT is
effectively an array of vectors, i.e. an embedding list.
Milvus already supports searching embedding lists with metrics whose
names start with the prefix MAX_SIM_.
This PR allows Milvus to search embeddings inside an embedding list
using the same metrics as normal embedding fields. Each embedding in the
list is treated as an independent vector and participates in ANN search.
Further, since STRUCT may contain scalar fields that are highly related
to the embedding field, this PR introduces an element-level filter
expression to refine search results.
The grammar of the element-level filter is:
element_filter(structFieldName, $[subFieldName] == 3)
where $[subFieldName] refers to the value of subFieldName in each
element of the STRUCT array structFieldName.
It can be combined with existing filter expressions, for example:
"varcharField == 'aaa' && element_filter(struct_field, $[struct_int] ==
3)"
A full example:
```
struct_schema = milvus_client.create_struct_field_schema()
struct_schema.add_field("struct_str", DataType.VARCHAR, max_length=65535)
struct_schema.add_field("struct_int", DataType.INT32)
struct_schema.add_field("struct_float_vec", DataType.FLOAT_VECTOR, dim=EMBEDDING_DIM)
schema.add_field(
"struct_field",
datatype=DataType.ARRAY,
element_type=DataType.STRUCT,
struct_schema=struct_schema,
max_capacity=1000,
)
...
filter = "varcharField == 'aaa' && element_filter(struct_field, $[struct_int] == 3 && $[struct_str] == 'abc')"
res = milvus_client.search(
COLLECTION_NAME,
data=query_embeddings,
limit=10,
anns_field="struct_field[struct_float_vec]",
filter=filter,
output_fields=["struct_field[struct_int]", "varcharField"],
)
```
TODO:
1. When an `element_filter` expression is used, a regular filter
expression must also be present. Remove this restriction.
2. Implement `element_filter` expressions in the `query`.
---------
Signed-off-by: SpadeA <tangchenjie1210@gmail.com>