Chris Lu
4 years ago
5 changed files with 504 additions and 0 deletions
-
6weed/command/scaffold.go
-
363weed/filer/leveldb3/leveldb3_store.go
-
46weed/filer/leveldb3/leveldb3_store_kv.go
-
88weed/filer/leveldb3/leveldb3_store_test.go
-
1weed/server/filer_server.go
@ -0,0 +1,363 @@ |
|||
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:]) |
|||
} |
|||
|
|||
println("bucket:", bucket, "shortPath", shortPath) |
|||
|
|||
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, 2) |
|||
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, 2) |
|||
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 |
|||
} |
|||
|
|||
} |
Write
Preview
Loading…
Cancel
Save
Reference in new issue