diff options
Diffstat (limited to 'weed/filer')
23 files changed, 82 insertions, 92 deletions
diff --git a/weed/filer/abstract_sql/abstract_sql_store.go b/weed/filer/abstract_sql/abstract_sql_store.go index 26dff7fe7..4bf9b16fa 100644 --- a/weed/filer/abstract_sql/abstract_sql_store.go +++ b/weed/filer/abstract_sql/abstract_sql_store.go @@ -10,7 +10,6 @@ import ( "github.com/chrislusf/seaweedfs/weed/util" "strings" "sync" - "time" ) type SqlGenerator interface { @@ -262,7 +261,7 @@ func (store *AbstractSqlStore) DeleteEntry(ctx context.Context, fullpath util.Fu return nil } -func (store *AbstractSqlStore) DeleteFolderChildren(ctx context.Context, fullpath util.FullPath, limit int64) error { +func (store *AbstractSqlStore) DeleteFolderChildren(ctx context.Context, fullpath util.FullPath) error { db, bucket, shortPath, err := store.getTxOrDB(ctx, fullpath, true) if err != nil { @@ -280,22 +279,15 @@ func (store *AbstractSqlStore) DeleteFolderChildren(ctx context.Context, fullpat } } - for { - glog.V(4).Infof("delete %s SQL %s %d", string(shortPath), store.GetSqlDeleteFolderChildren(bucket), util.HashStringToLong(string(shortPath))) - res, err := db.ExecContext(ctx, store.GetSqlDeleteFolderChildren(bucket), util.HashStringToLong(string(shortPath)), string(shortPath), limit) - if err != nil { - return fmt.Errorf("deleteFolderChildren %s: %s", fullpath, err) - } + glog.V(4).Infof("delete %s SQL %s %d", string(shortPath), store.GetSqlDeleteFolderChildren(bucket), util.HashStringToLong(string(shortPath))) + res, err := db.ExecContext(ctx, store.GetSqlDeleteFolderChildren(bucket), util.HashStringToLong(string(shortPath)), string(shortPath)) + if err != nil { + return fmt.Errorf("deleteFolderChildren %s: %s", fullpath, err) + } - rowCount, err := res.RowsAffected() - if err != nil { - return fmt.Errorf("deleteFolderChildren %s but no rows affected: %s", fullpath, err) - } - if rowCount < limit { - break - } - // to give the Galera Cluster a chance to breath - time.Sleep(time.Second) + _, err = res.RowsAffected() + if err != nil { + return fmt.Errorf("deleteFolderChildren %s but no rows affected: %s", fullpath, err) } return nil } diff --git a/weed/filer/cassandra/cassandra_store.go b/weed/filer/cassandra/cassandra_store.go index f4856657e..fc0b52ac7 100644 --- a/weed/filer/cassandra/cassandra_store.go +++ b/weed/filer/cassandra/cassandra_store.go @@ -124,7 +124,7 @@ func (store *CassandraStore) FindEntry(ctx context.Context, fullpath util.FullPa var data []byte if err := store.session.Query( "SELECT meta FROM filemeta WHERE directory=? AND name=?", - dir, name).Consistency(gocql.One).Scan(&data); err != nil { + dir, name).Scan(&data); err != nil { if err != gocql.ErrNotFound { return nil, filer_pb.ErrNotFound } @@ -161,7 +161,7 @@ func (store *CassandraStore) DeleteEntry(ctx context.Context, fullpath util.Full return nil } -func (store *CassandraStore) DeleteFolderChildren(ctx context.Context, fullpath util.FullPath, limit int64) error { +func (store *CassandraStore) DeleteFolderChildren(ctx context.Context, fullpath util.FullPath) error { if _, ok := store.isSuperLargeDirectory(string(fullpath)); ok { return nil // filer.ErrUnsupportedSuperLargeDirectoryListing } diff --git a/weed/filer/elastic/v7/elastic_store.go b/weed/filer/elastic/v7/elastic_store.go index 986c55b38..a16e5ebca 100644 --- a/weed/filer/elastic/v7/elastic_store.go +++ b/weed/filer/elastic/v7/elastic_store.go @@ -186,7 +186,7 @@ func (store *ElasticStore) deleteEntry(ctx context.Context, index, id string) (e return fmt.Errorf("delete entry %v.", err) } -func (store *ElasticStore) DeleteFolderChildren(ctx context.Context, fullpath weed_util.FullPath, limit int64) (err error) { +func (store *ElasticStore) DeleteFolderChildren(ctx context.Context, fullpath weed_util.FullPath) (err error) { _, err = store.ListDirectoryEntries(ctx, fullpath, "", false, math.MaxInt32, func(entry *filer.Entry) bool { if err := store.DeleteEntry(ctx, entry.FullPath); err != nil { glog.Errorf("elastic delete %s: %v.", entry.FullPath, err) diff --git a/weed/filer/entry.go b/weed/filer/entry.go index b7c8370e6..ede58a384 100644 --- a/weed/filer/entry.go +++ b/weed/filer/entry.go @@ -42,6 +42,7 @@ type Entry struct { HardLinkId HardLinkId HardLinkCounter int32 Content []byte + Remote *filer_pb.Entry_Remote } func (entry *Entry) Size() uint64 { @@ -60,16 +61,34 @@ func (entry *Entry) ToProtoEntry() *filer_pb.Entry { if entry == nil { return nil } - return &filer_pb.Entry{ - Name: entry.FullPath.Name(), - IsDirectory: entry.IsDirectory(), - Attributes: EntryAttributeToPb(entry), - Chunks: entry.Chunks, - Extended: entry.Extended, - HardLinkId: entry.HardLinkId, - HardLinkCounter: entry.HardLinkCounter, - Content: entry.Content, + message := &filer_pb.Entry{} + message.Name = entry.FullPath.Name() + entry.ToExistingProtoEntry(message) + return message +} + +func (entry *Entry) ToExistingProtoEntry(message *filer_pb.Entry) { + if entry == nil { + return } + message.IsDirectory = entry.IsDirectory() + message.Attributes = EntryAttributeToPb(entry) + message.Chunks = entry.Chunks + message.Extended = entry.Extended + message.HardLinkId = entry.HardLinkId + message.HardLinkCounter = entry.HardLinkCounter + message.Content = entry.Content + message.Remote = entry.Remote +} + +func FromPbEntryToExistingEntry(message *filer_pb.Entry, fsEntry *Entry) { + fsEntry.Attr = PbToEntryAttribute(message.Attributes) + fsEntry.Chunks = message.Chunks + fsEntry.Extended = message.Extended + fsEntry.HardLinkId = HardLinkId(message.HardLinkId) + fsEntry.HardLinkCounter = message.HardLinkCounter + fsEntry.Content = message.Content + fsEntry.Remote = message.Remote } func (entry *Entry) ToProtoFullEntry() *filer_pb.FullEntry { @@ -83,26 +102,11 @@ func (entry *Entry) ToProtoFullEntry() *filer_pb.FullEntry { } } -func (entry *Entry) Clone() *Entry { - return &Entry{ - FullPath: entry.FullPath, - Attr: entry.Attr, - Chunks: entry.Chunks, - Extended: entry.Extended, - HardLinkId: entry.HardLinkId, - HardLinkCounter: entry.HardLinkCounter, - } -} - func FromPbEntry(dir string, entry *filer_pb.Entry) *Entry { - return &Entry{ - FullPath: util.NewFullPath(dir, entry.Name), - Attr: PbToEntryAttribute(entry.Attributes), - Chunks: entry.Chunks, - HardLinkId: HardLinkId(entry.HardLinkId), - HardLinkCounter: entry.HardLinkCounter, - Content: entry.Content, - } + t := &Entry{} + t.FullPath = util.NewFullPath(dir, entry.Name) + FromPbEntryToExistingEntry(entry, t) + return t } func maxUint64(x, y uint64) uint64 { diff --git a/weed/filer/entry_codec.go b/weed/filer/entry_codec.go index 4c613f068..55c937b39 100644 --- a/weed/filer/entry_codec.go +++ b/weed/filer/entry_codec.go @@ -12,14 +12,8 @@ import ( ) func (entry *Entry) EncodeAttributesAndChunks() ([]byte, error) { - message := &filer_pb.Entry{ - Attributes: EntryAttributeToPb(entry), - Chunks: entry.Chunks, - Extended: entry.Extended, - HardLinkId: entry.HardLinkId, - HardLinkCounter: entry.HardLinkCounter, - Content: entry.Content, - } + message := &filer_pb.Entry{} + entry.ToExistingProtoEntry(message) return proto.Marshal(message) } @@ -31,15 +25,7 @@ func (entry *Entry) DecodeAttributesAndChunks(blob []byte) error { return fmt.Errorf("decoding value blob for %s: %v", entry.FullPath, err) } - entry.Attr = PbToEntryAttribute(message.Attributes) - - entry.Extended = message.Extended - - entry.Chunks = message.Chunks - - entry.HardLinkId = message.HardLinkId - entry.HardLinkCounter = message.HardLinkCounter - entry.Content = message.Content + FromPbEntryToExistingEntry(message, entry) return nil } @@ -129,6 +115,9 @@ func EqualEntry(a, b *Entry) bool { if !bytes.Equal(a.Content, b.Content) { return false } + if !proto.Equal(a.Remote, b.Remote) { + return false + } return true } diff --git a/weed/filer/etcd/etcd_store.go b/weed/filer/etcd/etcd_store.go index 96322081a..71ed738f9 100644 --- a/weed/filer/etcd/etcd_store.go +++ b/weed/filer/etcd/etcd_store.go @@ -130,7 +130,7 @@ func (store *EtcdStore) DeleteEntry(ctx context.Context, fullpath weed_util.Full return nil } -func (store *EtcdStore) DeleteFolderChildren(ctx context.Context, fullpath weed_util.FullPath, limit int64) (err error) { +func (store *EtcdStore) DeleteFolderChildren(ctx context.Context, fullpath weed_util.FullPath) (err error) { directoryPrefix := genDirectoryKeyPrefix(fullpath, "") if _, err := store.client.Delete(ctx, string(directoryPrefix), clientv3.WithPrefix()); err != nil { diff --git a/weed/filer/filechunk_manifest.go b/weed/filer/filechunk_manifest.go index c709dc819..0c6b0cfe3 100644 --- a/weed/filer/filechunk_manifest.go +++ b/weed/filer/filechunk_manifest.go @@ -16,7 +16,7 @@ import ( ) const ( - ManifestBatch = 1000 + ManifestBatch = 10000 ) func HasChunkManifest(chunks []*filer_pb.FileChunk) bool { @@ -39,9 +39,14 @@ func SeparateManifestChunks(chunks []*filer_pb.FileChunk) (manifestChunks, nonMa return } -func ResolveChunkManifest(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks []*filer_pb.FileChunk) (dataChunks, manifestChunks []*filer_pb.FileChunk, manifestResolveErr error) { +func ResolveChunkManifest(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks []*filer_pb.FileChunk, startOffset, stopOffset int64) (dataChunks, manifestChunks []*filer_pb.FileChunk, manifestResolveErr error) { // TODO maybe parallel this for _, chunk := range chunks { + + if max(chunk.Offset, startOffset) >= min(chunk.Offset+int64(chunk.Size), stopOffset) { + continue + } + if !chunk.IsChunkManifest { dataChunks = append(dataChunks, chunk) continue @@ -54,7 +59,7 @@ func ResolveChunkManifest(lookupFileIdFn wdclient.LookupFileIdFunctionType, chun manifestChunks = append(manifestChunks, chunk) // recursive - dchunks, mchunks, subErr := ResolveChunkManifest(lookupFileIdFn, resolvedChunks) + dchunks, mchunks, subErr := ResolveChunkManifest(lookupFileIdFn, resolvedChunks, startOffset, stopOffset) if subErr != nil { return chunks, nil, subErr } diff --git a/weed/filer/filechunks.go b/weed/filer/filechunks.go index 346eb3cfb..0dc03f6e2 100644 --- a/weed/filer/filechunks.go +++ b/weed/filer/filechunks.go @@ -53,7 +53,7 @@ func ETagChunks(chunks []*filer_pb.FileChunk) (etag string) { func CompactFileChunks(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks []*filer_pb.FileChunk) (compacted, garbage []*filer_pb.FileChunk) { - visibles, _ := NonOverlappingVisibleIntervals(lookupFileIdFn, chunks) + visibles, _ := NonOverlappingVisibleIntervals(lookupFileIdFn, chunks, 0, math.MaxInt64) fileIds := make(map[string]bool) for _, interval := range visibles { @@ -72,11 +72,11 @@ func CompactFileChunks(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks func MinusChunks(lookupFileIdFn wdclient.LookupFileIdFunctionType, as, bs []*filer_pb.FileChunk) (delta []*filer_pb.FileChunk, err error) { - aData, aMeta, aErr := ResolveChunkManifest(lookupFileIdFn, as) + aData, aMeta, aErr := ResolveChunkManifest(lookupFileIdFn, as, 0, math.MaxInt64) if aErr != nil { return nil, aErr } - bData, bMeta, bErr := ResolveChunkManifest(lookupFileIdFn, bs) + bData, bMeta, bErr := ResolveChunkManifest(lookupFileIdFn, bs, 0, math.MaxInt64) if bErr != nil { return nil, bErr } @@ -117,7 +117,7 @@ func (cv *ChunkView) IsFullChunk() bool { func ViewFromChunks(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks []*filer_pb.FileChunk, offset int64, size int64) (views []*ChunkView) { - visibles, _ := NonOverlappingVisibleIntervals(lookupFileIdFn, chunks) + visibles, _ := NonOverlappingVisibleIntervals(lookupFileIdFn, chunks, offset, offset+size) return ViewFromVisibleIntervals(visibles, offset, size) @@ -221,9 +221,9 @@ func MergeIntoVisibles(visibles []VisibleInterval, chunk *filer_pb.FileChunk) (n // NonOverlappingVisibleIntervals translates the file chunk into VisibleInterval in memory // If the file chunk content is a chunk manifest -func NonOverlappingVisibleIntervals(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks []*filer_pb.FileChunk) (visibles []VisibleInterval, err error) { +func NonOverlappingVisibleIntervals(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks []*filer_pb.FileChunk, startOffset int64, stopOffset int64) (visibles []VisibleInterval, err error) { - chunks, _, err = ResolveChunkManifest(lookupFileIdFn, chunks) + chunks, _, err = ResolveChunkManifest(lookupFileIdFn, chunks, startOffset, stopOffset) sort.Slice(chunks, func(i, j int) bool { if chunks[i].Mtime == chunks[j].Mtime { diff --git a/weed/filer/filechunks_test.go b/weed/filer/filechunks_test.go index 699e7e298..b0ea20848 100644 --- a/weed/filer/filechunks_test.go +++ b/weed/filer/filechunks_test.go @@ -90,7 +90,7 @@ func TestRandomFileChunksCompact(t *testing.T) { } } - visibles, _ := NonOverlappingVisibleIntervals(nil, chunks) + visibles, _ := NonOverlappingVisibleIntervals(nil, chunks, 0, math.MaxInt64) for _, v := range visibles { for x := v.start; x < v.stop; x++ { @@ -227,7 +227,7 @@ func TestIntervalMerging(t *testing.T) { for i, testcase := range testcases { log.Printf("++++++++++ merged test case %d ++++++++++++++++++++", i) - intervals, _ := NonOverlappingVisibleIntervals(nil, testcase.Chunks) + intervals, _ := NonOverlappingVisibleIntervals(nil, testcase.Chunks, 0, math.MaxInt64) for x, interval := range intervals { log.Printf("test case %d, interval %d, start=%d, stop=%d, fileId=%s", i, x, interval.start, interval.stop, interval.fileId) diff --git a/weed/filer/filer.go b/weed/filer/filer.go index 3082d0f55..d4c0b4eef 100644 --- a/weed/filer/filer.go +++ b/weed/filer/filer.go @@ -19,7 +19,6 @@ import ( const ( LogFlushInterval = time.Minute PaginationSize = 1024 - DeleteMaxRows = 10000 FilerStoreId = "filer.store.id" ) diff --git a/weed/filer/filer_conf.go b/weed/filer/filer_conf.go index da02c5b20..c58b26dc2 100644 --- a/weed/filer/filer_conf.go +++ b/weed/filer/filer_conf.go @@ -15,6 +15,7 @@ import ( const ( DirectoryEtcRoot = "/etc" DirectoryEtcSeaweedFS = "/etc/seaweedfs" + DirectoryEtcRemote = "/etc/remote" FilerConfName = "filer.conf" IamConfigDirecotry = "/etc/iam" IamIdentityFile = "identity.json" diff --git a/weed/filer/filer_delete_entry.go b/weed/filer/filer_delete_entry.go index be21801dc..35187d034 100644 --- a/weed/filer/filer_delete_entry.go +++ b/weed/filer/filer_delete_entry.go @@ -115,7 +115,7 @@ func (f *Filer) doBatchDeleteFolderMetaAndData(ctx context.Context, entry *Entry glog.V(3).Infof("deleting directory %v delete %d chunks: %v", entry.FullPath, len(chunks), shouldDeleteChunks) - if storeDeletionErr := f.Store.DeleteFolderChildren(ctx, entry.FullPath, DeleteMaxRows); storeDeletionErr != nil { + if storeDeletionErr := f.Store.DeleteFolderChildren(ctx, entry.FullPath); storeDeletionErr != nil { return nil, nil, fmt.Errorf("filer store delete: %v", storeDeletionErr) } diff --git a/weed/filer/filerstore.go b/weed/filer/filerstore.go index 63e2e7817..38927d6fb 100644 --- a/weed/filer/filerstore.go +++ b/weed/filer/filerstore.go @@ -25,7 +25,7 @@ type FilerStore interface { // err == filer_pb.ErrNotFound if not found FindEntry(context.Context, util.FullPath) (entry *Entry, err error) DeleteEntry(context.Context, util.FullPath) (err error) - DeleteFolderChildren(context.Context, util.FullPath, int64) (err error) + DeleteFolderChildren(context.Context, util.FullPath) (err error) ListDirectoryEntries(ctx context.Context, dirPath util.FullPath, startFileName string, includeStartFile bool, limit int64, eachEntryFunc ListEachEntryFunc) (lastFileName string, err error) ListDirectoryPrefixedEntries(ctx context.Context, dirPath util.FullPath, startFileName string, includeStartFile bool, limit int64, prefix string, eachEntryFunc ListEachEntryFunc) (lastFileName string, err error) diff --git a/weed/filer/filerstore_translate_path.go b/weed/filer/filerstore_translate_path.go index cb9fabfc0..00bf82ed4 100644 --- a/weed/filer/filerstore_translate_path.go +++ b/weed/filer/filerstore_translate_path.go @@ -100,10 +100,10 @@ func (t *FilerStorePathTranlator) DeleteOneEntry(ctx context.Context, existingEn return t.actualStore.DeleteEntry(ctx, existingEntry.FullPath) } -func (t *FilerStorePathTranlator) DeleteFolderChildren(ctx context.Context, fp util.FullPath, limit int64) (err error) { +func (t *FilerStorePathTranlator) DeleteFolderChildren(ctx context.Context, fp util.FullPath) (err error) { newFullPath := t.translatePath(fp) - return t.actualStore.DeleteFolderChildren(ctx, newFullPath, limit) + return t.actualStore.DeleteFolderChildren(ctx, newFullPath) } func (t *FilerStorePathTranlator) ListDirectoryEntries(ctx context.Context, dirPath util.FullPath, startFileName string, includeStartFile bool, limit int64, eachEntryFunc ListEachEntryFunc) (string, error) { diff --git a/weed/filer/filerstore_wrapper.go b/weed/filer/filerstore_wrapper.go index 997d70a80..2470f340c 100644 --- a/weed/filer/filerstore_wrapper.go +++ b/weed/filer/filerstore_wrapper.go @@ -213,7 +213,7 @@ func (fsw *FilerStoreWrapper) DeleteOneEntry(ctx context.Context, existingEntry return actualStore.DeleteEntry(ctx, existingEntry.FullPath) } -func (fsw *FilerStoreWrapper) DeleteFolderChildren(ctx context.Context, fp util.FullPath, limit int64) (err error) { +func (fsw *FilerStoreWrapper) DeleteFolderChildren(ctx context.Context, fp util.FullPath) (err error) { actualStore := fsw.getActualStore(fp + "/") stats.FilerStoreCounter.WithLabelValues(actualStore.GetName(), "deleteFolderChildren").Inc() start := time.Now() @@ -222,7 +222,7 @@ func (fsw *FilerStoreWrapper) DeleteFolderChildren(ctx context.Context, fp util. }() glog.V(4).Infof("DeleteFolderChildren %s", fp) - return actualStore.DeleteFolderChildren(ctx, fp, limit) + return actualStore.DeleteFolderChildren(ctx, fp) } func (fsw *FilerStoreWrapper) ListDirectoryEntries(ctx context.Context, dirPath util.FullPath, startFileName string, includeStartFile bool, limit int64, eachEntryFunc ListEachEntryFunc) (string, error) { diff --git a/weed/filer/hbase/hbase_store.go b/weed/filer/hbase/hbase_store.go index 43c14cc15..e0d878ca7 100644 --- a/weed/filer/hbase/hbase_store.go +++ b/weed/filer/hbase/hbase_store.go @@ -109,7 +109,7 @@ func (store *HbaseStore) DeleteEntry(ctx context.Context, path util.FullPath) (e return store.doDelete(ctx, store.cfMetaDir, []byte(path)) } -func (store *HbaseStore) DeleteFolderChildren(ctx context.Context, path util.FullPath, limit int64) (err error) { +func (store *HbaseStore) DeleteFolderChildren(ctx context.Context, path util.FullPath) (err error) { family := map[string][]string{store.cfMetaDir: {COLUMN_NAME}} expectedPrefix := []byte(path.Child("")) diff --git a/weed/filer/leveldb/leveldb_store.go b/weed/filer/leveldb/leveldb_store.go index 50367c87b..ce454f36a 100644 --- a/weed/filer/leveldb/leveldb_store.go +++ b/weed/filer/leveldb/leveldb_store.go @@ -136,7 +136,7 @@ func (store *LevelDBStore) DeleteEntry(ctx context.Context, fullpath weed_util.F return nil } -func (store *LevelDBStore) DeleteFolderChildren(ctx context.Context, fullpath weed_util.FullPath, limit int64) (err error) { +func (store *LevelDBStore) DeleteFolderChildren(ctx context.Context, fullpath weed_util.FullPath) (err error) { batch := new(leveldb.Batch) diff --git a/weed/filer/leveldb2/leveldb2_store.go b/weed/filer/leveldb2/leveldb2_store.go index bf81bface..4c4409c4d 100644 --- a/weed/filer/leveldb2/leveldb2_store.go +++ b/weed/filer/leveldb2/leveldb2_store.go @@ -147,7 +147,7 @@ func (store *LevelDB2Store) DeleteEntry(ctx context.Context, fullpath weed_util. return nil } -func (store *LevelDB2Store) DeleteFolderChildren(ctx context.Context, fullpath weed_util.FullPath, limit int64) (err error) { +func (store *LevelDB2Store) DeleteFolderChildren(ctx context.Context, fullpath weed_util.FullPath) (err error) { directoryPrefix, partitionId := genDirectoryKeyPrefix(fullpath, "", store.dbCount) batch := new(leveldb.Batch) diff --git a/weed/filer/leveldb3/leveldb3_store.go b/weed/filer/leveldb3/leveldb3_store.go index 70e5088a2..bc57a6605 100644 --- a/weed/filer/leveldb3/leveldb3_store.go +++ b/weed/filer/leveldb3/leveldb3_store.go @@ -249,7 +249,7 @@ func (store *LevelDB3Store) DeleteEntry(ctx context.Context, fullpath weed_util. return nil } -func (store *LevelDB3Store) DeleteFolderChildren(ctx context.Context, fullpath weed_util.FullPath, limit int64) (err error) { +func (store *LevelDB3Store) DeleteFolderChildren(ctx context.Context, fullpath weed_util.FullPath) (err error) { db, bucket, shortPath, err := store.findDB(fullpath, true) if err != nil { diff --git a/weed/filer/mongodb/mongodb_store.go b/weed/filer/mongodb/mongodb_store.go index 5861d86b0..1ef5056f4 100644 --- a/weed/filer/mongodb/mongodb_store.go +++ b/weed/filer/mongodb/mongodb_store.go @@ -167,7 +167,7 @@ func (store *MongodbStore) DeleteEntry(ctx context.Context, fullpath util.FullPa return nil } -func (store *MongodbStore) DeleteFolderChildren(ctx context.Context, fullpath util.FullPath, limit int64) error { +func (store *MongodbStore) DeleteFolderChildren(ctx context.Context, fullpath util.FullPath) error { where := bson.M{"directory": fullpath} _, err := store.connect.Database(store.database).Collection(store.collectionName).DeleteMany(ctx, where) diff --git a/weed/filer/redis/universal_redis_store.go b/weed/filer/redis/universal_redis_store.go index fb49740cd..30d11a7f4 100644 --- a/weed/filer/redis/universal_redis_store.go +++ b/weed/filer/redis/universal_redis_store.go @@ -107,7 +107,7 @@ func (store *UniversalRedisStore) DeleteEntry(ctx context.Context, fullpath util return nil } -func (store *UniversalRedisStore) DeleteFolderChildren(ctx context.Context, fullpath util.FullPath, limit int64) (err error) { +func (store *UniversalRedisStore) DeleteFolderChildren(ctx context.Context, fullpath util.FullPath) (err error) { members, err := store.Client.SMembers(ctx, genDirectoryListKey(string(fullpath))).Result() if err != nil { diff --git a/weed/filer/redis2/universal_redis_store.go b/weed/filer/redis2/universal_redis_store.go index 6bb56f5f8..aab3d1f4a 100644 --- a/weed/filer/redis2/universal_redis_store.go +++ b/weed/filer/redis2/universal_redis_store.go @@ -127,7 +127,7 @@ func (store *UniversalRedis2Store) DeleteEntry(ctx context.Context, fullpath uti return nil } -func (store *UniversalRedis2Store) DeleteFolderChildren(ctx context.Context, fullpath util.FullPath, limit int64) (err error) { +func (store *UniversalRedis2Store) DeleteFolderChildren(ctx context.Context, fullpath util.FullPath) (err error) { if store.isSuperLargeDirectory(string(fullpath)) { return nil diff --git a/weed/filer/rocksdb/rocksdb_store.go b/weed/filer/rocksdb/rocksdb_store.go index 467b07aaa..729da7d9b 100644 --- a/weed/filer/rocksdb/rocksdb_store.go +++ b/weed/filer/rocksdb/rocksdb_store.go @@ -156,7 +156,7 @@ func (store *RocksDBStore) DeleteEntry(ctx context.Context, fullpath weed_util.F return nil } -func (store *RocksDBStore) DeleteFolderChildren(ctx context.Context, fullpath weed_util.FullPath, limit int64) (err error) { +func (store *RocksDBStore) DeleteFolderChildren(ctx context.Context, fullpath weed_util.FullPath) (err error) { directoryPrefix := genDirectoryKeyPrefix(fullpath, "") batch := gorocksdb.NewWriteBatch() |
