Adjust rename events metadata format (#8854)
* rename metadata events * fix subscription filter to use NewEntry.Name for rename path matching The server-side subscription filter constructed the new path using OldEntry.Name instead of NewEntry.Name when checking if a rename event's destination matches the subscriber's path prefix. This could cause events to be incorrectly filtered when a rename changes the file name. * fix bucket events to handle rename of bucket directories onBucketEvents only checked IsCreate and IsDelete. A bucket directory rename via AtomicRenameEntry now emits a single rename event (both OldEntry and NewEntry non-nil), which matched neither check. Handle IsRename by deleting the old bucket and creating the new one. * fix replicator to handle rename events across directory boundaries Two issues fixed: 1. The replicator filtered events by checking if the key (old path) was under the source directory. Rename events now use the old path as key, so renames from outside into the watched directory were silently dropped. Now both old and new paths are checked, and cross-boundary renames are converted to create or delete. 2. NewParentPath was passed to the sink without remapping to the sink's target directory structure, causing the sink to write entries at the wrong location. Now NewParentPath is remapped alongside the key. * fix filer sync to handle rename events crossing directory boundaries The early directory-prefix filter only checked resp.Directory (old parent). Rename events now carry the old parent as Directory, so renames from outside the source path into it were dropped before reaching the existing cross-boundary handling logic. Check both old and new directories against sourcePath and excludePaths so the downstream old-key/new-key logic can properly convert these to create or delete operations. * fix metadata event path matching * fix metadata event consumers for rename targets * Fix replication rename target keys Logical rename events now reach replication sinks with distinct source and target paths.\n\nHandle non-filer sinks as delete-plus-create on the translated target key, and make the rename fallback path create at the translated target key too.\n\nAdd focused tests covering non-filer renames, filer rename updates, and the fallback path.\n\nCo-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> * Fix filer sync rename path scoping Use directory-boundary matching instead of raw prefix checks when classifying source and target paths during filer sync.\n\nAlso apply excludePaths per side so renames across excluded boundaries downgrade cleanly to create/delete instead of being misclassified as in-scope updates.\n\nAdd focused tests for boundary matching and rename classification.\n\nCo-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> * Fix replicator directory boundary checks Use directory-boundary matching instead of raw prefix checks when deciding whether a source or target path is inside the watched tree or an excluded subtree.\n\nThis prevents sibling paths such as /foo and /foobar from being misclassified during rename handling, and preserves the earlier rename-target-key fix.\n\nAdd focused tests for boundary matching and rename classification across sibling/excluded directories.\n\nCo-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> * Fix etc-remote rename-out handling Use boundary-safe source/target directory membership when classifying metadata events under DirectoryEtcRemote.\n\nThis prevents rename-out events from being processed as config updates, while still treating them as removals where appropriate for the remote sync and remote gateway command paths.\n\nAdd focused tests for update/removal classification and sibling-prefix handling.\n\nCo-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> * Defer rename events until commit Queue logical rename metadata events during atomic and streaming renames and publish them only after the transaction commits successfully.\n\nThis prevents subscribers from seeing delete or logical rename events for operations that later fail during delete or commit.\n\nAlso serialize notification.Queue swaps in rename tests and add failure-path coverage.\n\nCo-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> * Skip descendant rename target lookups Avoid redundant target lookups during recursive directory renames once the destination subtree is known absent.\n\nThe recursive move path now inserts known-absent descendants directly, and the test harness exercises prefixed directory listing so the optimization is covered by a directory rename regression test.\n\nCo-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> * Tighten rename review tests Return filer_pb.ErrNotFound from the bucket tracking store test stub so it follows the FilerStore contract, and add a webhook filter case for same-name renames across parent directories.\n\nCo-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> * fix HardLinkId format verb in InsertEntryKnownAbsent error HardLinkId is a byte slice. %d prints each byte as a decimal number which is not useful for an identifier. Use %x to match the log line two lines above. * only skip descendant target lookup when source and dest use same store moveFolderSubEntries unconditionally passed skipTargetLookup=true for every descendant. This is safe when all paths resolve to the same underlying store, but with path-specific store configuration a child's destination may map to a different backend that already holds an entry at that path. Use FilerStoreWrapper.SameActualStore to check per-child and fall back to the full CreateEntry path when stores differ. * add nil and create edge-case tests for metadata event scope helpers * extract pathIsEqualOrUnder into util.IsEqualOrUnder Identical implementations existed in both replication/replicator.go and command/filer_sync.go. Move to util.IsEqualOrUnder (alongside the existing FullPath.IsUnder) and remove the duplicates. * use MetadataEventTargetDirectory for new-side directory in filer sync The new-side directory checks and sourceNewKey computation used message.NewParentPath directly. If NewParentPath were empty (legacy events, older filer versions during rolling upgrades), sourceNewKey would be wrong (/filename instead of /dir/filename) and the UpdateEntry parent path rewrite would panic on slice bounds. Derive targetDir once from MetadataEventTargetDirectory, which falls back to resp.Directory when NewParentPath is empty, and use it consistently for all new-side checks and the sink parent path.
This commit is contained in:
@@ -34,7 +34,8 @@ func (fs *FilerServer) AtomicRenameEntry(ctx context.Context, req *filer_pb.Atom
|
||||
return nil, fmt.Errorf("%s/%s not found: %v", req.OldDirectory, req.OldName, err)
|
||||
}
|
||||
|
||||
moveErr := fs.moveEntry(ctx, nil, oldParent, oldEntry, newParent, req.NewName, req.Signatures)
|
||||
var metadataEvents []metadataEvent
|
||||
moveErr := fs.moveEntry(ctx, nil, oldParent, oldEntry, newParent, req.NewName, req.Signatures, false, &metadataEvents)
|
||||
if moveErr != nil {
|
||||
fs.filer.RollbackTransaction(ctx)
|
||||
return nil, fmt.Errorf("%s/%s move error: %v", req.OldDirectory, req.OldName, moveErr)
|
||||
@@ -44,6 +45,9 @@ func (fs *FilerServer) AtomicRenameEntry(ctx context.Context, req *filer_pb.Atom
|
||||
return nil, fmt.Errorf("%s/%s move commit error: %v", req.OldDirectory, req.OldName, commitError)
|
||||
}
|
||||
}
|
||||
for _, event := range metadataEvents {
|
||||
event.notify(fs.filer, ctx, req.Signatures)
|
||||
}
|
||||
|
||||
return &filer_pb.AtomicRenameEntryResponse{}, nil
|
||||
}
|
||||
@@ -87,7 +91,8 @@ func (fs *FilerServer) StreamRenameEntry(req *filer_pb.StreamRenameEntryRequest,
|
||||
}
|
||||
}
|
||||
|
||||
moveErr := fs.moveEntry(ctx, stream, oldParent, oldEntry, newParent, req.NewName, req.Signatures)
|
||||
var metadataEvents []metadataEvent
|
||||
moveErr := fs.moveEntry(ctx, stream, oldParent, oldEntry, newParent, req.NewName, req.Signatures, false, &metadataEvents)
|
||||
if moveErr != nil {
|
||||
fs.filer.RollbackTransaction(ctx)
|
||||
return fmt.Errorf("%s/%s move error: %v", req.OldDirectory, req.OldName, moveErr)
|
||||
@@ -97,27 +102,40 @@ func (fs *FilerServer) StreamRenameEntry(req *filer_pb.StreamRenameEntryRequest,
|
||||
return fmt.Errorf("%s/%s move commit error: %v", req.OldDirectory, req.OldName, commitError)
|
||||
}
|
||||
}
|
||||
for _, event := range metadataEvents {
|
||||
event.notify(fs.filer, ctx, req.Signatures)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (fs *FilerServer) moveEntry(ctx context.Context, stream filer_pb.SeaweedFiler_StreamRenameEntryServer, oldParent util.FullPath, entry *filer.Entry, newParent util.FullPath, newName string, signatures []int32) error {
|
||||
type metadataEvent struct {
|
||||
oldEntry *filer.Entry
|
||||
newEntry *filer.Entry
|
||||
deleteChunks bool
|
||||
}
|
||||
|
||||
func (event metadataEvent) notify(f *filer.Filer, ctx context.Context, signatures []int32) {
|
||||
f.NotifyUpdateEvent(ctx, event.oldEntry, event.newEntry, event.deleteChunks, false, signatures)
|
||||
}
|
||||
|
||||
func (fs *FilerServer) moveEntry(ctx context.Context, stream filer_pb.SeaweedFiler_StreamRenameEntryServer, oldParent util.FullPath, entry *filer.Entry, newParent util.FullPath, newName string, signatures []int32, skipTargetLookup bool, metadataEvents *[]metadataEvent) error {
|
||||
|
||||
if err := fs.moveSelfEntry(ctx, stream, oldParent, entry, newParent, newName, func() error {
|
||||
if entry.IsDirectory() {
|
||||
if err := fs.moveFolderSubEntries(ctx, stream, oldParent, entry, newParent, newName, signatures); err != nil {
|
||||
if err := fs.moveFolderSubEntries(ctx, stream, oldParent, entry, newParent, newName, signatures, metadataEvents); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}, signatures); err != nil {
|
||||
}, signatures, skipTargetLookup, metadataEvents); err != nil {
|
||||
return fmt.Errorf("fail to move %s => %s: %v", oldParent.Child(entry.Name()), newParent.Child(newName), err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (fs *FilerServer) moveFolderSubEntries(ctx context.Context, stream filer_pb.SeaweedFiler_StreamRenameEntryServer, oldParent util.FullPath, entry *filer.Entry, newParent util.FullPath, newName string, signatures []int32) error {
|
||||
func (fs *FilerServer) moveFolderSubEntries(ctx context.Context, stream filer_pb.SeaweedFiler_StreamRenameEntryServer, oldParent util.FullPath, entry *filer.Entry, newParent util.FullPath, newName string, signatures []int32, metadataEvents *[]metadataEvent) error {
|
||||
|
||||
currentDirPath := oldParent.Child(entry.Name())
|
||||
newDirPath := newParent.Child(newName)
|
||||
@@ -138,7 +156,9 @@ func (fs *FilerServer) moveFolderSubEntries(ctx context.Context, stream filer_pb
|
||||
for _, item := range entries {
|
||||
lastFileName = item.Name()
|
||||
// println("processing", lastFileName)
|
||||
err := fs.moveEntry(ctx, stream, currentDirPath, item, newDirPath, item.Name(), signatures)
|
||||
newChildPath := newDirPath.Child(item.Name())
|
||||
skipTarget := fs.filer.Store.SameActualStore(newDirPath, newChildPath)
|
||||
err := fs.moveEntry(ctx, stream, currentDirPath, item, newDirPath, item.Name(), signatures, skipTarget, metadataEvents)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
@@ -150,7 +170,7 @@ func (fs *FilerServer) moveFolderSubEntries(ctx context.Context, stream filer_pb
|
||||
return nil
|
||||
}
|
||||
|
||||
func (fs *FilerServer) moveSelfEntry(ctx context.Context, stream filer_pb.SeaweedFiler_StreamRenameEntryServer, oldParent util.FullPath, entry *filer.Entry, newParent util.FullPath, newName string, moveFolderSubEntries func() error, signatures []int32) error {
|
||||
func (fs *FilerServer) moveSelfEntry(ctx context.Context, stream filer_pb.SeaweedFiler_StreamRenameEntryServer, oldParent util.FullPath, entry *filer.Entry, newParent util.FullPath, newName string, moveFolderSubEntries func() error, signatures []int32, skipTargetLookup bool, metadataEvents *[]metadataEvent) error {
|
||||
|
||||
oldPath, newPath := oldParent.Child(entry.Name()), newParent.Child(newName)
|
||||
|
||||
@@ -161,6 +181,18 @@ func (fs *FilerServer) moveSelfEntry(ctx context.Context, stream filer_pb.Seawee
|
||||
return nil
|
||||
}
|
||||
|
||||
sourceEntry := entry.ShallowClone()
|
||||
sourceEntry.FullPath = oldPath
|
||||
|
||||
var existingTarget *filer.Entry
|
||||
if !skipTargetLookup {
|
||||
if targetEntry, findErr := fs.filer.FindEntry(ctx, newPath); findErr == nil {
|
||||
existingTarget = targetEntry.ShallowClone()
|
||||
} else if findErr != filer_pb.ErrNotFound {
|
||||
return findErr
|
||||
}
|
||||
}
|
||||
|
||||
// add to new directory
|
||||
newEntry := &filer.Entry{
|
||||
FullPath: newPath,
|
||||
@@ -173,8 +205,17 @@ func (fs *FilerServer) moveSelfEntry(ctx context.Context, stream filer_pb.Seawee
|
||||
Remote: entry.Remote,
|
||||
Quota: entry.Quota,
|
||||
}
|
||||
if createErr := fs.filer.CreateEntry(ctx, newEntry, false, false, signatures, false, fs.filer.MaxFilenameLength); createErr != nil {
|
||||
return createErr
|
||||
if skipTargetLookup {
|
||||
if newEntry.FullPath.IsLongerFileName(fs.filer.MaxFilenameLength) {
|
||||
return filer_pb.ErrEntryNameTooLong
|
||||
}
|
||||
if createErr := fs.filer.Store.InsertEntryKnownAbsent(filer.WithSuppressedMetadataEvents(ctx), newEntry); createErr != nil {
|
||||
return fmt.Errorf("insert entry %s: %v", newEntry.FullPath, createErr)
|
||||
}
|
||||
} else {
|
||||
if createErr := fs.filer.CreateEntry(filer.WithSuppressedMetadataEvents(ctx), newEntry, false, false, signatures, false, fs.filer.MaxFilenameLength); createErr != nil {
|
||||
return createErr
|
||||
}
|
||||
}
|
||||
if stream != nil {
|
||||
if err := stream.Send(&filer_pb.StreamRenameEntryResponse{
|
||||
@@ -195,6 +236,17 @@ func (fs *FilerServer) moveSelfEntry(ctx context.Context, stream filer_pb.Seawee
|
||||
}
|
||||
}
|
||||
|
||||
if existingTarget != nil {
|
||||
*metadataEvents = append(*metadataEvents, metadataEvent{
|
||||
oldEntry: existingTarget,
|
||||
deleteChunks: true,
|
||||
})
|
||||
}
|
||||
*metadataEvents = append(*metadataEvents, metadataEvent{
|
||||
oldEntry: sourceEntry,
|
||||
newEntry: newEntry,
|
||||
})
|
||||
|
||||
if moveFolderSubEntries != nil {
|
||||
if moveChildrenErr := moveFolderSubEntries(); moveChildrenErr != nil {
|
||||
return moveChildrenErr
|
||||
@@ -203,7 +255,7 @@ func (fs *FilerServer) moveSelfEntry(ctx context.Context, stream filer_pb.Seawee
|
||||
|
||||
// delete old entry
|
||||
ctx = context.WithValue(ctx, "OP", "MV")
|
||||
deleteErr := fs.filer.DeleteEntryMetaAndData(ctx, oldPath, false, false, false, false, signatures, 0)
|
||||
deleteErr := fs.filer.DeleteEntryMetaAndData(filer.WithSuppressedMetadataEvents(ctx), oldPath, false, false, false, false, signatures, 0)
|
||||
if deleteErr != nil {
|
||||
return deleteErr
|
||||
}
|
||||
|
||||
472
weed/server/filer_grpc_server_rename_test.go
Normal file
472
weed/server/filer_grpc_server_rename_test.go
Normal file
@@ -0,0 +1,472 @@
|
||||
package weed_server
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"os"
|
||||
"sort"
|
||||
"strings"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/seaweedfs/seaweedfs/weed/cluster"
|
||||
"github.com/seaweedfs/seaweedfs/weed/filer"
|
||||
"github.com/seaweedfs/seaweedfs/weed/notification"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
|
||||
"github.com/seaweedfs/seaweedfs/weed/wdclient"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
type renameTestStore struct {
|
||||
mu sync.Mutex
|
||||
entries map[string]*filer.Entry
|
||||
findCalls map[string]int
|
||||
commitErr error
|
||||
deleteErr error
|
||||
}
|
||||
|
||||
func newRenameTestStore() *renameTestStore {
|
||||
return &renameTestStore{
|
||||
entries: make(map[string]*filer.Entry),
|
||||
findCalls: make(map[string]int),
|
||||
}
|
||||
}
|
||||
|
||||
func (s *renameTestStore) GetName() string { return "rename_test" }
|
||||
func (s *renameTestStore) Initialize(util.Configuration, string) error { return nil }
|
||||
func (s *renameTestStore) Shutdown() {}
|
||||
func (s *renameTestStore) BeginTransaction(ctx context.Context) (context.Context, error) {
|
||||
return ctx, nil
|
||||
}
|
||||
func (s *renameTestStore) CommitTransaction(context.Context) error { return s.commitErr }
|
||||
func (s *renameTestStore) RollbackTransaction(context.Context) error { return nil }
|
||||
func (s *renameTestStore) KvPut(context.Context, []byte, []byte) error {
|
||||
return nil
|
||||
}
|
||||
func (s *renameTestStore) KvGet(context.Context, []byte) ([]byte, error) {
|
||||
return nil, filer.ErrKvNotFound
|
||||
}
|
||||
func (s *renameTestStore) KvDelete(context.Context, []byte) error { return nil }
|
||||
|
||||
func (s *renameTestStore) InsertEntry(_ context.Context, entry *filer.Entry) error {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
s.entries[string(entry.FullPath)] = entry.ShallowClone()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *renameTestStore) UpdateEntry(_ context.Context, entry *filer.Entry) error {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
s.entries[string(entry.FullPath)] = entry.ShallowClone()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *renameTestStore) FindEntry(_ context.Context, p util.FullPath) (*filer.Entry, error) {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
s.findCalls[string(p)]++
|
||||
entry, found := s.entries[string(p)]
|
||||
if !found {
|
||||
return nil, filer_pb.ErrNotFound
|
||||
}
|
||||
return entry.ShallowClone(), nil
|
||||
}
|
||||
|
||||
func (s *renameTestStore) DeleteEntry(_ context.Context, p util.FullPath) error {
|
||||
if s.deleteErr != nil {
|
||||
return s.deleteErr
|
||||
}
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
delete(s.entries, string(p))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *renameTestStore) DeleteFolderChildren(_ context.Context, p util.FullPath) error {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
prefix := string(p) + "/"
|
||||
for path := range s.entries {
|
||||
if len(path) > len(prefix) && path[:len(prefix)] == prefix {
|
||||
delete(s.entries, path)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *renameTestStore) listDirectoryEntries(dirPath util.FullPath, startFileName string, includeStartFile bool, limit int64, prefix string, eachEntryFunc filer.ListEachEntryFunc) (string, error) {
|
||||
s.mu.Lock()
|
||||
var entries []*filer.Entry
|
||||
for path, entry := range s.entries {
|
||||
if path == string(dirPath) {
|
||||
continue
|
||||
}
|
||||
parent, _ := util.FullPath(path).DirAndName()
|
||||
if parent != string(dirPath) {
|
||||
continue
|
||||
}
|
||||
if prefix != "" && !strings.HasPrefix(entry.Name(), prefix) {
|
||||
continue
|
||||
}
|
||||
entries = append(entries, entry.ShallowClone())
|
||||
}
|
||||
s.mu.Unlock()
|
||||
|
||||
sort.Slice(entries, func(i, j int) bool {
|
||||
return entries[i].Name() < entries[j].Name()
|
||||
})
|
||||
|
||||
count := int64(0)
|
||||
lastFileName := ""
|
||||
for _, entry := range entries {
|
||||
name := entry.Name()
|
||||
if startFileName != "" {
|
||||
if includeStartFile {
|
||||
if name < startFileName {
|
||||
continue
|
||||
}
|
||||
} else if name <= startFileName {
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
lastFileName = name
|
||||
if eachEntryFunc != nil {
|
||||
includeMore, err := eachEntryFunc(entry)
|
||||
if err != nil {
|
||||
return lastFileName, err
|
||||
}
|
||||
if !includeMore {
|
||||
return lastFileName, nil
|
||||
}
|
||||
}
|
||||
|
||||
count++
|
||||
if limit > 0 && count >= limit {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
return lastFileName, nil
|
||||
}
|
||||
|
||||
func (s *renameTestStore) ListDirectoryEntries(_ context.Context, dirPath util.FullPath, startFileName string, includeStartFile bool, limit int64, eachEntryFunc filer.ListEachEntryFunc) (string, error) {
|
||||
return s.listDirectoryEntries(dirPath, startFileName, includeStartFile, limit, "", eachEntryFunc)
|
||||
}
|
||||
|
||||
func (s *renameTestStore) ListDirectoryPrefixedEntries(_ context.Context, dirPath util.FullPath, startFileName string, includeStartFile bool, limit int64, prefix string, eachEntryFunc filer.ListEachEntryFunc) (string, error) {
|
||||
return s.listDirectoryEntries(dirPath, startFileName, includeStartFile, limit, prefix, eachEntryFunc)
|
||||
}
|
||||
|
||||
func (s *renameTestStore) findEntryCallCount(path string) int {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
return s.findCalls[path]
|
||||
}
|
||||
|
||||
type capturedEvent struct {
|
||||
key string
|
||||
notification *filer_pb.EventNotification
|
||||
}
|
||||
|
||||
type captureQueue struct {
|
||||
mu sync.Mutex
|
||||
events []capturedEvent
|
||||
}
|
||||
|
||||
var notificationQueueSwapMu sync.Mutex
|
||||
|
||||
func (q *captureQueue) GetName() string { return "capture" }
|
||||
func (q *captureQueue) Initialize(util.Configuration, string) error { return nil }
|
||||
func (q *captureQueue) SendMessage(key string, message proto.Message) error {
|
||||
notification, ok := message.(*filer_pb.EventNotification)
|
||||
if !ok {
|
||||
return nil
|
||||
}
|
||||
|
||||
q.mu.Lock()
|
||||
defer q.mu.Unlock()
|
||||
q.events = append(q.events, capturedEvent{
|
||||
key: key,
|
||||
notification: proto.Clone(notification).(*filer_pb.EventNotification),
|
||||
})
|
||||
return nil
|
||||
}
|
||||
|
||||
func (q *captureQueue) snapshot() []capturedEvent {
|
||||
q.mu.Lock()
|
||||
defer q.mu.Unlock()
|
||||
events := make([]capturedEvent, len(q.events))
|
||||
copy(events, q.events)
|
||||
return events
|
||||
}
|
||||
|
||||
func swapNotificationQueue(t *testing.T, q notification.MessageQueue) {
|
||||
t.Helper()
|
||||
notificationQueueSwapMu.Lock()
|
||||
prevQueue := notification.Queue
|
||||
notification.Queue = q
|
||||
t.Cleanup(func() {
|
||||
notification.Queue = prevQueue
|
||||
notificationQueueSwapMu.Unlock()
|
||||
})
|
||||
}
|
||||
|
||||
func newRenameTestFiler(store *renameTestStore) *filer.Filer {
|
||||
dialOption := grpc.WithTransportCredentials(insecure.NewCredentials())
|
||||
masterClient := wdclient.NewMasterClient(
|
||||
dialOption,
|
||||
"test",
|
||||
cluster.FilerType,
|
||||
pb.ServerAddress("localhost:0"),
|
||||
"",
|
||||
"",
|
||||
*pb.NewServiceDiscoveryFromMap(map[string]pb.ServerAddress{}),
|
||||
)
|
||||
|
||||
return &filer.Filer{
|
||||
Store: filer.NewFilerStoreWrapper(store),
|
||||
MasterClient: masterClient,
|
||||
FilerConf: filer.NewFilerConf(),
|
||||
RemoteStorage: filer.NewFilerRemoteStorage(),
|
||||
MaxFilenameLength: 255,
|
||||
LocalMetaLogBuffer: log_buffer.NewLogBuffer(
|
||||
"test",
|
||||
time.Minute,
|
||||
func(*log_buffer.LogBuffer, time.Time, time.Time, []byte, int64, int64) {},
|
||||
nil,
|
||||
func() {},
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
func newFileEntry(path string, inode uint64) *filer.Entry {
|
||||
now := time.Unix(1700000000, 0)
|
||||
return &filer.Entry{
|
||||
FullPath: util.FullPath(path),
|
||||
Attr: filer.Attr{
|
||||
Mtime: now,
|
||||
Crtime: now,
|
||||
Mode: 0644,
|
||||
Inode: inode,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func newDirectoryEntry(path string, inode uint64) *filer.Entry {
|
||||
now := time.Unix(1700000000, 0)
|
||||
return &filer.Entry{
|
||||
FullPath: util.FullPath(path),
|
||||
Attr: filer.Attr{
|
||||
Mtime: now,
|
||||
Crtime: now,
|
||||
Mode: os.ModeDir | 0755,
|
||||
Inode: inode,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func TestAtomicRenameEntryEmitsLogicalRenameEvent(t *testing.T) {
|
||||
store := newRenameTestStore()
|
||||
store.entries["/src.txt"] = newFileEntry("/src.txt", 101)
|
||||
|
||||
queue := &captureQueue{}
|
||||
swapNotificationQueue(t, queue)
|
||||
|
||||
server := &FilerServer{filer: newRenameTestFiler(store)}
|
||||
_, err := server.AtomicRenameEntry(context.Background(), &filer_pb.AtomicRenameEntryRequest{
|
||||
OldDirectory: "/",
|
||||
OldName: "src.txt",
|
||||
NewDirectory: "/",
|
||||
NewName: "dst.txt",
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("AtomicRenameEntry: %v", err)
|
||||
}
|
||||
|
||||
events := queue.snapshot()
|
||||
if len(events) != 1 {
|
||||
t.Fatalf("event count = %d, want 1", len(events))
|
||||
}
|
||||
|
||||
event := events[0]
|
||||
if event.key != "/src.txt" {
|
||||
t.Fatalf("event key = %q, want /src.txt", event.key)
|
||||
}
|
||||
if event.notification.OldEntry == nil || event.notification.OldEntry.Name != "src.txt" {
|
||||
t.Fatalf("old entry = %+v, want src.txt", event.notification.OldEntry)
|
||||
}
|
||||
if event.notification.NewEntry == nil || event.notification.NewEntry.Name != "dst.txt" {
|
||||
t.Fatalf("new entry = %+v, want dst.txt", event.notification.NewEntry)
|
||||
}
|
||||
if event.notification.NewParentPath != "/" {
|
||||
t.Fatalf("new parent path = %q, want /", event.notification.NewParentPath)
|
||||
}
|
||||
|
||||
if _, err := store.FindEntry(context.Background(), "/src.txt"); err != filer_pb.ErrNotFound {
|
||||
t.Fatalf("source entry error = %v, want %v", err, filer_pb.ErrNotFound)
|
||||
}
|
||||
dst, err := store.FindEntry(context.Background(), "/dst.txt")
|
||||
if err != nil {
|
||||
t.Fatalf("find destination: %v", err)
|
||||
}
|
||||
if dst.Attr.Inode != 101 {
|
||||
t.Fatalf("destination inode = %d, want 101", dst.Attr.Inode)
|
||||
}
|
||||
}
|
||||
|
||||
func TestAtomicRenameEntryOverwriteEmitsDeleteThenRename(t *testing.T) {
|
||||
store := newRenameTestStore()
|
||||
store.entries["/src.txt"] = newFileEntry("/src.txt", 101)
|
||||
store.entries["/dst.txt"] = newFileEntry("/dst.txt", 202)
|
||||
|
||||
queue := &captureQueue{}
|
||||
swapNotificationQueue(t, queue)
|
||||
|
||||
server := &FilerServer{filer: newRenameTestFiler(store)}
|
||||
_, err := server.AtomicRenameEntry(context.Background(), &filer_pb.AtomicRenameEntryRequest{
|
||||
OldDirectory: "/",
|
||||
OldName: "src.txt",
|
||||
NewDirectory: "/",
|
||||
NewName: "dst.txt",
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("AtomicRenameEntry: %v", err)
|
||||
}
|
||||
|
||||
events := queue.snapshot()
|
||||
if len(events) != 2 {
|
||||
t.Fatalf("event count = %d, want 2", len(events))
|
||||
}
|
||||
|
||||
deleteEvent := events[0]
|
||||
if deleteEvent.key != "/dst.txt" {
|
||||
t.Fatalf("delete event key = %q, want /dst.txt", deleteEvent.key)
|
||||
}
|
||||
if deleteEvent.notification.OldEntry == nil || deleteEvent.notification.OldEntry.Name != "dst.txt" {
|
||||
t.Fatalf("delete old entry = %+v, want dst.txt", deleteEvent.notification.OldEntry)
|
||||
}
|
||||
if deleteEvent.notification.NewEntry != nil {
|
||||
t.Fatalf("delete new entry = %+v, want nil", deleteEvent.notification.NewEntry)
|
||||
}
|
||||
if !deleteEvent.notification.DeleteChunks {
|
||||
t.Fatal("delete event should delete chunks")
|
||||
}
|
||||
|
||||
renameEvent := events[1]
|
||||
if renameEvent.key != "/src.txt" {
|
||||
t.Fatalf("rename event key = %q, want /src.txt", renameEvent.key)
|
||||
}
|
||||
if renameEvent.notification.OldEntry == nil || renameEvent.notification.OldEntry.Name != "src.txt" {
|
||||
t.Fatalf("rename old entry = %+v, want src.txt", renameEvent.notification.OldEntry)
|
||||
}
|
||||
if renameEvent.notification.NewEntry == nil || renameEvent.notification.NewEntry.Name != "dst.txt" {
|
||||
t.Fatalf("rename new entry = %+v, want dst.txt", renameEvent.notification.NewEntry)
|
||||
}
|
||||
if renameEvent.notification.NewParentPath != "/" {
|
||||
t.Fatalf("rename new parent path = %q, want /", renameEvent.notification.NewParentPath)
|
||||
}
|
||||
|
||||
if _, err := store.FindEntry(context.Background(), "/src.txt"); err != filer_pb.ErrNotFound {
|
||||
t.Fatalf("source entry error = %v, want %v", err, filer_pb.ErrNotFound)
|
||||
}
|
||||
dst, err := store.FindEntry(context.Background(), "/dst.txt")
|
||||
if err != nil {
|
||||
t.Fatalf("find destination: %v", err)
|
||||
}
|
||||
if dst.Attr.Inode != 101 {
|
||||
t.Fatalf("destination inode = %d, want 101", dst.Attr.Inode)
|
||||
}
|
||||
}
|
||||
|
||||
func TestAtomicRenameEntryDoesNotEmitEventOnDeleteFailure(t *testing.T) {
|
||||
store := newRenameTestStore()
|
||||
store.entries["/src.txt"] = newFileEntry("/src.txt", 101)
|
||||
store.deleteErr = errors.New("delete failed")
|
||||
|
||||
queue := &captureQueue{}
|
||||
swapNotificationQueue(t, queue)
|
||||
|
||||
server := &FilerServer{filer: newRenameTestFiler(store)}
|
||||
_, err := server.AtomicRenameEntry(context.Background(), &filer_pb.AtomicRenameEntryRequest{
|
||||
OldDirectory: "/",
|
||||
OldName: "src.txt",
|
||||
NewDirectory: "/",
|
||||
NewName: "dst.txt",
|
||||
})
|
||||
if err == nil {
|
||||
t.Fatal("expected delete failure")
|
||||
}
|
||||
|
||||
if events := queue.snapshot(); len(events) != 0 {
|
||||
t.Fatalf("event count = %d, want 0", len(events))
|
||||
}
|
||||
}
|
||||
|
||||
func TestAtomicRenameEntryDoesNotEmitEventOnCommitFailure(t *testing.T) {
|
||||
store := newRenameTestStore()
|
||||
store.entries["/src.txt"] = newFileEntry("/src.txt", 101)
|
||||
store.commitErr = errors.New("commit failed")
|
||||
|
||||
queue := &captureQueue{}
|
||||
swapNotificationQueue(t, queue)
|
||||
|
||||
server := &FilerServer{filer: newRenameTestFiler(store)}
|
||||
_, err := server.AtomicRenameEntry(context.Background(), &filer_pb.AtomicRenameEntryRequest{
|
||||
OldDirectory: "/",
|
||||
OldName: "src.txt",
|
||||
NewDirectory: "/",
|
||||
NewName: "dst.txt",
|
||||
})
|
||||
if err == nil {
|
||||
t.Fatal("expected commit failure")
|
||||
}
|
||||
|
||||
if events := queue.snapshot(); len(events) != 0 {
|
||||
t.Fatalf("event count = %d, want 0", len(events))
|
||||
}
|
||||
}
|
||||
|
||||
func TestAtomicRenameEntrySkipsDescendantTargetLookups(t *testing.T) {
|
||||
store := newRenameTestStore()
|
||||
store.entries["/srcdir"] = newDirectoryEntry("/srcdir", 100)
|
||||
store.entries["/srcdir/subdir"] = newDirectoryEntry("/srcdir/subdir", 101)
|
||||
store.entries["/srcdir/subdir/file.txt"] = newFileEntry("/srcdir/subdir/file.txt", 102)
|
||||
|
||||
queue := &captureQueue{}
|
||||
swapNotificationQueue(t, queue)
|
||||
|
||||
server := &FilerServer{filer: newRenameTestFiler(store)}
|
||||
_, err := server.AtomicRenameEntry(context.Background(), &filer_pb.AtomicRenameEntryRequest{
|
||||
OldDirectory: "/",
|
||||
OldName: "srcdir",
|
||||
NewDirectory: "/",
|
||||
NewName: "dstdir",
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("AtomicRenameEntry: %v", err)
|
||||
}
|
||||
|
||||
for _, target := range []string{"/dstdir/subdir", "/dstdir/subdir/file.txt"} {
|
||||
if calls := store.findEntryCallCount(target); calls != 0 {
|
||||
t.Fatalf("FindEntry(%q) called %d times, want 0", target, calls)
|
||||
}
|
||||
}
|
||||
|
||||
for _, target := range []string{"/dstdir", "/dstdir/subdir", "/dstdir/subdir/file.txt"} {
|
||||
if _, err := store.FindEntry(context.Background(), util.FullPath(target)); err != nil {
|
||||
t.Fatalf("find renamed target %q: %v", target, err)
|
||||
}
|
||||
}
|
||||
|
||||
if got := len(queue.snapshot()); got != 3 {
|
||||
t.Fatalf("event count = %d, want 3", got)
|
||||
}
|
||||
}
|
||||
@@ -47,10 +47,10 @@ const (
|
||||
// current time (backlog catch-up), multiple events are packed into a single
|
||||
// stream.Send() using the Events field. Otherwise events are sent one-by-one.
|
||||
type pipelinedSender struct {
|
||||
sendCh chan *filer_pb.SubscribeMetadataResponse
|
||||
errCh chan error
|
||||
done chan struct{}
|
||||
canBatch bool // true only if client set ClientSupportsBatching
|
||||
sendCh chan *filer_pb.SubscribeMetadataResponse
|
||||
errCh chan error
|
||||
done chan struct{}
|
||||
canBatch bool // true only if client set ClientSupportsBatching
|
||||
}
|
||||
|
||||
func newPipelinedSender(stream metadataStreamSender, bufSize int, clientSupportsBatching bool) *pipelinedSender {
|
||||
@@ -529,31 +529,19 @@ func (fs *FilerServer) eachEventNotificationFn(req *filer_pb.SubscribeMetadataRe
|
||||
return nil
|
||||
}
|
||||
|
||||
if hasPrefixIn(fullpath, req.PathPrefixes) {
|
||||
// good
|
||||
} else if matchByDirectory(dirPath, req.Directories) {
|
||||
// good
|
||||
} else {
|
||||
if !strings.HasPrefix(fullpath, req.PathPrefix) {
|
||||
if eventNotification.NewParentPath != "" {
|
||||
newFullPath := util.Join(eventNotification.NewParentPath, entryName)
|
||||
if !strings.HasPrefix(newFullPath, req.PathPrefix) {
|
||||
return nil
|
||||
}
|
||||
} else {
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// collect timestamps for path
|
||||
stats.FilerServerLastSendTsOfSubscribeGauge.WithLabelValues(fs.option.Host.String(), req.ClientName, req.PathPrefix).Set(float64(tsNs))
|
||||
|
||||
message := &filer_pb.SubscribeMetadataResponse{
|
||||
Directory: dirPath,
|
||||
EventNotification: eventNotification,
|
||||
TsNs: tsNs,
|
||||
}
|
||||
|
||||
if !filer_pb.MetadataEventMatchesSubscription(message, req.PathPrefix, req.PathPrefixes, req.Directories) {
|
||||
return nil
|
||||
}
|
||||
|
||||
// collect timestamps for path
|
||||
stats.FilerServerLastSendTsOfSubscribeGauge.WithLabelValues(fs.option.Host.String(), req.ClientName, req.PathPrefix).Set(float64(tsNs))
|
||||
|
||||
// println("sending", dirPath, entryName)
|
||||
if err := sender.Send(message); err != nil {
|
||||
glog.V(0).Infof("=> client %v: %+v", clientName, err)
|
||||
@@ -564,24 +552,6 @@ func (fs *FilerServer) eachEventNotificationFn(req *filer_pb.SubscribeMetadataRe
|
||||
}
|
||||
}
|
||||
|
||||
func hasPrefixIn(text string, prefixes []string) bool {
|
||||
for _, p := range prefixes {
|
||||
if strings.HasPrefix(text, p) {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func matchByDirectory(dirPath string, directories []string) bool {
|
||||
for _, dir := range directories {
|
||||
if dirPath == dir {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (fs *FilerServer) addClient(prefix string, clientType string, clientAddress string, clientId int32, clientEpoch int32) (isReplacing, alreadyKnown bool, clientName string) {
|
||||
clientName = clientType + "@" + clientAddress
|
||||
glog.V(0).Infof("+ %v listener %v clientId %v clientEpoch %v", prefix, clientName, clientId, clientEpoch)
|
||||
|
||||
@@ -7,6 +7,8 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/seaweedfs/seaweedfs/weed/filer"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
||||
)
|
||||
|
||||
@@ -25,6 +27,15 @@ func (s *slowStream) Send(msg *filer_pb.SubscribeMetadataResponse) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
type collectingStream struct {
|
||||
messages []*filer_pb.SubscribeMetadataResponse
|
||||
}
|
||||
|
||||
func (s *collectingStream) Send(msg *filer_pb.SubscribeMetadataResponse) error {
|
||||
s.messages = append(s.messages, msg)
|
||||
return nil
|
||||
}
|
||||
|
||||
func makeEvent(dir, name string, tsNs int64) *filer_pb.SubscribeMetadataResponse {
|
||||
return &filer_pb.SubscribeMetadataResponse{
|
||||
Directory: dir,
|
||||
@@ -69,11 +80,11 @@ func makeRecentEvents(n int) []*filer_pb.SubscribeMetadataResponse {
|
||||
// - Pipelined+batched: file I/O overlaps with batched sending
|
||||
func TestPipelinedSenderThroughput(t *testing.T) {
|
||||
const (
|
||||
eventsPerFile = 300 // events in one minute-log file
|
||||
numFiles = 7 // files to process
|
||||
totalEvents = eventsPerFile * numFiles // 2100
|
||||
fileReadDelay = 5 * time.Millisecond // volume server read per log file
|
||||
sendDelay = 50 * time.Microsecond // gRPC round-trip per Send()
|
||||
eventsPerFile = 300 // events in one minute-log file
|
||||
numFiles = 7 // files to process
|
||||
totalEvents = eventsPerFile * numFiles // 2100
|
||||
fileReadDelay = 5 * time.Millisecond // volume server read per log file
|
||||
sendDelay = 50 * time.Microsecond // gRPC round-trip per Send()
|
||||
)
|
||||
|
||||
// Partition old events into file-sized bursts
|
||||
@@ -138,6 +149,58 @@ func TestPipelinedSenderThroughput(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestEachEventNotificationFnMatchesRenameTargetsForAllWatchTypes(t *testing.T) {
|
||||
fs := &FilerServer{
|
||||
option: &FilerOption{Host: pb.ServerAddress("127.0.0.1:8888")},
|
||||
filer: &filer.Filer{Signature: 123},
|
||||
}
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
req *filer_pb.SubscribeMetadataRequest
|
||||
}{
|
||||
{
|
||||
name: "additional path prefix",
|
||||
req: &filer_pb.SubscribeMetadataRequest{
|
||||
ClientName: "test",
|
||||
PathPrefix: "/data/",
|
||||
PathPrefixes: []string{"/etc/remote"},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "directory watch",
|
||||
req: &filer_pb.SubscribeMetadataRequest{
|
||||
ClientName: "test",
|
||||
PathPrefix: "/data/",
|
||||
Directories: []string{"/etc/iam/identities"},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
stream := &collectingStream{}
|
||||
eachEventFn := fs.eachEventNotificationFn(tt.req, stream, "client")
|
||||
|
||||
newDir := "/etc/remote"
|
||||
if len(tt.req.Directories) > 0 {
|
||||
newDir = tt.req.Directories[0]
|
||||
}
|
||||
err := eachEventFn("/tmp", &filer_pb.EventNotification{
|
||||
OldEntry: &filer_pb.Entry{Name: "old"},
|
||||
NewEntry: &filer_pb.Entry{Name: "new"},
|
||||
NewParentPath: newDir,
|
||||
}, time.Now().UnixNano())
|
||||
if err != nil {
|
||||
t.Fatalf("eachEventFn: %v", err)
|
||||
}
|
||||
if len(stream.messages) != 1 {
|
||||
t.Fatalf("messages sent = %d, want 1", len(stream.messages))
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// TestBatchingAdaptive verifies the adaptive behavior: old events are batched,
|
||||
// recent events are sent one-by-one.
|
||||
func TestBatchingAdaptive(t *testing.T) {
|
||||
@@ -243,8 +306,8 @@ func TestPipelinedSenderErrorPropagation(t *testing.T) {
|
||||
func TestPipelinedSingleVsParallelStreams(t *testing.T) {
|
||||
const (
|
||||
numDirs = 10
|
||||
filesPerDir = 7 // log files per directory
|
||||
eventsPerFile = 300 // events per log file
|
||||
filesPerDir = 7 // log files per directory
|
||||
eventsPerFile = 300 // events per log file
|
||||
totalEvents = numDirs * filesPerDir * eventsPerFile // 21000
|
||||
fileReadDelay = 5 * time.Millisecond
|
||||
sendDelay = 50 * time.Microsecond
|
||||
|
||||
Reference in New Issue
Block a user