Yoni Nakache
4 years ago
committed by
GitHub
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
55 changed files with 1177 additions and 323 deletions
-
4README.md
-
2docker/local-dev-compose.yml
-
10go.mod
-
54go.sum
-
3weed/command/command.go
-
6weed/command/filer.go
-
118weed/command/filer_cat.go
-
2weed/command/filer_copy.go
-
201weed/command/filer_meta_tail.go
-
3weed/command/filer_replication.go
-
2weed/command/mount.go
-
10weed/command/scaffold.go
-
9weed/command/server.go
-
113weed/command/watch.go
-
6weed/filer/configuration.go
-
2weed/filer/etcd/etcd_store.go
-
7weed/filer/filechunk_manifest.go
-
9weed/filer/filechunks.go
-
2weed/filer/filer_notify.go
-
3weed/filer/hbase/hbase_store_kv.go
-
2weed/filer/leveldb/leveldb_store.go
-
2weed/filer/leveldb2/leveldb2_store.go
-
361weed/filer/leveldb3/leveldb3_store.go
-
46weed/filer/leveldb3/leveldb3_store_kv.go
-
88weed/filer/leveldb3/leveldb3_store_test.go
-
6weed/filer/reader_at.go
-
6weed/filer/redis/redis_cluster_store.go
-
2weed/filer/redis/redis_store.go
-
22weed/filer/redis/universal_redis_store.go
-
8weed/filer/redis/universal_redis_store_kv.go
-
6weed/filer/redis2/redis_cluster_store.go
-
2weed/filer/redis2/redis_store.go
-
26weed/filer/redis2/universal_redis_store.go
-
8weed/filer/redis2/universal_redis_store_kv.go
-
99weed/filer/rocksdb/rocksdb_store.go
-
10weed/filer/rocksdb/rocksdb_store_kv.go
-
40weed/filer/rocksdb/rocksdb_ttl.go
-
8weed/filer/stream.go
-
26weed/messaging/broker/broker_grpc_server_subscribe.go
-
5weed/notification/configuration.go
-
3weed/replication/sink/filersink/filer_sink.go
-
2weed/s3api/s3api_bucket_handlers.go
-
2weed/s3api/s3api_objects_list_handlers.go
-
7weed/security/tls.go
-
52weed/server/filer_grpc_server_sub_meta.go
-
3weed/server/filer_server.go
-
10weed/server/filer_server_handlers_read.go
-
14weed/server/filer_server_handlers_write_autochunk.go
-
4weed/storage/backend/backend.go
-
2weed/util/chunk_cache/chunk_cache_in_memory.go
-
18weed/util/config.go
-
2weed/util/constants.go
-
24weed/util/log_buffer/log_buffer.go
-
8weed/util/log_buffer/log_read.go
-
10weed/wdclient/vid_map.go
@ -0,0 +1,118 @@ |
|||
package command |
|||
|
|||
import ( |
|||
"context" |
|||
"fmt" |
|||
"github.com/chrislusf/seaweedfs/weed/filer" |
|||
"github.com/chrislusf/seaweedfs/weed/pb" |
|||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb" |
|||
"github.com/chrislusf/seaweedfs/weed/wdclient" |
|||
"google.golang.org/grpc" |
|||
"math" |
|||
"net/url" |
|||
"os" |
|||
"strings" |
|||
|
|||
"github.com/chrislusf/seaweedfs/weed/security" |
|||
"github.com/chrislusf/seaweedfs/weed/util" |
|||
) |
|||
|
|||
var ( |
|||
filerCat FilerCatOptions |
|||
) |
|||
|
|||
type FilerCatOptions struct { |
|||
grpcDialOption grpc.DialOption |
|||
filerAddress string |
|||
filerClient filer_pb.SeaweedFilerClient |
|||
output *string |
|||
} |
|||
|
|||
func (fco *FilerCatOptions) GetLookupFileIdFunction() wdclient.LookupFileIdFunctionType { |
|||
return func(fileId string) (targetUrls []string, err error) { |
|||
vid := filer.VolumeId(fileId) |
|||
resp, err := fco.filerClient.LookupVolume(context.Background(), &filer_pb.LookupVolumeRequest{ |
|||
VolumeIds: []string{vid}, |
|||
}) |
|||
if err != nil { |
|||
return nil, err |
|||
} |
|||
locations := resp.LocationsMap[vid] |
|||
for _, loc := range locations.Locations { |
|||
targetUrls = append(targetUrls, fmt.Sprintf("http://%s/%s", loc.Url, fileId)) |
|||
} |
|||
return |
|||
} |
|||
} |
|||
|
|||
func init() { |
|||
cmdFilerCat.Run = runFilerCat // break init cycle
|
|||
filerCat.output = cmdFilerCat.Flag.String("o", "", "write to file instead of stdout") |
|||
} |
|||
|
|||
var cmdFilerCat = &Command{ |
|||
UsageLine: "filer.cat [-o <file>] http://localhost:8888/path/to/file", |
|||
Short: "copy one file to local", |
|||
Long: `read one file to stdout or write to a file |
|||
|
|||
`, |
|||
} |
|||
|
|||
func runFilerCat(cmd *Command, args []string) bool { |
|||
|
|||
util.LoadConfiguration("security", false) |
|||
|
|||
if len(args) == 0 { |
|||
return false |
|||
} |
|||
filerSource := args[len(args)-1] |
|||
|
|||
filerUrl, err := url.Parse(filerSource) |
|||
if err != nil { |
|||
fmt.Printf("The last argument should be a URL on filer: %v\n", err) |
|||
return false |
|||
} |
|||
urlPath := filerUrl.Path |
|||
if strings.HasSuffix(urlPath, "/") { |
|||
fmt.Printf("The last argument should be a file: %v\n", err) |
|||
return false |
|||
} |
|||
|
|||
filerCat.filerAddress = filerUrl.Host |
|||
filerCat.grpcDialOption = security.LoadClientTLS(util.GetViper(), "grpc.client") |
|||
|
|||
dir, name := util.FullPath(urlPath).DirAndName() |
|||
|
|||
writer := os.Stdout |
|||
if *filerCat.output != "" { |
|||
|
|||
fmt.Printf("saving %s to %s\n", filerSource, *filerCat.output) |
|||
|
|||
f, err := os.OpenFile(*filerCat.output, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0755) |
|||
if err != nil { |
|||
fmt.Printf("open file %s: %v\n", *filerCat.output, err) |
|||
return false |
|||
} |
|||
defer f.Close() |
|||
writer = f |
|||
} |
|||
|
|||
pb.WithFilerClient(filerCat.filerAddress, filerCat.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { |
|||
|
|||
request := &filer_pb.LookupDirectoryEntryRequest{ |
|||
Name: name, |
|||
Directory: dir, |
|||
} |
|||
respLookupEntry, err := filer_pb.LookupEntry(client, request) |
|||
if err != nil { |
|||
return err |
|||
} |
|||
|
|||
filerCat.filerClient = client |
|||
|
|||
return filer.StreamContent(&filerCat, writer, respLookupEntry.Entry.Chunks, 0, math.MaxInt64) |
|||
|
|||
}) |
|||
|
|||
return true |
|||
} |
@ -0,0 +1,201 @@ |
|||
package command |
|||
|
|||
import ( |
|||
"context" |
|||
"fmt" |
|||
jsoniter "github.com/json-iterator/go" |
|||
"github.com/olivere/elastic/v7" |
|||
"io" |
|||
"path/filepath" |
|||
"strings" |
|||
"time" |
|||
|
|||
"github.com/chrislusf/seaweedfs/weed/pb" |
|||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb" |
|||
"github.com/chrislusf/seaweedfs/weed/security" |
|||
"github.com/chrislusf/seaweedfs/weed/util" |
|||
) |
|||
|
|||
func init() { |
|||
cmdFilerMetaTail.Run = runFilerMetaTail // break init cycle
|
|||
} |
|||
|
|||
var cmdFilerMetaTail = &Command{ |
|||
UsageLine: "filer.meta.tail [-filer=localhost:8888] [-target=/]", |
|||
Short: "see recent changes on a filer", |
|||
Long: `See recent changes on a filer. |
|||
|
|||
`, |
|||
} |
|||
|
|||
var ( |
|||
tailFiler = cmdFilerMetaTail.Flag.String("filer", "localhost:8888", "filer hostname:port") |
|||
tailTarget = cmdFilerMetaTail.Flag.String("pathPrefix", "/", "path to a folder or file, or common prefix for the folders or files on filer") |
|||
tailStart = cmdFilerMetaTail.Flag.Duration("timeAgo", 0, "start time before now. \"300ms\", \"1.5h\" or \"2h45m\". Valid time units are \"ns\", \"us\" (or \"µs\"), \"ms\", \"s\", \"m\", \"h\"") |
|||
tailPattern = cmdFilerMetaTail.Flag.String("pattern", "", "full path or just filename pattern, ex: \"/home/?opher\", \"*.pdf\", see https://golang.org/pkg/path/filepath/#Match ") |
|||
esServers = cmdFilerMetaTail.Flag.String("es", "", "comma-separated elastic servers http://<host:port>") |
|||
esIndex = cmdFilerMetaTail.Flag.String("es.index", "seaweedfs", "ES index name") |
|||
) |
|||
|
|||
func runFilerMetaTail(cmd *Command, args []string) bool { |
|||
|
|||
grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.client") |
|||
|
|||
var filterFunc func(dir, fname string) bool |
|||
if *tailPattern != "" { |
|||
if strings.Contains(*tailPattern, "/") { |
|||
println("watch path pattern", *tailPattern) |
|||
filterFunc = func(dir, fname string) bool { |
|||
matched, err := filepath.Match(*tailPattern, dir+"/"+fname) |
|||
if err != nil { |
|||
fmt.Printf("error: %v", err) |
|||
} |
|||
return matched |
|||
} |
|||
} else { |
|||
println("watch file pattern", *tailPattern) |
|||
filterFunc = func(dir, fname string) bool { |
|||
matched, err := filepath.Match(*tailPattern, fname) |
|||
if err != nil { |
|||
fmt.Printf("error: %v", err) |
|||
} |
|||
return matched |
|||
} |
|||
} |
|||
} |
|||
|
|||
shouldPrint := func(resp *filer_pb.SubscribeMetadataResponse) bool { |
|||
if filterFunc == nil { |
|||
return true |
|||
} |
|||
if resp.EventNotification.OldEntry == nil && resp.EventNotification.NewEntry == nil { |
|||
return false |
|||
} |
|||
if resp.EventNotification.OldEntry != nil && filterFunc(resp.Directory, resp.EventNotification.OldEntry.Name) { |
|||
return true |
|||
} |
|||
if resp.EventNotification.NewEntry != nil && filterFunc(resp.EventNotification.NewParentPath, resp.EventNotification.NewEntry.Name) { |
|||
return true |
|||
} |
|||
return false |
|||
} |
|||
|
|||
eachEntryFunc := func(resp *filer_pb.SubscribeMetadataResponse) error { |
|||
fmt.Printf("dir:%s %+v\n", resp.Directory, resp.EventNotification) |
|||
return nil |
|||
} |
|||
if *esServers != "" { |
|||
var err error |
|||
eachEntryFunc, err = sendToElasticSearchFunc(*esServers, *esIndex) |
|||
if err != nil { |
|||
fmt.Printf("create elastic search client to %s: %+v\n", *esServers, err) |
|||
return false |
|||
} |
|||
} |
|||
|
|||
tailErr := pb.WithFilerClient(*tailFiler, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { |
|||
|
|||
ctx, cancel := context.WithCancel(context.Background()) |
|||
defer cancel() |
|||
|
|||
stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{ |
|||
ClientName: "tail", |
|||
PathPrefix: *tailTarget, |
|||
SinceNs: time.Now().Add(-*tailStart).UnixNano(), |
|||
}) |
|||
if err != nil { |
|||
return fmt.Errorf("listen: %v", err) |
|||
} |
|||
|
|||
for { |
|||
resp, listenErr := stream.Recv() |
|||
if listenErr == io.EOF { |
|||
return nil |
|||
} |
|||
if listenErr != nil { |
|||
return listenErr |
|||
} |
|||
if !shouldPrint(resp) { |
|||
continue |
|||
} |
|||
if err = eachEntryFunc(resp); err != nil { |
|||
return err |
|||
} |
|||
} |
|||
|
|||
}) |
|||
if tailErr != nil { |
|||
fmt.Printf("tail %s: %v\n", *tailFiler, tailErr) |
|||
} |
|||
|
|||
return true |
|||
} |
|||
|
|||
type EsDocument struct { |
|||
Dir string `json:"dir,omitempty"` |
|||
Name string `json:"name,omitempty"` |
|||
IsDirectory bool `json:"isDir,omitempty"` |
|||
Size uint64 `json:"size,omitempty"` |
|||
Uid uint32 `json:"uid,omitempty"` |
|||
Gid uint32 `json:"gid,omitempty"` |
|||
UserName string `json:"userName,omitempty"` |
|||
Collection string `json:"collection,omitempty"` |
|||
Crtime int64 `json:"crtime,omitempty"` |
|||
Mtime int64 `json:"mtime,omitempty"` |
|||
Mime string `json:"mime,omitempty"` |
|||
} |
|||
|
|||
func toEsEntry(event *filer_pb.EventNotification) (*EsDocument, string) { |
|||
entry := event.NewEntry |
|||
dir, name := event.NewParentPath, entry.Name |
|||
id := util.Md5String([]byte(util.NewFullPath(dir, name))) |
|||
esEntry := &EsDocument{ |
|||
Dir: dir, |
|||
Name: name, |
|||
IsDirectory: entry.IsDirectory, |
|||
Size: entry.Attributes.FileSize, |
|||
Uid: entry.Attributes.Uid, |
|||
Gid: entry.Attributes.Gid, |
|||
UserName: entry.Attributes.UserName, |
|||
Collection: entry.Attributes.Collection, |
|||
Crtime: entry.Attributes.Crtime, |
|||
Mtime: entry.Attributes.Mtime, |
|||
Mime: entry.Attributes.Mime, |
|||
} |
|||
return esEntry, id |
|||
} |
|||
|
|||
func sendToElasticSearchFunc(servers string, esIndex string) (func(resp *filer_pb.SubscribeMetadataResponse) error, error) { |
|||
options := []elastic.ClientOptionFunc{} |
|||
options = append(options, elastic.SetURL(strings.Split(servers, ",")...)) |
|||
options = append(options, elastic.SetSniff(false)) |
|||
options = append(options, elastic.SetHealthcheck(false)) |
|||
client, err := elastic.NewClient(options...) |
|||
if err != nil { |
|||
return nil, err |
|||
} |
|||
return func(resp *filer_pb.SubscribeMetadataResponse) error { |
|||
event := resp.EventNotification |
|||
if event.OldEntry != nil && |
|||
(event.NewEntry == nil || resp.Directory != event.NewParentPath || event.OldEntry.Name != event.NewEntry.Name) { |
|||
// delete or not update the same file
|
|||
dir, name := resp.Directory, event.OldEntry.Name |
|||
id := util.Md5String([]byte(util.NewFullPath(dir, name))) |
|||
println("delete", id) |
|||
_, err := client.Delete().Index(esIndex).Id(id).Do(context.Background()) |
|||
return err |
|||
} |
|||
if event.NewEntry != nil { |
|||
// add a new file or update the same file
|
|||
esEntry, id := toEsEntry(event) |
|||
value, err := jsoniter.Marshal(esEntry) |
|||
if err != nil { |
|||
return err |
|||
} |
|||
println(string(value)) |
|||
_, err = client.Index().Index(esIndex).Id(id).BodyJson(string(value)).Do(context.Background()) |
|||
return err |
|||
} |
|||
return nil |
|||
}, nil |
|||
} |
@ -1,113 +0,0 @@ |
|||
package command |
|||
|
|||
import ( |
|||
"context" |
|||
"fmt" |
|||
"io" |
|||
"path/filepath" |
|||
"strings" |
|||
"time" |
|||
|
|||
"github.com/chrislusf/seaweedfs/weed/pb" |
|||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb" |
|||
"github.com/chrislusf/seaweedfs/weed/security" |
|||
"github.com/chrislusf/seaweedfs/weed/util" |
|||
) |
|||
|
|||
func init() { |
|||
cmdWatch.Run = runWatch // break init cycle
|
|||
} |
|||
|
|||
var cmdWatch = &Command{ |
|||
UsageLine: "watch [-filer=localhost:8888] [-target=/]", |
|||
Short: "see recent changes on a filer", |
|||
Long: `See recent changes on a filer. |
|||
|
|||
`, |
|||
} |
|||
|
|||
var ( |
|||
watchFiler = cmdWatch.Flag.String("filer", "localhost:8888", "filer hostname:port") |
|||
watchTarget = cmdWatch.Flag.String("pathPrefix", "/", "path to a folder or file, or common prefix for the folders or files on filer") |
|||
watchStart = cmdWatch.Flag.Duration("timeAgo", 0, "start time before now. \"300ms\", \"1.5h\" or \"2h45m\". Valid time units are \"ns\", \"us\" (or \"µs\"), \"ms\", \"s\", \"m\", \"h\"") |
|||
watchPattern = cmdWatch.Flag.String("pattern", "", "full path or just filename pattern, ex: \"/home/?opher\", \"*.pdf\", see https://golang.org/pkg/path/filepath/#Match ") |
|||
) |
|||
|
|||
func runWatch(cmd *Command, args []string) bool { |
|||
|
|||
grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.client") |
|||
|
|||
var filterFunc func(dir, fname string) bool |
|||
if *watchPattern != "" { |
|||
if strings.Contains(*watchPattern, "/") { |
|||
println("watch path pattern", *watchPattern) |
|||
filterFunc = func(dir, fname string) bool { |
|||
matched, err := filepath.Match(*watchPattern, dir+"/"+fname) |
|||
if err != nil { |
|||
fmt.Printf("error: %v", err) |
|||
} |
|||
return matched |
|||
} |
|||
} else { |
|||
println("watch file pattern", *watchPattern) |
|||
filterFunc = func(dir, fname string) bool { |
|||
matched, err := filepath.Match(*watchPattern, fname) |
|||
if err != nil { |
|||
fmt.Printf("error: %v", err) |
|||
} |
|||
return matched |
|||
} |
|||
} |
|||
} |
|||
|
|||
shouldPrint := func(resp *filer_pb.SubscribeMetadataResponse) bool { |
|||
if filterFunc == nil { |
|||
return true |
|||
} |
|||
if resp.EventNotification.OldEntry == nil && resp.EventNotification.NewEntry == nil { |
|||
return false |
|||
} |
|||
if resp.EventNotification.OldEntry != nil && filterFunc(resp.Directory, resp.EventNotification.OldEntry.Name) { |
|||
return true |
|||
} |
|||
if resp.EventNotification.NewEntry != nil && filterFunc(resp.EventNotification.NewParentPath, resp.EventNotification.NewEntry.Name) { |
|||
return true |
|||
} |
|||
return false |
|||
} |
|||
|
|||
watchErr := pb.WithFilerClient(*watchFiler, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { |
|||
|
|||
ctx, cancel := context.WithCancel(context.Background()) |
|||
defer cancel() |
|||
|
|||
stream, err := client.SubscribeMetadata(ctx, &filer_pb.SubscribeMetadataRequest{ |
|||
ClientName: "watch", |
|||
PathPrefix: *watchTarget, |
|||
SinceNs: time.Now().Add(-*watchStart).UnixNano(), |
|||
}) |
|||
if err != nil { |
|||
return fmt.Errorf("listen: %v", err) |
|||
} |
|||
|
|||
for { |
|||
resp, listenErr := stream.Recv() |
|||
if listenErr == io.EOF { |
|||
return nil |
|||
} |
|||
if listenErr != nil { |
|||
return listenErr |
|||
} |
|||
if !shouldPrint(resp) { |
|||
continue |
|||
} |
|||
fmt.Printf("dir:%s %+v\n", resp.Directory, resp.EventNotification) |
|||
} |
|||
|
|||
}) |
|||
if watchErr != nil { |
|||
fmt.Printf("watch %s: %v\n", *watchFiler, watchErr) |
|||
} |
|||
|
|||
return true |
|||
} |
@ -0,0 +1,361 @@ |
|||
package leveldb |
|||
|
|||
import ( |
|||
"bytes" |
|||
"context" |
|||
"crypto/md5" |
|||
"fmt" |
|||
"github.com/syndtr/goleveldb/leveldb" |
|||
leveldb_errors "github.com/syndtr/goleveldb/leveldb/errors" |
|||
"github.com/syndtr/goleveldb/leveldb/opt" |
|||
leveldb_util "github.com/syndtr/goleveldb/leveldb/util" |
|||
"io" |
|||
"os" |
|||
"strings" |
|||
"sync" |
|||
|
|||
"github.com/chrislusf/seaweedfs/weed/filer" |
|||
"github.com/chrislusf/seaweedfs/weed/glog" |
|||
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb" |
|||
weed_util "github.com/chrislusf/seaweedfs/weed/util" |
|||
) |
|||
|
|||
const ( |
|||
DEFAULT = "_main" |
|||
) |
|||
|
|||
func init() { |
|||
filer.Stores = append(filer.Stores, &LevelDB3Store{}) |
|||
} |
|||
|
|||
type LevelDB3Store struct { |
|||
dir string |
|||
dbs map[string]*leveldb.DB |
|||
dbsLock sync.RWMutex |
|||
} |
|||
|
|||
func (store *LevelDB3Store) GetName() string { |
|||
return "leveldb3" |
|||
} |
|||
|
|||
func (store *LevelDB3Store) Initialize(configuration weed_util.Configuration, prefix string) (err error) { |
|||
dir := configuration.GetString(prefix + "dir") |
|||
return store.initialize(dir) |
|||
} |
|||
|
|||
func (store *LevelDB3Store) initialize(dir string) (err error) { |
|||
glog.Infof("filer store leveldb3 dir: %s", dir) |
|||
if err := weed_util.TestFolderWritable(dir); err != nil { |
|||
return fmt.Errorf("Check Level Folder %s Writable: %s", dir, err) |
|||
} |
|||
store.dir = dir |
|||
|
|||
db, loadDbErr := store.loadDB(DEFAULT) |
|||
if loadDbErr != nil { |
|||
return loadDbErr |
|||
} |
|||
store.dbs = make(map[string]*leveldb.DB) |
|||
store.dbs[DEFAULT] = db |
|||
|
|||
return |
|||
} |
|||
|
|||
func (store *LevelDB3Store) loadDB(name string) (*leveldb.DB, error) { |
|||
|
|||
opts := &opt.Options{ |
|||
BlockCacheCapacity: 32 * 1024 * 1024, // default value is 8MiB
|
|||
WriteBuffer: 16 * 1024 * 1024, // default value is 4MiB
|
|||
CompactionTableSizeMultiplier: 4, |
|||
} |
|||
if name != DEFAULT { |
|||
opts = &opt.Options{ |
|||
BlockCacheCapacity: 4 * 1024 * 1024, // default value is 8MiB
|
|||
WriteBuffer: 2 * 1024 * 1024, // default value is 4MiB
|
|||
CompactionTableSizeMultiplier: 4, |
|||
} |
|||
} |
|||
|
|||
dbFolder := fmt.Sprintf("%s/%s", store.dir, name) |
|||
os.MkdirAll(dbFolder, 0755) |
|||
db, dbErr := leveldb.OpenFile(dbFolder, opts) |
|||
if leveldb_errors.IsCorrupted(dbErr) { |
|||
db, dbErr = leveldb.RecoverFile(dbFolder, opts) |
|||
} |
|||
if dbErr != nil { |
|||
glog.Errorf("filer store open dir %s: %v", dbFolder, dbErr) |
|||
return nil, dbErr |
|||
} |
|||
return db, nil |
|||
} |
|||
|
|||
func (store *LevelDB3Store) findDB(fullpath weed_util.FullPath, isForChildren bool) (*leveldb.DB, string, weed_util.FullPath, error) { |
|||
|
|||
store.dbsLock.RLock() |
|||
|
|||
defaultDB := store.dbs[DEFAULT] |
|||
if !strings.HasPrefix(string(fullpath), "/buckets/") { |
|||
store.dbsLock.RUnlock() |
|||
return defaultDB, DEFAULT, fullpath, nil |
|||
} |
|||
|
|||
// detect bucket
|
|||
bucketAndObjectKey := string(fullpath)[len("/buckets/"):] |
|||
t := strings.Index(bucketAndObjectKey, "/") |
|||
if t < 0 && !isForChildren { |
|||
store.dbsLock.RUnlock() |
|||
return defaultDB, DEFAULT, fullpath, nil |
|||
} |
|||
bucket := bucketAndObjectKey |
|||
shortPath := weed_util.FullPath("/") |
|||
if t > 0 { |
|||
bucket = bucketAndObjectKey[:t] |
|||
shortPath = weed_util.FullPath(bucketAndObjectKey[t:]) |
|||
} |
|||
|
|||
if db, found := store.dbs[bucket]; found { |
|||
store.dbsLock.RUnlock() |
|||
return db, bucket, shortPath, nil |
|||
} |
|||
|
|||
store.dbsLock.RUnlock() |
|||
// upgrade to write lock
|
|||
store.dbsLock.Lock() |
|||
defer store.dbsLock.Unlock() |
|||
|
|||
// double check after getting the write lock
|
|||
if db, found := store.dbs[bucket]; found { |
|||
return db, bucket, shortPath, nil |
|||
} |
|||
|
|||
// create db
|
|||
db, err := store.loadDB(bucket) |
|||
if err != nil { |
|||
return nil, bucket, shortPath, err |
|||
} |
|||
store.dbs[bucket] = db |
|||
|
|||
return db, bucket, shortPath, nil |
|||
} |
|||
|
|||
func (store *LevelDB3Store) BeginTransaction(ctx context.Context) (context.Context, error) { |
|||
return ctx, nil |
|||
} |
|||
func (store *LevelDB3Store) CommitTransaction(ctx context.Context) error { |
|||
return nil |
|||
} |
|||
func (store *LevelDB3Store) RollbackTransaction(ctx context.Context) error { |
|||
return nil |
|||
} |
|||
|
|||
func (store *LevelDB3Store) InsertEntry(ctx context.Context, entry *filer.Entry) (err error) { |
|||
|
|||
db, _, shortPath, err := store.findDB(entry.FullPath, false) |
|||
if err != nil { |
|||
return fmt.Errorf("findDB %s : %v", entry.FullPath, err) |
|||
} |
|||
|
|||
dir, name := shortPath.DirAndName() |
|||
key := genKey(dir, name) |
|||
|
|||
value, err := entry.EncodeAttributesAndChunks() |
|||
if err != nil { |
|||
return fmt.Errorf("encoding %s %+v: %v", entry.FullPath, entry.Attr, err) |
|||
} |
|||
|
|||
if len(entry.Chunks) > 50 { |
|||
value = weed_util.MaybeGzipData(value) |
|||
} |
|||
|
|||
err = db.Put(key, value, nil) |
|||
|
|||
if err != nil { |
|||
return fmt.Errorf("persisting %s : %v", entry.FullPath, err) |
|||
} |
|||
|
|||
// println("saved", entry.FullPath, "chunks", len(entry.Chunks))
|
|||
|
|||
return nil |
|||
} |
|||
|
|||
func (store *LevelDB3Store) UpdateEntry(ctx context.Context, entry *filer.Entry) (err error) { |
|||
|
|||
return store.InsertEntry(ctx, entry) |
|||
} |
|||
|
|||
func (store *LevelDB3Store) FindEntry(ctx context.Context, fullpath weed_util.FullPath) (entry *filer.Entry, err error) { |
|||
|
|||
db, _, shortPath, err := store.findDB(fullpath, false) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("findDB %s : %v", fullpath, err) |
|||
} |
|||
|
|||
dir, name := shortPath.DirAndName() |
|||
key := genKey(dir, name) |
|||
|
|||
data, err := db.Get(key, nil) |
|||
|
|||
if err == leveldb.ErrNotFound { |
|||
return nil, filer_pb.ErrNotFound |
|||
} |
|||
if err != nil { |
|||
return nil, fmt.Errorf("get %s : %v", fullpath, err) |
|||
} |
|||
|
|||
entry = &filer.Entry{ |
|||
FullPath: fullpath, |
|||
} |
|||
err = entry.DecodeAttributesAndChunks(weed_util.MaybeDecompressData(data)) |
|||
if err != nil { |
|||
return entry, fmt.Errorf("decode %s : %v", entry.FullPath, err) |
|||
} |
|||
|
|||
// println("read", entry.FullPath, "chunks", len(entry.Chunks), "data", len(data), string(data))
|
|||
|
|||
return entry, nil |
|||
} |
|||
|
|||
func (store *LevelDB3Store) DeleteEntry(ctx context.Context, fullpath weed_util.FullPath) (err error) { |
|||
|
|||
db, _, shortPath, err := store.findDB(fullpath, false) |
|||
if err != nil { |
|||
return fmt.Errorf("findDB %s : %v", fullpath, err) |
|||
} |
|||
|
|||
dir, name := shortPath.DirAndName() |
|||
key := genKey(dir, name) |
|||
|
|||
err = db.Delete(key, nil) |
|||
if err != nil { |
|||
return fmt.Errorf("delete %s : %v", fullpath, err) |
|||
} |
|||
|
|||
return nil |
|||
} |
|||
|
|||
func (store *LevelDB3Store) DeleteFolderChildren(ctx context.Context, fullpath weed_util.FullPath) (err error) { |
|||
|
|||
db, bucket, shortPath, err := store.findDB(fullpath, true) |
|||
if err != nil { |
|||
return fmt.Errorf("findDB %s : %v", fullpath, err) |
|||
} |
|||
|
|||
if bucket != DEFAULT && shortPath == "/" { |
|||
db.Close() |
|||
if bucket != "" { // just to make sure
|
|||
os.RemoveAll(store.dir + "/" + bucket) |
|||
} |
|||
return nil |
|||
} |
|||
|
|||
directoryPrefix := genDirectoryKeyPrefix(shortPath, "") |
|||
|
|||
batch := new(leveldb.Batch) |
|||
|
|||
iter := db.NewIterator(&leveldb_util.Range{Start: directoryPrefix}, nil) |
|||
for iter.Next() { |
|||
key := iter.Key() |
|||
if !bytes.HasPrefix(key, directoryPrefix) { |
|||
break |
|||
} |
|||
fileName := getNameFromKey(key) |
|||
if fileName == "" { |
|||
continue |
|||
} |
|||
batch.Delete(append(directoryPrefix, []byte(fileName)...)) |
|||
} |
|||
iter.Release() |
|||
|
|||
err = db.Write(batch, nil) |
|||
|
|||
if err != nil { |
|||
return fmt.Errorf("delete %s : %v", fullpath, err) |
|||
} |
|||
|
|||
return nil |
|||
} |
|||
|
|||
func (store *LevelDB3Store) ListDirectoryEntries(ctx context.Context, fullpath weed_util.FullPath, startFileName string, inclusive bool, |
|||
limit int) (entries []*filer.Entry, err error) { |
|||
return store.ListDirectoryPrefixedEntries(ctx, fullpath, startFileName, inclusive, limit, "") |
|||
} |
|||
|
|||
func (store *LevelDB3Store) ListDirectoryPrefixedEntries(ctx context.Context, fullpath weed_util.FullPath, startFileName string, inclusive bool, limit int, prefix string) (entries []*filer.Entry, err error) { |
|||
|
|||
db, _, shortPath, err := store.findDB(fullpath, true) |
|||
if err != nil { |
|||
return nil, fmt.Errorf("findDB %s : %v", fullpath, err) |
|||
} |
|||
|
|||
directoryPrefix := genDirectoryKeyPrefix(shortPath, prefix) |
|||
lastFileStart := directoryPrefix |
|||
if startFileName != "" { |
|||
lastFileStart = genDirectoryKeyPrefix(shortPath, startFileName) |
|||
} |
|||
|
|||
iter := db.NewIterator(&leveldb_util.Range{Start: lastFileStart}, nil) |
|||
for iter.Next() { |
|||
key := iter.Key() |
|||
if !bytes.HasPrefix(key, directoryPrefix) { |
|||
break |
|||
} |
|||
fileName := getNameFromKey(key) |
|||
if fileName == "" { |
|||
continue |
|||
} |
|||
if fileName == startFileName && !inclusive { |
|||
continue |
|||
} |
|||
limit-- |
|||
if limit < 0 { |
|||
break |
|||
} |
|||
entry := &filer.Entry{ |
|||
FullPath: weed_util.NewFullPath(string(fullpath), fileName), |
|||
} |
|||
|
|||
// println("list", entry.FullPath, "chunks", len(entry.Chunks))
|
|||
if decodeErr := entry.DecodeAttributesAndChunks(weed_util.MaybeDecompressData(iter.Value())); decodeErr != nil { |
|||
err = decodeErr |
|||
glog.V(0).Infof("list %s : %v", entry.FullPath, err) |
|||
break |
|||
} |
|||
entries = append(entries, entry) |
|||
} |
|||
iter.Release() |
|||
|
|||
return entries, err |
|||
} |
|||
|
|||
func genKey(dirPath, fileName string) (key []byte) { |
|||
key = hashToBytes(dirPath) |
|||
key = append(key, []byte(fileName)...) |
|||
return key |
|||
} |
|||
|
|||
func genDirectoryKeyPrefix(fullpath weed_util.FullPath, startFileName string) (keyPrefix []byte) { |
|||
keyPrefix = hashToBytes(string(fullpath)) |
|||
if len(startFileName) > 0 { |
|||
keyPrefix = append(keyPrefix, []byte(startFileName)...) |
|||
} |
|||
return keyPrefix |
|||
} |
|||
|
|||
func getNameFromKey(key []byte) string { |
|||
|
|||
return string(key[md5.Size:]) |
|||
|
|||
} |
|||
|
|||
// hash directory
|
|||
func hashToBytes(dir string) []byte { |
|||
h := md5.New() |
|||
io.WriteString(h, dir) |
|||
b := h.Sum(nil) |
|||
return b |
|||
} |
|||
|
|||
func (store *LevelDB3Store) Shutdown() { |
|||
for _, db := range store.dbs { |
|||
db.Close() |
|||
} |
|||
} |
@ -0,0 +1,46 @@ |
|||
package leveldb |
|||
|
|||
import ( |
|||
"context" |
|||
"fmt" |
|||
|
|||
"github.com/chrislusf/seaweedfs/weed/filer" |
|||
"github.com/syndtr/goleveldb/leveldb" |
|||
) |
|||
|
|||
func (store *LevelDB3Store) KvPut(ctx context.Context, key []byte, value []byte) (err error) { |
|||
|
|||
err = store.dbs[DEFAULT].Put(key, value, nil) |
|||
|
|||
if err != nil { |
|||
return fmt.Errorf("kv put: %v", err) |
|||
} |
|||
|
|||
return nil |
|||
} |
|||
|
|||
func (store *LevelDB3Store) KvGet(ctx context.Context, key []byte) (value []byte, err error) { |
|||
|
|||
value, err = store.dbs[DEFAULT].Get(key, nil) |
|||
|
|||
if err == leveldb.ErrNotFound { |
|||
return nil, filer.ErrKvNotFound |
|||
} |
|||
|
|||
if err != nil { |
|||
return nil, fmt.Errorf("kv get: %v", err) |
|||
} |
|||
|
|||
return |
|||
} |
|||
|
|||
func (store *LevelDB3Store) KvDelete(ctx context.Context, key []byte) (err error) { |
|||
|
|||
err = store.dbs[DEFAULT].Delete(key, nil) |
|||
|
|||
if err != nil { |
|||
return fmt.Errorf("kv delete: %v", err) |
|||
} |
|||
|
|||
return nil |
|||
} |
@ -0,0 +1,88 @@ |
|||
package leveldb |
|||
|
|||
import ( |
|||
"context" |
|||
"io/ioutil" |
|||
"os" |
|||
"testing" |
|||
|
|||
"github.com/chrislusf/seaweedfs/weed/filer" |
|||
"github.com/chrislusf/seaweedfs/weed/util" |
|||
) |
|||
|
|||
func TestCreateAndFind(t *testing.T) { |
|||
testFiler := filer.NewFiler(nil, nil, "", 0, "", "", "", nil) |
|||
dir, _ := ioutil.TempDir("", "seaweedfs_filer_test") |
|||
defer os.RemoveAll(dir) |
|||
store := &LevelDB3Store{} |
|||
store.initialize(dir) |
|||
testFiler.SetStore(store) |
|||
|
|||
fullpath := util.FullPath("/home/chris/this/is/one/file1.jpg") |
|||
|
|||
ctx := context.Background() |
|||
|
|||
entry1 := &filer.Entry{ |
|||
FullPath: fullpath, |
|||
Attr: filer.Attr{ |
|||
Mode: 0440, |
|||
Uid: 1234, |
|||
Gid: 5678, |
|||
}, |
|||
} |
|||
|
|||
if err := testFiler.CreateEntry(ctx, entry1, false, false, nil); err != nil { |
|||
t.Errorf("create entry %v: %v", entry1.FullPath, err) |
|||
return |
|||
} |
|||
|
|||
entry, err := testFiler.FindEntry(ctx, fullpath) |
|||
|
|||
if err != nil { |
|||
t.Errorf("find entry: %v", err) |
|||
return |
|||
} |
|||
|
|||
if entry.FullPath != entry1.FullPath { |
|||
t.Errorf("find wrong entry: %v", entry.FullPath) |
|||
return |
|||
} |
|||
|
|||
// checking one upper directory
|
|||
entries, _ := testFiler.ListDirectoryEntries(ctx, util.FullPath("/home/chris/this/is/one"), "", false, 100, "", "") |
|||
if len(entries) != 1 { |
|||
t.Errorf("list entries count: %v", len(entries)) |
|||
return |
|||
} |
|||
|
|||
// checking one upper directory
|
|||
entries, _ = testFiler.ListDirectoryEntries(ctx, util.FullPath("/"), "", false, 100, "", "") |
|||
if len(entries) != 1 { |
|||
t.Errorf("list entries count: %v", len(entries)) |
|||
return |
|||
} |
|||
|
|||
} |
|||
|
|||
func TestEmptyRoot(t *testing.T) { |
|||
testFiler := filer.NewFiler(nil, nil, "", 0, "", "", "", nil) |
|||
dir, _ := ioutil.TempDir("", "seaweedfs_filer_test2") |
|||
defer os.RemoveAll(dir) |
|||
store := &LevelDB3Store{} |
|||
store.initialize(dir) |
|||
testFiler.SetStore(store) |
|||
|
|||
ctx := context.Background() |
|||
|
|||
// checking one upper directory
|
|||
entries, err := testFiler.ListDirectoryEntries(ctx, util.FullPath("/"), "", false, 100, "", "") |
|||
if err != nil { |
|||
t.Errorf("list entries: %v", err) |
|||
return |
|||
} |
|||
if len(entries) != 0 { |
|||
t.Errorf("list entries count: %v", len(entries)) |
|||
return |
|||
} |
|||
|
|||
} |
@ -0,0 +1,40 @@ |
|||
//+build rocksdb
|
|||
|
|||
package rocksdb |
|||
|
|||
import ( |
|||
"time" |
|||
|
|||
"github.com/tecbot/gorocksdb" |
|||
|
|||
"github.com/chrislusf/seaweedfs/weed/filer" |
|||
) |
|||
|
|||
type TTLFilter struct { |
|||
skipLevel0 bool |
|||
} |
|||
|
|||
func NewTTLFilter() gorocksdb.CompactionFilter { |
|||
return &TTLFilter{ |
|||
skipLevel0: true, |
|||
} |
|||
} |
|||
|
|||
func (t *TTLFilter) Filter(level int, key, val []byte) (remove bool, newVal []byte) { |
|||
// decode could be slow, causing write stall
|
|||
// level >0 sst can run compaction in parallel
|
|||
if !t.skipLevel0 || level > 0 { |
|||
entry := filer.Entry{} |
|||
if err := entry.DecodeAttributesAndChunks(val); err == nil { |
|||
if entry.TtlSec > 0 && |
|||
entry.Crtime.Add(time.Duration(entry.TtlSec)*time.Second).Before(time.Now()) { |
|||
return true, nil |
|||
} |
|||
} |
|||
} |
|||
return false, val |
|||
} |
|||
|
|||
func (t *TTLFilter) Name() string { |
|||
return "TTLFilter" |
|||
} |
Write
Preview
Loading…
Cancel
Save
Reference in new issue