mirror of
https://gitee.com/milvus-io/milvus.git
synced 2026-01-07 19:31:51 +08:00
11620 Commits
| Author | SHA1 | Message | Date | |
|---|---|---|---|---|
|
|
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> |
||
|
|
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> |
||
|
|
56e82c78e1
|
fix: add load config watcher to avoid load config modification lost (#46784)
issue: #46778 Signed-off-by: chyezh <chyezh@outlook.com> |
||
|
|
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> |
||
|
|
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> |
||
|
|
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> |
||
|
|
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> |
||
|
|
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> |
||
|
|
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> |
||
|
|
52a05831b5
|
enhance: Tidy build index logs (#46531)
Signed-off-by: yangxuan <xuan.yang@zilliz.com> |
||
|
|
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> |
||
|
|
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> |
||
|
|
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> |
||
|
|
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> |
||
|
|
146f173eea
|
enhance: tidy code in import plugin context (#46374)
Signed-off-by: yangxuan <xuan.yang@zilliz.com> |
||
|
|
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> |
||
|
|
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> |
||
|
|
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> |
||
|
|
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> |
||
|
|
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> |
||
|
|
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> |
||
|
|
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> |
||
|
|
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> |
||
|
|
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> |
||
|
|
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> |
||
|
|
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> |
||
|
|
1a6f3c4305
|
enhance: batch processing for ngram (#46648)
issue: https://github.com/milvus-io/milvus/issues/42053 Process ngram in batch rather than all by once. <!-- This is an auto-generated comment: release notes by coderabbit.ai --> ## Batch Processing for N-gram Queries **Core Invariant:** All data iteration is now driven by `batch_size_` as the fundamental unit; for sealed chunked segments processing string/JSON data, processing is strictly stateless to allow specialized batched algorithms. **Simplified Logic:** - Removed the `process_all_chunks` boolean flag from `ProcessMultipleChunksCommon` (renamed to `ProcessDataChunksForMultipleChunk`) as it was redundant—all iteration paths now converge on the same `batch_size_`-driven chunking strategy with unified data size clamping (`std::min(chunk_size, batch_size_ - processed_size)`). - Eliminated wrapper delegation methods (`ProcessDataChunksForMultipleChunk` and `ProcessAllChunksForMultipleChunk` old wrappers) that pointed to a single common implementation with a conditional flag. **No Data Loss or Behavior Regression:** - The new `ProcessAllDataChunkBatched<T>` is an additional stateless public path (requires sealed + chunked segments, type constraints: `std::string_view|Json|ArrayView`) that iterates all `num_data_chunk_` chunks in `batch_size_` granularity without mutating cursor state (`current_data_chunk_`, `current_data_chunk_pos_`), ensuring deterministic re-entrant processing. - Existing cursor-based APIs (`ProcessDataChunksForMultipleChunk`, `ProcessChunkForSealedSeg`) remain unchanged for standard expression evaluation—no segment state is corrupted. - N-gram query execution now routes through `ExecuteQueryWithPredicate<T, Predicate>(literal, segment, predicate, need_post_filter)` which forwards generic predicates and delegates to `segment->ProcessAllDataChunkBatched<T>(execute_batch, res)` for post-filtering, avoiding per-chunk single-pass traversal. **Enhancement:** Generic predicate template `template <typename T, typename Predicate>` with perfect forwarding (`Predicate&& predicate`) replaces the fixed `std::function<bool(const T&)>` signature, eliminating function wrapper overhead for n-gram matcher closures and enabling efficient batch processing callbacks. <!-- end of auto-generated comment: release notes by coderabbit.ai --> --------- Signed-off-by: SpadeA <tangchenjie1210@gmail.com> |
||
|
|
83ab90af93
|
enhance: modify bson thirdparty lib compile mode (#45406)
#42533 Signed-off-by: luzhang <luzhang@zilliz.com> Co-authored-by: luzhang <luzhang@zilliz.com> |
||
|
|
b18ebd9468
|
enhance: Remove legacy cdc/replication (#46603)
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> |
||
|
|
b7761d67a3
|
enhance: Enhance logs for proxy and rootcoord meta table (#46652)
issue: https://github.com/milvus-io/milvus/issues/46651 <!-- This is an auto-generated comment: release notes by coderabbit.ai --> ## Enhancement: Add Context-Aware Logging for Proxy and RootCoord Meta Table Operations **Core Invariant**: All changes maintain existing cache behavior and state transition logic by purely enhancing observability through context-aware logging without modifying control flow, return values, or data structures. **Logic Simplified Without Regression**: - Removed internal helper method `getFullCollectionInfo` from MetaCache by inlining its logic directly into GetCollectionInfo, eliminating an unnecessary abstraction layer while preserving the exact same cache-hit/miss and fetch-or-update paths - This consolidation has no impact on behavior because the helper was only called from one location and the inlined logic executes identically **Enhanced Logging for Observability (No Behavior Changes)**: - Added context-aware logging (log.Ctx(ctx)) to cache miss scenarios and timestamp comparisons in proxy MetaCache, enabling request tracing without altering cache lookup logic - Expanded RootCoord MetaTable's internal helper method signatures to propagate context for contextual logging across collection lifecycle events (begin truncate, update state, remove names/aliases, delete from collections map), while keeping all call sites and state transitions unchanged - Enhanced DescribeCollection logging in proxy to capture request scope (role, database, collection IDs, timestamp) and response schema at operation boundaries **Type**: Enhancement focused on improved observability. All modifications are strictly additive logging; no data structures, caching strategies, or core logic paths were altered. <!-- end of auto-generated comment: release notes by coderabbit.ai --> Signed-off-by: bigsheeper <yihao.dai@zilliz.com> |
||
|
|
b4682b7352
|
fix: use LoadDeltaData instead of Delete for L0 growing forward (#46657)
Related to #46660 Replace segment.Delete() with segment.LoadDeltaData() when forwarding L0 deletions to growing segments. LoadDeltaData is the more appropriate API for bulk loading delta data compared to individual Delete calls. <!-- This is an auto-generated comment: release notes by coderabbit.ai --> • Core invariant: forwarding L0 deletions to growing segments must use the bulk-delta API (storage.DeltaData + segment.LoadDeltaData) because LoadDeltaData preserves paired primary keys and timestamps as a single atomic delta payload; segment.Delete was intended for per-delete RPCs and not for loading L0 delta payloads. • Logic removed/simplified: addL0GrowingBF() no longer calls segment.Delete for buffered L0 keys. Instead the buffered callback builds a storage.DeltaData via storage.NewDeltaDataWithData(pks, tss) and calls segment.LoadDeltaData(ctx, dd). This eliminates the previous per-batch Delete call path and centralizes forwarding as a single delta-load operation. • Why this does not cause data loss or regression: the new path supplies identical PK+timestamp pairs to the segment via DeltaData; LoadDeltaData applies the same delete semantics but accepts batched delta payloads. The change is limited to the L0→growing Bloom-Filter forward path (addL0GrowingBF/addL0ForGrowingLoad), leaving sealed-segment deletes, streaming direct forwarding, and remote-load policies unchanged. Also, the prior code would fail on L0Segment.Delete (L0 segments prohibit Delete), so switching to LoadDeltaData prevents lost-forwarding caused by unsupported Delete calls. • Category: Enhancement / Refactor — replaces inappropriate per-delete calls with the correct bulk delta-load API, simplifying error handling around NewDeltaDataWithData and ensuring API contract correctness for L0→growing forwarding. <!-- end of auto-generated comment: release notes by coderabbit.ai --> --------- Signed-off-by: Congqi Xia <congqi.xia@zilliz.com> |
||
|
|
90809d1d86
|
fix: highlight with multi analyzer failed (#46527)
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> |
||
|
|
da732ec04d
|
enhance: change credential provider to singleton(#46649) (#46653)
related: #46649 <!-- This is an auto-generated comment: release notes by coderabbit.ai --> - Core invariant: STS IAM credential providers for Aliyun, Tencent Cloud, and Huawei Cloud are global, stateless resources that must be instantiated once and reused across all ChunkManager instances (singleton), rather than created per-manager. - Logic removed/simplified: Removed per-instance Aws::MakeShared instantiation of STSAssumeRoleWebIdentityCredentialsProvider inside Aliyun/Tencent/Huawei ChunkManager constructors and replaced them with public static Get...CredentialsProvider() methods that return a thread-safe, lazily-initialized shared_ptr singleton (static local variable). This eliminates duplicate provider construction and header/signal dependency usages tied to per-constructor instantiation. - Why this does NOT introduce data loss or behavior regression: Credential acquisition and usage paths are unchanged — callers still call provider->GetAWSCredentials() and use the returned AWSCredentials to construct Aws::S3::S3Client. The singleton returns the same provider object but the provider is stateless with respect to per-manager data (it only reads environment/platform credentials and produces AWSCredentials). C++11+ static local initialization provides atomic, thread-safe construction, so first-access semantics and validation checks (AssertInfo on access key/secret/token) remain intact. - PR type (Enhancement/Refactor): Improves credential management by centralizing provider lifecycle, removing redundant allocations and header dependencies, and enforcing a single shared provider per cloud vendor where IAM is used. <!-- 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> |
||
|
|
dc7c92d398
|
fix: scalar bench builds on its own, removing related target from milvus (#46658)
issue: https://github.com/milvus-io/milvus/issues/44452 <!-- This is an auto-generated comment: release notes by coderabbit.ai --> **Scalar Bench Decoupled from Milvus Build System** - **Core assumption**: Scalar-bench is now managed as an independent build artifact outside the milvus repository, eliminating the need for conditional compilation integration within milvus's Makefile and CMakeLists.txt. - **Build infrastructure simplified**: Removed `scalar-bench` and `scalar-bench-ui` targets from Makefile and deleted the entire `ENABLE_SCALAR_BENCH` conditional block in `internal/core/unittest/CMakeLists.txt` (which handled FetchContent, cache variables, and subdirectory integration)—this eliminates optional, redundant build-time coupling that is no longer necessary. - **No regression introduced**: The removal only affects optional build-time integration paths. Core C++ builds continue functioning as before, and unit tests remain unaffected since `ENABLE_SCALAR_BENCH` was always optional (not a required dependency); the newly added `plan-parser-so` dependency on core build targets appears to be a separate, required component. - **Decoupling benefit**: Scalar-benchmark can now evolve and release on its own schedule independent of milvus release cycles, while maintaining clean separation of concerns between the two projects. <!-- end of auto-generated comment: release notes by coderabbit.ai --> Signed-off-by: Buqian Zheng <zhengbuqian@gmail.com> |
||
|
|
f9827392bb
|
enhance: implement external collection update task with source change detection (#45905)
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> |
||
|
|
293838bb67
|
enhance: add delegator catching up streaming data state tracking (#46551)
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> |
||
|
|
4230a5beaa
|
enhance: revert mmap configs (#46581)
Signed-off-by: yangxuan <xuan.yang@zilliz.com> |
||
|
|
f087b7432e
|
fix: increase expiry time for huawei cloud(#46296) (#46298)
related: #46296 <!-- This is an auto-generated comment: release notes by coderabbit.ai --> - Core invariant: expiration comparisons use Aws::Utils::DateTime::Now().count() which returns milliseconds; any expiration grace period must be expressed in milliseconds and compared via (GetExpiration() - Now()).count() in ExpiresSoon() (Huawei and Tencent providers). - Root cause and fix: the grace period constant was authored as 7200 (seconds) but used against millisecond counts, causing premature refreshes. The PR changes STS_CREDENTIAL_PROVIDER_EXPIRATION_GRACE_PERIOD to 180 * 1000 (180000 ms) in HuaweiCloudCredentialsProvider.cpp and TencentCloudCredentialsProvider.cpp to align units and stop unnecessary refreshes. - Removed/replaced redundant/incorrect behavior: the PR does not add new control flow but corrects unit mismatch and simplifies logging/STS request handling — HuaweiCloudSTSClient now explicitly requests a 7200-second token by adding "token": {"duration_seconds": 7200} to the JSON body and uses JsonValue(...).View() for parsing; Huawei logging level raised from TRACE to DEBUG and now logs expiration_count_diff_ms for clarity. These changes remove ambiguity about requested token lifetime and improve diagnostic output. - No data loss or regression: credential contents and assignment are unchanged — Reload()/RefreshIfExpired()/ExpiresSoon() still populate m_credentials from STS responses and return them via GetAWSCredentials(); only the grace-period unit and the Huawei STS request body/parsing/logging were adjusted. Code paths affected are ExpiresSoon()/RefreshIfExpired()/Reload() in both providers and HuaweiCloudSTSCredentialsClient::callHuaweiCloudSTS; since credentials are still read from the same response fields (access, secret, securitytoken, expires_at) and assigned to result.creds, there is no data loss or altered persistence/authorization semantics beyond aligning requested token duration and correct refresh timing. <!-- 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> |
||
|
|
1399d955fc
|
enhance: optimize timestamptz comparison without interval (#46619)
issue: https://github.com/milvus-io/milvus/issues/46618 <!-- This is an auto-generated comment: release notes by coderabbit.ai --> • **Core Invariant**: TIMESTAMPTZ values are internally stored as int64 Unix microseconds. Simple comparisons without intervals can safely use native int64 range evaluation (`ExecRangeVisitorImpl<int64_t>`) and `UnaryRangeExpr` to leverage index-based scans, since the underlying data type and comparison semantics remain unchanged. • **Logic Optimization**: The parser now branches on interval presence. When `ctx.GetOp1() == nil` (no interval), it returns a lightweight `UnaryRangeExpr` for fast indexed range scans. When an interval exists, it falls back to the heavier `TimestamptzArithCompareExpr` for arithmetic evaluation. This eliminates redundant ISO interval parsing and type conversions for the common case of interval-free comparisons. • **No Regression**: The `UnaryRangeExpr` path preserves exact comparison semantics by treating TIMESTAMPTZ as int64 directly, matching the storage format. For reverse comparisons (e.g., `'2025-01-01' > column`), operator reversal correctly normalizes to column-centric form (`column < '2025-01-01'`), maintaining logical equivalence. Interval-based comparisons continue through the unchanged `TimestamptzArithCompareExpr` path. • **Coverage**: Both forward (column left of operator) and reverse (column right of operator) comparison syntaxes are handled with explicit branching logic, ensuring the optimization applies uniformly across comparison patterns. <!-- end of auto-generated comment: release notes by coderabbit.ai --> Signed-off-by: zhenshan.cao <zhenshan.cao@zilliz.com> |
||
|
|
0114bd1dc9
|
feat: support match operator family (#46518)
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> |
||
|
|
0d70d2b98c
|
enhance: simplify seek position selection in WatchDmChannels (#46567)
issue: #46566 Remove the complex comparison logic between seekPosition and deleteCheckpoint. Use seekPosition directly since: - L0 segments are loaded before consuming message stream, which contain delete records from [deleteCheckpoint, L0.endPosition] - DataCoord ensures seekPosition is based on channel checkpoint, updated after data (including deletes) is flushed - L0 segments should cover up to seekPosition, avoiding data loss - This eliminates redundant message consumption when seekPosition > deleteCheckpoint <!-- This is an auto-generated comment: release notes by coderabbit.ai --> - Core invariant: L0 segments are loaded before consuming the DM channel stream and contain delete records for range [deleteCheckpoint, L0.endPosition]; DataCoord guarantees channel.GetSeekPosition() is derived from the channel checkpoint after data (including deletes) is flushed, so L0 segments collectively cover up to that seekPosition. - Change made: removed the prior branching that built a synthetic seek position from deleteCheckpoint vs. channel checkpoint and instead always calls channel.GetSeekPosition() (used directly in ConsumeMsgStream). Added an informational log comparing seekPosition and deleteCheckpoint. - Why the removed logic was redundant: deleteCheckpoint represented the smallest start position of L0 segments and was used to avoid re-consuming delete messages already present in loaded L0 segments. Because L0 segments already include deletes up to the channel checkpoint and DataCoord updates the channel checkpoint after flush, using deleteCheckpoint to alter the seek introduces duplicate consumption without benefit. - Why this is safe (no data loss/regression): L0 segments are guaranteed to be loaded before consumption, so deletes present in L0 cover the range up to channel.GetSeekPosition(); delete records earlier than deleteCheckpoint have been compacted to L1 and can be evicted from the delete buffer. The code path still calls ConsumeMsgStream with the channel seek position, preserving original consumption/error handling, so no messages are skipped and no additional delete application occurs beyond what L0/L1 already cover. <!-- end of auto-generated comment: release notes by coderabbit.ai --> Signed-off-by: Wei Liu <wei.liu@zilliz.com> |
||
|
|
3c2cf2c066
|
feat: Add nullable vector support in import utility layer (#46142)
related: #45993 Add nullable vector support in import utility layer Key changes: ImportV2 util: - Add nullable vector types (FloatVector, Float16Vector, BFloat16Vector, BinaryVector, SparseFloatVector, Int8Vector) to AppendNullableDefaultFieldsData() - Add tests for nullable vector field data appending CSV/JSON/Numpy readers: - Add nullPercent parameter to test data generation for better null coverage - Mark vector fields as nullable in test schemas - Add test cases for nullable vector field parsing - Refactor tests to use loop-based approach with 0%, 50%, 100% null percentages Parquet field reader: - Add ReadNullableBinaryData() for nullable BinaryVector/Float16Vector/BFloat16Vector - Add ReadNullableFloatVectorData() for nullable FloatVector - Add ReadNullableSparseFloatVectorData() for nullable SparseFloatVector - Add ReadNullableInt8VectorData() for nullable Int8Vector - Add ReadNullableStructData() for generic nullable struct data - Update Next() to use nullable read methods when field is nullable - Add null data validation for non-nullable fields <!-- This is an auto-generated comment: release notes by coderabbit.ai --> - Core invariant: import must preserve per-row alignment and validity for every field — nullable vector fields are expected to be encoded with per-row validity masks and all readers/writers must emit arrays aligned to original input rows (null entries represented explicitly). - New feature & scope: adds end-to-end nullable-vector support in the import utility layer — AppendNullableDefaultFieldsData in internal/datanode/importv2/util.go now appends nil placeholders for nullable vectors (FloatVector, Float16Vector, BFloat16Vector, BinaryVector, SparseFloatVector, Int8Vector); parquet reader (internal/util/importutilv2/parquet/field_reader.go) adds ReadNullableBinaryData, ReadNullableFloatVectorData, ReadNullableSparseFloatVectorData, ReadNullableInt8VectorData, ReadNullableStructData and routes nullable branches to these helpers; CSV/JSON/Numpy readers and test utilities updated to generate and validate 0/50/100% null scenarios and mark vector fields as nullable in test schemas. - Logic removed / simplified: eliminates ad-hoc "parameter-invalid" rejections for nullable vectors inside FieldReader.Next by centralizing nullable handling into ReadNullable* helpers and shared validators (getArrayDataNullable, checkNullableVectorAlignWithDim/checkNullableVectorAligned), simplifying control flow and removing scattered special-case checks. - No data loss / no regression (concrete code paths): nulls are preserved end-to-end — AppendNullableDefaultFieldsData explicitly inserts nil entries per null row (datanode import append path); ReadNullable*Data helpers return both data and []bool validity masks so callers in field_reader.go and downstream readers receive exact per-row validity; testutil.BuildSparseVectorData was extended to accept validData so sparse vectors are materialized only for valid rows while null rows are represented as missing. These concrete paths ensure null rows are represented rather than dropped, preventing data loss or behavioral regression. <!-- end of auto-generated comment: release notes by coderabbit.ai --> Signed-off-by: marcelo-cjl <marcelo.chen@zilliz.com> |
||
|
|
0a54c93227
|
fix: etcd RPC size limit exceeded when dropping collection (#46414)
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> |
||
|
|
55feb7ded8
|
feat: set related resource ids in collection schema (#46423)
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> |
||
|
|
512884524b
|
enhance: Maintain compatibility with the legacy FlushAll (#46564)
issue: https://github.com/milvus-io/milvus/issues/45919 <!-- This is an auto-generated comment: release notes by coderabbit.ai --> - Core invariant: FlushAll verification must accept both per-channel FlushAllTss (new schema) and the legacy single FlushAllTs; GetFlushAllState chooses the verification path based on which field is present and treats a channel as flushed only if its channel checkpoint timestamp >= the applicable threshold (per-channel timestamp or legacy FlushAllTs). - Logic removed/simplified: The previous mixed/ambiguous checks were split into two focused routines—verifyFlushAllStateByChannelFlushAllTs(logger, channel, flushAllTss) and verifyFlushAllStateByLegacyFlushAllTs(logger, channel, flushAllTs)—and GetFlushAllState now selects one path. This centralizes compatibility logic, eliminates interleaved/duplicated checks, and retains the outer-loop short-circuiting on the first unflushed channel. - Why this does NOT cause data loss or regressions: Changes only affect read-only verification paths (GetFlushAllState/GetFlushState) that compare in-memory channel checkpoints (meta.GetChannelCheckpoint) to provided thresholds; no writes to checkpoints or persisted state occur and FlushAll enqueue/wait behavior is unchanged. Unit tests were added to cover legacy FlushAllTs behavior and the new FlushAllMsgs→FlushAllTs extraction, exercising both code paths. - Enhancement scope and location: Adds backward-compatible support and concrete FlushAllTs extraction from streaming FlushAllMsgs in Proxy (internal/proxy/task_flush_all_streaming.go) and compatibility verifiers in DataCoord (internal/datacoord/services.go), plus corresponding tests (internal/datacoord/services_test.go, internal/proxy/*_test.go). <!-- end of auto-generated comment: release notes by coderabbit.ai --> --------- Signed-off-by: bigsheeper <yihao.dai@zilliz.com> |
||
|
|
8d12bfb436
|
fix: Restore the compaction task correctly to ensure it can be properly cleaned up (#46577)
issue: #46576 <!-- This is an auto-generated comment: release notes by coderabbit.ai --> - Core invariant: During meta load, only tasks that are truly terminal-cleaned (states cleaned or unknown) should be dropped; all other non-terminal tasks (including timeout and completed) must be restored so the inspector can reattach them to executing/cleaning queues and finish their cleanup lifecycle. - Removed/simplified logic: loadMeta no longer uses the broad isCompactionTaskFinished predicate (which treated timeout, completed, cleaned, unknown as terminal). It now uses the new isCompactionTaskCleaned predicate that only treats cleaned/unknown as terminal. This removes the redundant exclusion of timeout/completed tasks and simplifies the guard to drop only cleaned/unknown tasks. - Bug fix (root cause & exact change): Fixes issue #46576 — the previous isCompactionTaskFinished caused timeout/completed tasks to be skipped during meta load and thus not passed into restoreTask(). The PR adds isCompactionTaskCleaned and replaces the finished check so timeout and completed tasks are included in restoreTask() and re-attached to the inspector’s existing executing/cleaning queues. - No data loss or regression: Tasks in cleaned/unknown remain dropped (isCompactionTaskCleaned still returns true for cleaned/unknown). Non-terminal timeout/completed tasks now follow the same restoreTask() control path used previously for restored tasks — they are enqueued into the inspector’s queue/executing/cleaning flows rather than being discarded. No exported signatures changed and all restored tasks flow into existing handlers, avoiding behavior regression or data loss. <!-- end of auto-generated comment: release notes by coderabbit.ai --> Signed-off-by: Cai Zhang <cai.zhang@zilliz.com> |
||
|
|
e0fd091d41
|
fix: Fix replicate lag when server is idle (#46574)
issue: https://github.com/milvus-io/milvus/issues/46116 <!-- This is an auto-generated comment: release notes by coderabbit.ai --> - Core invariant: the metric CDCLastReplicatedTimeTick must reflect the most recent time-tick when replication has effectively processed all pending messages (including idle periods), so reported replicate lag = confirmed WAL tick − last replicated tick can reach zero when the server is idle. - Exact fix (bug): addresses issue #46116 by ensuring the last-replicated metric is updated when the server is idle. Concretely, a new ReplicateMetrics.UpdateLastReplicatedTimeTick(ts uint64) was added and called from OnConfirmed (OnConfirmed now delegates to UpdateLastReplicatedTimeTick(msg.TimeTick())), and from Replicate’s self-controlled-message path when the pending queue is empty — so the code records the time tick before returning ErrReplicateIgnored. - Logic simplified / removed: direct, ad-hoc metric writes in OnConfirmed were replaced by a single UpdateLastReplicatedTimeTick helper on the metrics implementation. The scattered manual set of CDCLastReplicatedTimeTick is consolidated into one method, removing redundant direct metric manipulations and centralizing timestamp conversion (tsoutil.PhysicalTimeSeconds). - No data loss / no behavior regression: this change only updates monitoring metrics and does not alter replication control flow or message processing. Replicate still returns ErrReplicateIgnored for self-controlled messages and does not change message persistence or acknowledgement paths; OnConfirmed continues to be invoked on confirmed messages but now delegates metric recording to the new method. Therefore no replication state, message ordering, or persistence semantics are modified. <!-- end of auto-generated comment: release notes by coderabbit.ai --> Signed-off-by: bigsheeper <yihao.dai@zilliz.com> |
||
|
|
2c2cbe89c2
|
fix: flush log when os exit (#46608)
issue: #45640 Signed-off-by: chyezh <chyezh@outlook.com> |
||
|
|
ef6d9c25c2
|
fix: check final result only in LeaderCacheObserver flaky test (#46601)
Related to #46600 The test previously checked if all 3 collection IDs were batched together in a single InvalidateShardLeaderCache call. This caused flakiness because the observer may split events across multiple calls. Fix by accumulating all collection IDs across multiple calls and verifying that eventually all expected IDs (1, 2, 3) are processed. <!-- This is an auto-generated comment: release notes by coderabbit.ai --> - Core invariant: the test asserts that all registered collection IDs {1,2,3} are eventually processed by InvalidateShardLeaderCache across any number of calls — i.e., the observer must invalidate every registered collection ID, not necessarily in a single batched RPC (fixes flaky assumption from issue #46600). - Logic removed/simplified: the strict expectation that all three IDs arrive in one InvalidateShardLeaderCache call was replaced by accumulating IDs into a ConcurrentSet (collectionIDs.Upsert in the mock) and asserting eventual containment of 1,2,3. This removes the brittle per-call batching assertion and uses a set-based accumulation (lines where the mock calls Upsert and final Eventually checks collectionIDs.Contain(...)). - Why this is safe (no data loss or behavior regression): only test assertions changed — production code (LeaderCacheObserver calling InvalidateShardLeaderCache) is unchanged. The mock intercepts InvalidateShardLeaderCache and accumulates req.GetCollectionIDs(); the test still verifies single-ID handling via the existing len==1 && lo.Contains(... ) check (first mock block) and verifies that all IDs were invalidated over time in the batch scenario (second mock block). No production code paths were modified, so invalidation behavior and RPC usage remain identical. - Bug-fix note: this is a targeted test-only fix for issue #46600 — it tolerates legitimate splitting of events across multiple InvalidateShardLeaderCache invocations by aggregating IDs across calls in the test mock, eliminating flakiness without altering runtime behavior. <!-- end of auto-generated comment: release notes by coderabbit.ai --> Signed-off-by: Congqi Xia <congqi.xia@zilliz.com> |
||
|
|
fc45905ee0
|
enhance: Optimize QuotaCenter CPU usage (#46388)
issue: https://github.com/milvus-io/milvus/issues/46387 --------- Signed-off-by: sijie-ni-0214 <sijie.ni@zilliz.com> |