mirror of
https://gitee.com/milvus-io/milvus.git
synced 2026-01-07 19:31:51 +08:00
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>
203 lines
7.5 KiB
Go
203 lines
7.5 KiB
Go
// Licensed to the LF AI & Data foundation under one
|
|
// or more contributor license agreements. See the NOTICE file
|
|
// distributed with this work for additional information
|
|
// regarding copyright ownership. The ASF licenses this file
|
|
// to you under the Apache License, Version 2.0 (the
|
|
// "License"); you may not use this file except in compliance
|
|
// with the License. You may obtain a copy of the License at
|
|
//
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
//
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
package datacoord
|
|
|
|
import (
|
|
"context"
|
|
"fmt"
|
|
|
|
"go.uber.org/zap"
|
|
|
|
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster"
|
|
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/broadcast"
|
|
"github.com/milvus-io/milvus/internal/streamingcoord/server/broadcaster/registry"
|
|
"github.com/milvus-io/milvus/pkg/v2/log"
|
|
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
|
|
)
|
|
|
|
// RegisterDDLCallbacks registers the ddl callbacks.
|
|
func RegisterDDLCallbacks(s *Server) {
|
|
ddlCallback := &DDLCallbacks{
|
|
Server: s,
|
|
}
|
|
ddlCallback.registerIndexCallbacks()
|
|
registry.RegisterFlushAllV2AckCallback(ddlCallback.flushAllV2AckCallback)
|
|
ddlCallback.registerSnapshotCallbacks()
|
|
}
|
|
|
|
type DDLCallbacks struct {
|
|
*Server
|
|
}
|
|
|
|
func (c *DDLCallbacks) registerIndexCallbacks() {
|
|
registry.RegisterCreateIndexV2AckCallback(c.createIndexV2AckCallback)
|
|
registry.RegisterAlterIndexV2AckCallback(c.alterIndexV2AckCallback)
|
|
registry.RegisterDropIndexV2AckCallback(c.dropIndexV2Callback)
|
|
}
|
|
|
|
func (c *DDLCallbacks) registerSnapshotCallbacks() {
|
|
registry.RegisterCreateSnapshotV2AckCallback(c.createSnapshotV2AckCallback)
|
|
registry.RegisterDropSnapshotV2AckCallback(c.dropSnapshotV2AckCallback)
|
|
registry.RegisterRestoreSnapshotV2AckCallback(c.restoreSnapshotV2AckCallback)
|
|
}
|
|
|
|
// startBroadcastWithCollectionID starts a broadcast with collection name.
|
|
func (s *Server) startBroadcastWithCollectionID(ctx context.Context, collectionID int64) (broadcaster.BroadcastAPI, error) {
|
|
coll, err := s.broker.DescribeCollectionInternal(ctx, collectionID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
dbName := coll.GetDbName()
|
|
collectionName := coll.GetCollectionName()
|
|
broadcaster, err := broadcast.StartBroadcastWithResourceKeys(ctx, message.NewSharedDBNameResourceKey(dbName), message.NewExclusiveCollectionNameResourceKey(dbName, collectionName))
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
return broadcaster, nil
|
|
}
|
|
|
|
// startBroadcastForRestoreSnapshot starts a broadcast for restore snapshot operations.
|
|
// Unlike startBroadcastRestoreSnapshot, this function does NOT validate resources -
|
|
// it only creates the broadcaster with appropriate resource keys (collection + snapshot).
|
|
// Use this when you need a broadcaster before all resources are created (e.g., for index restoration).
|
|
func (s *Server) startBroadcastForRestoreSnapshot(ctx context.Context, collectionID int64, snapshotName string) (broadcaster.BroadcastAPI, error) {
|
|
coll, err := s.broker.DescribeCollectionInternal(ctx, collectionID)
|
|
if err != nil {
|
|
return nil, fmt.Errorf("collection %d does not exist: %w", collectionID, err)
|
|
}
|
|
dbName := coll.GetDbName()
|
|
collectionName := coll.GetCollectionName()
|
|
|
|
b, err := broadcast.StartBroadcastWithResourceKeys(
|
|
ctx,
|
|
message.NewSharedDBNameResourceKey(dbName),
|
|
message.NewExclusiveCollectionNameResourceKey(dbName, collectionName),
|
|
message.NewExclusiveSnapshotNameResourceKey(snapshotName),
|
|
)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
log.Ctx(ctx).Info("broadcast started for restore snapshot",
|
|
zap.Int64("collectionID", collectionID),
|
|
zap.String("snapshotName", snapshotName))
|
|
return b, nil
|
|
}
|
|
|
|
// validateRestoreSnapshotResources validates that all required resources exist for restore.
|
|
// This includes collection, partitions, and indexes.
|
|
func (s *Server) validateRestoreSnapshotResources(ctx context.Context, collectionID int64, snapshotData *SnapshotData) error {
|
|
log := log.Ctx(ctx).With(zap.Int64("collectionID", collectionID))
|
|
|
|
// Validate partitions exist
|
|
partitionsResp, err := s.broker.ShowPartitions(ctx, collectionID)
|
|
if err != nil {
|
|
return fmt.Errorf("failed to get partitions for collection %d: %w", collectionID, err)
|
|
}
|
|
|
|
existingPartitions := make(map[string]bool)
|
|
for _, name := range partitionsResp.GetPartitionNames() {
|
|
existingPartitions[name] = true
|
|
}
|
|
|
|
for partName := range snapshotData.Collection.GetPartitions() {
|
|
if !existingPartitions[partName] {
|
|
return fmt.Errorf("partition %s does not exist in collection %d", partName, collectionID)
|
|
}
|
|
}
|
|
log.Info("partitions validated", zap.Int("count", len(existingPartitions)))
|
|
|
|
return nil
|
|
}
|
|
|
|
// startBroadcastRestoreSnapshot starts a broadcast for restore snapshot.
|
|
// It validates that all previously created resources (collection, partitions, indexes)
|
|
// exist before starting the broadcast.
|
|
// Deprecated: Use startBroadcastForRestoreSnapshot + validateRestoreSnapshotResources instead.
|
|
func (s *Server) startBroadcastRestoreSnapshot(
|
|
ctx context.Context,
|
|
collectionID int64,
|
|
snapshotData *SnapshotData,
|
|
) (broadcaster.BroadcastAPI, error) {
|
|
log := log.Ctx(ctx).With(zap.Int64("collectionID", collectionID))
|
|
|
|
// ========== Validate Collection Exists ==========
|
|
coll, err := s.broker.DescribeCollectionInternal(ctx, collectionID)
|
|
if err != nil {
|
|
return nil, fmt.Errorf("collection %d does not exist: %w", collectionID, err)
|
|
}
|
|
dbName := coll.GetDbName()
|
|
collectionName := coll.GetCollectionName()
|
|
log.Info("collection validated",
|
|
zap.String("dbName", dbName),
|
|
zap.String("collectionName", collectionName))
|
|
|
|
// ========== Validate Partitions Exist ==========
|
|
partitionsResp, err := s.broker.ShowPartitions(ctx, collectionID)
|
|
if err != nil {
|
|
return nil, fmt.Errorf("failed to get partitions for collection %d: %w", collectionID, err)
|
|
}
|
|
|
|
// Build set of existing partition names
|
|
existingPartitions := make(map[string]bool)
|
|
for _, name := range partitionsResp.GetPartitionNames() {
|
|
existingPartitions[name] = true
|
|
}
|
|
|
|
// Check all snapshot partitions exist
|
|
for partName := range snapshotData.Collection.GetPartitions() {
|
|
if !existingPartitions[partName] {
|
|
return nil, fmt.Errorf("partition %s does not exist in collection %d",
|
|
partName, collectionID)
|
|
}
|
|
}
|
|
log.Info("partitions validated", zap.Int("count", len(existingPartitions)))
|
|
|
|
// ========== Validate Indexes Exist ==========
|
|
for _, indexInfo := range snapshotData.Indexes {
|
|
// Check if index exists for this field
|
|
indexes := s.meta.indexMeta.GetIndexesForCollection(collectionID, "")
|
|
|
|
indexFound := false
|
|
for _, idx := range indexes {
|
|
if idx.FieldID == indexInfo.GetFieldID() && idx.IndexName == indexInfo.GetIndexName() {
|
|
indexFound = true
|
|
break
|
|
}
|
|
}
|
|
if !indexFound {
|
|
return nil, fmt.Errorf("index %s for field %d does not exist in collection %d",
|
|
indexInfo.GetIndexName(), indexInfo.GetFieldID(), collectionID)
|
|
}
|
|
}
|
|
log.Info("indexes validated", zap.Int("count", len(snapshotData.Indexes)))
|
|
|
|
// ========== Start Broadcast ==========
|
|
b, err := broadcast.StartBroadcastWithResourceKeys(
|
|
ctx,
|
|
message.NewSharedDBNameResourceKey(dbName),
|
|
message.NewExclusiveCollectionNameResourceKey(dbName, collectionName),
|
|
message.NewExclusiveSnapshotNameResourceKey(snapshotData.SnapshotInfo.GetName()),
|
|
)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
log.Info("broadcast started for restore snapshot")
|
|
return b, nil
|
|
}
|