{{ range $entry := .Breadcrumbs }}
-
+
{{ $entry.Name }}
{{ end }}
@@ -78,20 +78,19 @@ var StatusTpl = template.Must(template.New("status").Funcs(funcMap).Parse(`
{{end}}
-
+ |
{{if $entry.IsDirectory}}
{{else}}
- {{ $entry.Mime }}
+ {{ $entry.Mime }}
{{end}}
|
-
+ |
{{if $entry.IsDirectory}}
{{else}}
- {{ $entry.Size | humanizeBytes }}
-
+ {{ $entry.Size | humanizeBytes }}
{{end}}
|
-
+ |
{{ $entry.Timestamp.Format "2006-01-02 15:04" }}
|
diff --git a/weed/server/master_grpc_server.go b/weed/server/master_grpc_server.go
index 82a190e39..84087df8b 100644
--- a/weed/server/master_grpc_server.go
+++ b/weed/server/master_grpc_server.go
@@ -1,17 +1,20 @@
package weed_server
import (
+ "context"
"fmt"
"net"
"strings"
"time"
"github.com/chrislusf/raft"
+ "google.golang.org/grpc/peer"
+
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/chrislusf/seaweedfs/weed/topology"
- "google.golang.org/grpc/peer"
)
func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServer) error {
@@ -60,14 +63,6 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
t.Sequence.SetMax(heartbeat.MaxFileKey)
if dn == nil {
- if heartbeat.Ip == "" {
- if pr, ok := peer.FromContext(stream.Context()); ok {
- if pr.Addr != net.Addr(nil) {
- heartbeat.Ip = pr.Addr.String()[0:strings.LastIndex(pr.Addr.String(), ":")]
- glog.V(0).Infof("remote IP address is detected as %v", heartbeat.Ip)
- }
- }
- }
dcName, rackName := t.Configuration.Locate(heartbeat.Ip, heartbeat.DataCenter, heartbeat.Rack)
dc := t.GetOrCreateDataCenter(dcName)
rack := dc.GetOrCreateRack(rackName)
@@ -76,7 +71,10 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
int64(heartbeat.MaxVolumeCount))
glog.V(0).Infof("added volume server %v:%d", heartbeat.GetIp(), heartbeat.GetPort())
if err := stream.Send(&master_pb.HeartbeatResponse{
- VolumeSizeLimit: uint64(ms.option.VolumeSizeLimitMB) * 1024 * 1024,
+ VolumeSizeLimit: uint64(ms.option.VolumeSizeLimitMB) * 1024 * 1024,
+ MetricsAddress: ms.option.MetricsAddress,
+ MetricsIntervalSeconds: uint32(ms.option.MetricsIntervalSec),
+ StorageBackends: backend.ToPbStorageBackends(),
}); err != nil {
glog.Warningf("SendHeartbeat.Send volume size to %s:%d %v", dn.Ip, dn.Port, err)
return err
@@ -164,9 +162,7 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
return err
}
if err := stream.Send(&master_pb.HeartbeatResponse{
- Leader: newLeader,
- MetricsAddress: ms.option.MetricsAddress,
- MetricsIntervalSeconds: uint32(ms.option.MetricsIntervalSec),
+ Leader: newLeader,
}); err != nil {
glog.Warningf("SendHeartbeat.Send response to to %s:%d %v", dn.Ip, dn.Port, err)
return err
@@ -187,35 +183,13 @@ func (ms *MasterServer) KeepConnected(stream master_pb.Seaweed_KeepConnectedServ
return ms.informNewLeader(stream)
}
- // remember client address
- ctx := stream.Context()
- // fmt.Printf("FromContext %+v\n", ctx)
- pr, ok := peer.FromContext(ctx)
- if !ok {
- glog.Error("failed to get peer from ctx")
- return fmt.Errorf("failed to get peer from ctx")
- }
- if pr.Addr == net.Addr(nil) {
- glog.Error("failed to get peer address")
- return fmt.Errorf("failed to get peer address")
- }
+ peerAddress := findClientAddress(stream.Context(), req.GrpcPort)
- clientName := req.Name + pr.Addr.String()
- glog.V(0).Infof("+ client %v", clientName)
-
- messageChan := make(chan *master_pb.VolumeLocation)
stopChan := make(chan bool)
- ms.clientChansLock.Lock()
- ms.clientChans[clientName] = messageChan
- ms.clientChansLock.Unlock()
+ clientName, messageChan := ms.addClient(req.Name, peerAddress)
- defer func() {
- glog.V(0).Infof("- client %v", clientName)
- ms.clientChansLock.Lock()
- delete(ms.clientChans, clientName)
- ms.clientChansLock.Unlock()
- }()
+ defer ms.deleteClient(clientName)
for _, message := range ms.Topo.ToVolumeLocations() {
if err := stream.Send(message); err != nil {
@@ -267,3 +241,57 @@ func (ms *MasterServer) informNewLeader(stream master_pb.Seaweed_KeepConnectedSe
}
return nil
}
+
+func (ms *MasterServer) addClient(clientType string, clientAddress string) (clientName string, messageChan chan *master_pb.VolumeLocation) {
+ clientName = clientType + "@" + clientAddress
+ glog.V(0).Infof("+ client %v", clientName)
+
+ messageChan = make(chan *master_pb.VolumeLocation)
+
+ ms.clientChansLock.Lock()
+ ms.clientChans[clientName] = messageChan
+ ms.clientChansLock.Unlock()
+ return
+}
+
+func (ms *MasterServer) deleteClient(clientName string) {
+ glog.V(0).Infof("- client %v", clientName)
+ ms.clientChansLock.Lock()
+ delete(ms.clientChans, clientName)
+ ms.clientChansLock.Unlock()
+}
+
+func findClientAddress(ctx context.Context, grpcPort uint32) string {
+ // fmt.Printf("FromContext %+v\n", ctx)
+ pr, ok := peer.FromContext(ctx)
+ if !ok {
+ glog.Error("failed to get peer from ctx")
+ return ""
+ }
+ if pr.Addr == net.Addr(nil) {
+ glog.Error("failed to get peer address")
+ return ""
+ }
+ if grpcPort == 0 {
+ return pr.Addr.String()
+ }
+ if tcpAddr, ok := pr.Addr.(*net.TCPAddr); ok {
+ externalIP := tcpAddr.IP
+ return fmt.Sprintf("%s:%d", externalIP, grpcPort)
+ }
+ return pr.Addr.String()
+
+}
+
+func (ms *MasterServer) ListMasterClients(ctx context.Context, req *master_pb.ListMasterClientsRequest) (*master_pb.ListMasterClientsResponse, error) {
+ resp := &master_pb.ListMasterClientsResponse{}
+ ms.clientChansLock.RLock()
+ defer ms.clientChansLock.RUnlock()
+
+ for k := range ms.clientChans {
+ if strings.HasPrefix(k, req.ClientType+"@") {
+ resp.GrpcAddresses = append(resp.GrpcAddresses, k[len(req.ClientType)+1:])
+ }
+ }
+ return resp, nil
+}
diff --git a/weed/server/master_grpc_server_collection.go b/weed/server/master_grpc_server_collection.go
index f8e0785f6..b92d6bcbe 100644
--- a/weed/server/master_grpc_server_collection.go
+++ b/weed/server/master_grpc_server_collection.go
@@ -4,6 +4,7 @@ import (
"context"
"github.com/chrislusf/raft"
+
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
diff --git a/weed/server/master_grpc_server_volume.go b/weed/server/master_grpc_server_volume.go
index 8fc56e9b8..856c07890 100644
--- a/weed/server/master_grpc_server_volume.go
+++ b/weed/server/master_grpc_server_volume.go
@@ -5,10 +5,11 @@ import (
"fmt"
"github.com/chrislusf/raft"
+
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/security"
- "github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"github.com/chrislusf/seaweedfs/weed/topology"
)
@@ -52,7 +53,7 @@ func (ms *MasterServer) Assign(ctx context.Context, req *master_pb.AssignRequest
if req.Replication == "" {
req.Replication = ms.option.DefaultReplicaPlacement
}
- replicaPlacement, err := storage.NewReplicaPlacementFromString(req.Replication)
+ replicaPlacement, err := super_block.NewReplicaPlacementFromString(req.Replication)
if err != nil {
return nil, err
}
@@ -108,7 +109,7 @@ func (ms *MasterServer) Statistics(ctx context.Context, req *master_pb.Statistic
if req.Replication == "" {
req.Replication = ms.option.DefaultReplicaPlacement
}
- replicaPlacement, err := storage.NewReplicaPlacementFromString(req.Replication)
+ replicaPlacement, err := super_block.NewReplicaPlacementFromString(req.Replication)
if err != nil {
return nil, err
}
diff --git a/weed/server/master_server.go b/weed/server/master_server.go
index 33a5129da..a9ae6b888 100644
--- a/weed/server/master_server.go
+++ b/weed/server/master_server.go
@@ -1,7 +1,6 @@
package weed_server
import (
- "context"
"fmt"
"net/http"
"net/http/httputil"
@@ -14,6 +13,9 @@ import (
"time"
"github.com/chrislusf/raft"
+ "github.com/gorilla/mux"
+ "google.golang.org/grpc"
+
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/security"
@@ -22,9 +24,6 @@ import (
"github.com/chrislusf/seaweedfs/weed/topology"
"github.com/chrislusf/seaweedfs/weed/util"
"github.com/chrislusf/seaweedfs/weed/wdclient"
- "github.com/gorilla/mux"
- "github.com/spf13/viper"
- "google.golang.org/grpc"
)
const (
@@ -69,7 +68,7 @@ type MasterServer struct {
func NewMasterServer(r *mux.Router, option *MasterOption, peers []string) *MasterServer {
- v := viper.GetViper()
+ v := util.GetViper()
signingKey := v.GetString("jwt.signing.key")
v.SetDefault("jwt.signing.expires_after_seconds", 10)
expiresAfterSec := v.GetInt("jwt.signing.expires_after_seconds")
@@ -83,13 +82,13 @@ func NewMasterServer(r *mux.Router, option *MasterOption, peers []string) *Maste
preallocateSize = int64(option.VolumeSizeLimitMB) * (1 << 20)
}
- grpcDialOption := security.LoadClientTLS(v.Sub("grpc"), "master")
+ grpcDialOption := security.LoadClientTLS(v, "grpc.master")
ms := &MasterServer{
option: option,
preallocateSize: preallocateSize,
clientChans: make(map[string]chan *master_pb.VolumeLocation),
grpcDialOption: grpcDialOption,
- MasterClient: wdclient.NewMasterClient(context.Background(), grpcDialOption, "master", peers),
+ MasterClient: wdclient.NewMasterClient(grpcDialOption, "master", 0, peers),
}
ms.bounedLeaderChan = make(chan int, 16)
@@ -115,9 +114,11 @@ func NewMasterServer(r *mux.Router, option *MasterOption, peers []string) *Maste
r.HandleFunc("/vol/status", ms.proxyToLeader(ms.guard.WhiteList(ms.volumeStatusHandler)))
r.HandleFunc("/vol/vacuum", ms.proxyToLeader(ms.guard.WhiteList(ms.volumeVacuumHandler)))
r.HandleFunc("/submit", ms.guard.WhiteList(ms.submitFromMasterServerHandler))
- r.HandleFunc("/stats/health", ms.guard.WhiteList(statsHealthHandler))
- r.HandleFunc("/stats/counter", ms.guard.WhiteList(statsCounterHandler))
- r.HandleFunc("/stats/memory", ms.guard.WhiteList(statsMemoryHandler))
+ /*
+ r.HandleFunc("/stats/health", ms.guard.WhiteList(statsHealthHandler))
+ r.HandleFunc("/stats/counter", ms.guard.WhiteList(statsCounterHandler))
+ r.HandleFunc("/stats/memory", ms.guard.WhiteList(statsMemoryHandler))
+ */
r.HandleFunc("/{fileId}", ms.redirectHandler)
}
@@ -183,7 +184,7 @@ func (ms *MasterServer) proxyToLeader(f func(w http.ResponseWriter, r *http.Requ
func (ms *MasterServer) startAdminScripts() {
var err error
- v := viper.GetViper()
+ v := util.GetViper()
adminScripts := v.GetString("master.maintenance.scripts")
glog.V(0).Infof("adminScripts:\n%v", adminScripts)
if adminScripts == "" {
@@ -201,7 +202,7 @@ func (ms *MasterServer) startAdminScripts() {
masterAddress := "localhost:" + strconv.Itoa(ms.option.Port)
var shellOptions shell.ShellOptions
- shellOptions.GrpcDialOption = security.LoadClientTLS(viper.Sub("grpc"), "master")
+ shellOptions.GrpcDialOption = security.LoadClientTLS(v, "grpc.master")
shellOptions.Masters = &masterAddress
shellOptions.FilerHost, shellOptions.FilerPort, shellOptions.Directory, err = util.ParseFilerUrl(filerURL)
@@ -220,7 +221,7 @@ func (ms *MasterServer) startAdminScripts() {
commandEnv.MasterClient.WaitUntilConnected()
c := time.Tick(time.Duration(sleepMinutes) * time.Minute)
- for _ = range c {
+ for range c {
if ms.Topo.IsLeader() {
for _, line := range scriptLines {
@@ -250,7 +251,7 @@ func (ms *MasterServer) startAdminScripts() {
func (ms *MasterServer) createSequencer(option *MasterOption) sequence.Sequencer {
var seq sequence.Sequencer
- v := viper.GetViper()
+ v := util.GetViper()
seqType := strings.ToLower(v.GetString(SequencerType))
glog.V(1).Infof("[%s] : [%s]", SequencerType, seqType)
switch strings.ToLower(seqType) {
diff --git a/weed/server/master_server_handlers_admin.go b/weed/server/master_server_handlers_admin.go
index a5d976008..5d0986f97 100644
--- a/weed/server/master_server_handlers_admin.go
+++ b/weed/server/master_server_handlers_admin.go
@@ -10,9 +10,9 @@ import (
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
- "github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/backend/memory_map"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"github.com/chrislusf/seaweedfs/weed/topology"
"github.com/chrislusf/seaweedfs/weed/util"
)
@@ -61,7 +61,7 @@ func (ms *MasterServer) volumeVacuumHandler(w http.ResponseWriter, r *http.Reque
return
}
}
- glog.Infoln("garbageThreshold =", gcThreshold)
+ // glog.Infoln("garbageThreshold =", gcThreshold)
ms.Topo.Vacuum(ms.grpcDialOption, gcThreshold, ms.preallocateSize)
ms.dirStatusHandler(w, r)
}
@@ -145,7 +145,7 @@ func (ms *MasterServer) getVolumeGrowOption(r *http.Request) (*topology.VolumeGr
if replicationString == "" {
replicationString = ms.option.DefaultReplicaPlacement
}
- replicaPlacement, err := storage.NewReplicaPlacementFromString(replicationString)
+ replicaPlacement, err := super_block.NewReplicaPlacementFromString(replicationString)
if err != nil {
return nil, err
}
diff --git a/weed/server/msg_broker_grpc_server.go b/weed/server/msg_broker_grpc_server.go
new file mode 100644
index 000000000..8b13aac76
--- /dev/null
+++ b/weed/server/msg_broker_grpc_server.go
@@ -0,0 +1,23 @@
+package weed_server
+
+import (
+ "context"
+
+ "github.com/chrislusf/seaweedfs/weed/pb/queue_pb"
+)
+
+func (broker *MessageBroker) ConfigureTopic(context.Context, *queue_pb.ConfigureTopicRequest) (*queue_pb.ConfigureTopicResponse, error) {
+ panic("implement me")
+}
+
+func (broker *MessageBroker) DeleteTopic(context.Context, *queue_pb.DeleteTopicRequest) (*queue_pb.DeleteTopicResponse, error) {
+ panic("implement me")
+}
+
+func (broker *MessageBroker) StreamWrite(queue_pb.SeaweedQueue_StreamWriteServer) error {
+ panic("implement me")
+}
+
+func (broker *MessageBroker) StreamRead(*queue_pb.ReadMessageRequest, queue_pb.SeaweedQueue_StreamReadServer) error {
+ panic("implement me")
+}
diff --git a/weed/server/msg_broker_server.go b/weed/server/msg_broker_server.go
new file mode 100644
index 000000000..a9d908581
--- /dev/null
+++ b/weed/server/msg_broker_server.go
@@ -0,0 +1,121 @@
+package weed_server
+
+import (
+ "context"
+ "fmt"
+ "time"
+
+ "google.golang.org/grpc"
+
+ "github.com/chrislusf/seaweedfs/weed/pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/security"
+ "github.com/chrislusf/seaweedfs/weed/util"
+)
+
+type MessageBrokerOption struct {
+ Filers []string
+ DefaultReplication string
+ MaxMB int
+ Port int
+}
+
+type MessageBroker struct {
+ option *MessageBrokerOption
+ grpcDialOption grpc.DialOption
+}
+
+func NewMessageBroker(option *MessageBrokerOption) (messageBroker *MessageBroker, err error) {
+
+ messageBroker = &MessageBroker{
+ option: option,
+ grpcDialOption: security.LoadClientTLS(util.GetViper(), "grpc.msg_broker"),
+ }
+
+ go messageBroker.loopForEver()
+
+ return messageBroker, nil
+}
+
+func (broker *MessageBroker) loopForEver() {
+
+ for {
+ broker.checkPeers()
+ time.Sleep(3 * time.Second)
+ }
+
+}
+
+func (broker *MessageBroker) checkPeers() {
+
+ // contact a filer about masters
+ var masters []string
+ for _, filer := range broker.option.Filers {
+ err := broker.withFilerClient(filer, func(client filer_pb.SeaweedFilerClient) error {
+ resp, err := client.GetFilerConfiguration(context.Background(), &filer_pb.GetFilerConfigurationRequest{})
+ if err != nil {
+ return err
+ }
+ masters = append(masters, resp.Masters...)
+ return nil
+ })
+ if err != nil {
+ fmt.Printf("failed to read masters from %+v: %v\n", broker.option.Filers, err)
+ return
+ }
+ }
+
+ // contact each masters for filers
+ var filers []string
+ for _, master := range masters {
+ err := broker.withMasterClient(master, func(client master_pb.SeaweedClient) error {
+ resp, err := client.ListMasterClients(context.Background(), &master_pb.ListMasterClientsRequest{
+ ClientType: "filer",
+ })
+ if err != nil {
+ return err
+ }
+
+ fmt.Printf("filers: %+v\n", resp.GrpcAddresses)
+ filers = append(filers, resp.GrpcAddresses...)
+
+ return nil
+ })
+ if err != nil {
+ fmt.Printf("failed to list filers: %v\n", err)
+ return
+ }
+ }
+
+ // contact each filer about brokers
+ for _, filer := range filers {
+ err := broker.withFilerClient(filer, func(client filer_pb.SeaweedFilerClient) error {
+ resp, err := client.GetFilerConfiguration(context.Background(), &filer_pb.GetFilerConfigurationRequest{})
+ if err != nil {
+ return err
+ }
+ masters = append(masters, resp.Masters...)
+ return nil
+ })
+ if err != nil {
+ fmt.Printf("failed to read masters from %+v: %v\n", broker.option.Filers, err)
+ return
+ }
+ }
+
+}
+
+func (broker *MessageBroker) withFilerClient(filer string, fn func(filer_pb.SeaweedFilerClient) error) error {
+
+ return pb.WithFilerClient(filer, broker.grpcDialOption, fn)
+
+}
+
+func (broker *MessageBroker) withMasterClient(master string, fn func(client master_pb.SeaweedClient) error) error {
+
+ return pb.WithMasterClient(master, broker.grpcDialOption, func(client master_pb.SeaweedClient) error {
+ return fn(client)
+ })
+
+}
diff --git a/weed/server/raft_server.go b/weed/server/raft_server.go
index 53289f1c1..0381c7feb 100644
--- a/weed/server/raft_server.go
+++ b/weed/server/raft_server.go
@@ -2,8 +2,6 @@ package weed_server
import (
"encoding/json"
- "github.com/chrislusf/seaweedfs/weed/util"
- "google.golang.org/grpc"
"io/ioutil"
"os"
"path"
@@ -11,7 +9,12 @@ import (
"sort"
"time"
+ "google.golang.org/grpc"
+
+ "github.com/chrislusf/seaweedfs/weed/pb"
+
"github.com/chrislusf/raft"
+
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/topology"
)
@@ -61,7 +64,7 @@ func NewRaftServer(grpcDialOption grpc.DialOption, peers []string, serverAddr, d
s.raftServer.Start()
for _, peer := range s.peers {
- s.raftServer.AddPeer(peer, util.ServerToGrpcAddress(peer))
+ s.raftServer.AddPeer(peer, pb.ServerToGrpcAddress(peer))
}
s.GrpcServer = raft.NewGrpcServer(s.raftServer)
@@ -72,7 +75,7 @@ func NewRaftServer(grpcDialOption grpc.DialOption, peers []string, serverAddr, d
_, err := s.raftServer.Do(&raft.DefaultJoinCommand{
Name: s.raftServer.Name(),
- ConnectionString: util.ServerToGrpcAddress(s.serverAddr),
+ ConnectionString: pb.ServerToGrpcAddress(s.serverAddr),
})
if err != nil {
diff --git a/weed/server/volume_grpc_admin.go b/weed/server/volume_grpc_admin.go
index c631d2535..27b21ac09 100644
--- a/weed/server/volume_grpc_admin.go
+++ b/weed/server/volume_grpc_admin.go
@@ -2,10 +2,14 @@ package weed_server
import (
"context"
+ "fmt"
+ "path/filepath"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/stats"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
)
func (vs *VolumeServer) DeleteCollection(ctx context.Context, req *volume_server_pb.DeleteCollectionRequest) (*volume_server_pb.DeleteCollectionResponse, error) {
@@ -96,6 +100,41 @@ func (vs *VolumeServer) VolumeDelete(ctx context.Context, req *volume_server_pb.
}
+func (vs *VolumeServer) VolumeConfigure(ctx context.Context, req *volume_server_pb.VolumeConfigureRequest) (*volume_server_pb.VolumeConfigureResponse, error) {
+
+ resp := &volume_server_pb.VolumeConfigureResponse{}
+
+ // check replication format
+ if _, err := super_block.NewReplicaPlacementFromString(req.Replication); err != nil {
+ resp.Error = fmt.Sprintf("volume configure replication %v: %v", req, err)
+ return resp, nil
+ }
+
+ // unmount
+ if err := vs.store.UnmountVolume(needle.VolumeId(req.VolumeId)); err != nil {
+ glog.Errorf("volume configure unmount %v: %v", req, err)
+ resp.Error = fmt.Sprintf("volume configure unmount %v: %v", req, err)
+ return resp, nil
+ }
+
+ // modify the volume info file
+ if err := vs.store.ConfigureVolume(needle.VolumeId(req.VolumeId), req.Replication); err != nil {
+ glog.Errorf("volume configure %v: %v", req, err)
+ resp.Error = fmt.Sprintf("volume configure %v: %v", req, err)
+ return resp, nil
+ }
+
+ // mount
+ if err := vs.store.MountVolume(needle.VolumeId(req.VolumeId)); err != nil {
+ glog.Errorf("volume configure mount %v: %v", req, err)
+ resp.Error = fmt.Sprintf("volume configure mount %v: %v", req, err)
+ return resp, nil
+ }
+
+ return resp, nil
+
+}
+
func (vs *VolumeServer) VolumeMarkReadonly(ctx context.Context, req *volume_server_pb.VolumeMarkReadonlyRequest) (*volume_server_pb.VolumeMarkReadonlyResponse, error) {
resp := &volume_server_pb.VolumeMarkReadonlyResponse{}
@@ -111,3 +150,19 @@ func (vs *VolumeServer) VolumeMarkReadonly(ctx context.Context, req *volume_serv
return resp, err
}
+
+func (vs *VolumeServer) VolumeServerStatus(ctx context.Context, req *volume_server_pb.VolumeServerStatusRequest) (*volume_server_pb.VolumeServerStatusResponse, error) {
+
+ resp := &volume_server_pb.VolumeServerStatusResponse{}
+
+ for _, loc := range vs.store.Locations {
+ if dir, e := filepath.Abs(loc.Directory); e == nil {
+ resp.DiskStatuses = append(resp.DiskStatuses, stats.NewDiskStatus(dir))
+ }
+ }
+
+ resp.MemoryStatus = stats.MemStat()
+
+ return resp, nil
+
+}
diff --git a/weed/server/volume_grpc_client_to_master.go b/weed/server/volume_grpc_client_to_master.go
index 731675b48..1f4d9df10 100644
--- a/weed/server/volume_grpc_client_to_master.go
+++ b/weed/server/volume_grpc_client_to_master.go
@@ -5,15 +5,18 @@ import (
"net"
"time"
+ "google.golang.org/grpc"
+
+ "github.com/chrislusf/seaweedfs/weed/pb"
"github.com/chrislusf/seaweedfs/weed/security"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
- "github.com/spf13/viper"
- "google.golang.org/grpc"
+
+ "golang.org/x/net/context"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/util"
- "golang.org/x/net/context"
)
func (vs *VolumeServer) GetMaster() string {
@@ -25,7 +28,7 @@ func (vs *VolumeServer) heartbeat() {
vs.store.SetDataCenter(vs.dataCenter)
vs.store.SetRack(vs.rack)
- grpcDialOption := security.LoadClientTLS(viper.Sub("grpc"), "volume")
+ grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.volume")
var err error
var newLeader string
@@ -34,13 +37,13 @@ func (vs *VolumeServer) heartbeat() {
if newLeader != "" {
master = newLeader
}
- masterGrpcAddress, parseErr := util.ParseServerToGrpcAddress(master)
+ masterGrpcAddress, parseErr := pb.ParseServerToGrpcAddress(master)
if parseErr != nil {
glog.V(0).Infof("failed to parse master grpc %v: %v", masterGrpcAddress, parseErr)
continue
}
vs.store.MasterAddress = master
- newLeader, err = vs.doHeartbeat(context.Background(), master, masterGrpcAddress, grpcDialOption, time.Duration(vs.pulseSeconds)*time.Second)
+ newLeader, err = vs.doHeartbeat(master, masterGrpcAddress, grpcDialOption, time.Duration(vs.pulseSeconds)*time.Second)
if err != nil {
glog.V(0).Infof("heartbeat error: %v", err)
time.Sleep(time.Duration(vs.pulseSeconds) * time.Second)
@@ -51,16 +54,16 @@ func (vs *VolumeServer) heartbeat() {
}
}
-func (vs *VolumeServer) doHeartbeat(ctx context.Context, masterNode, masterGrpcAddress string, grpcDialOption grpc.DialOption, sleepInterval time.Duration) (newLeader string, err error) {
+func (vs *VolumeServer) doHeartbeat(masterNode, masterGrpcAddress string, grpcDialOption grpc.DialOption, sleepInterval time.Duration) (newLeader string, err error) {
- grpcConection, err := util.GrpcDial(ctx, masterGrpcAddress, grpcDialOption)
+ grpcConection, err := pb.GrpcDial(context.Background(), masterGrpcAddress, grpcDialOption)
if err != nil {
return "", fmt.Errorf("fail to dial %s : %v", masterNode, err)
}
defer grpcConection.Close()
client := master_pb.NewSeaweedClient(grpcConection)
- stream, err := client.SendHeartbeat(ctx)
+ stream, err := client.SendHeartbeat(context.Background())
if err != nil {
glog.V(0).Infof("SendHeartbeat to %s: %v", masterNode, err)
return "", err
@@ -90,6 +93,9 @@ func (vs *VolumeServer) doHeartbeat(ctx context.Context, masterNode, masterGrpcA
vs.MetricsAddress = in.GetMetricsAddress()
vs.MetricsIntervalSec = int(in.GetMetricsIntervalSeconds())
}
+ if len(in.StorageBackends) > 0 {
+ backend.LoadFromPbStorageBackends(in.StorageBackends)
+ }
}
}()
diff --git a/weed/server/volume_grpc_copy.go b/weed/server/volume_grpc_copy.go
index 711a3ebad..5cc75e74c 100644
--- a/weed/server/volume_grpc_copy.go
+++ b/weed/server/volume_grpc_copy.go
@@ -20,7 +20,7 @@ import (
const BufferSizeLimit = 1024 * 1024 * 2
-// VolumeCopy copy the .idx .dat files, and mount the volume
+// VolumeCopy copy the .idx .dat .vif files, and mount the volume
func (vs *VolumeServer) VolumeCopy(ctx context.Context, req *volume_server_pb.VolumeCopyRequest) (*volume_server_pb.VolumeCopyResponse, error) {
v := vs.store.GetVolume(needle.VolumeId(req.VolumeId))
@@ -43,7 +43,7 @@ func (vs *VolumeServer) VolumeCopy(ctx context.Context, req *volume_server_pb.Vo
var volumeFileName, idxFileName, datFileName string
err := operation.WithVolumeServerClient(req.SourceDataNode, vs.grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
var err error
- volFileInfoResp, err = client.ReadVolumeFileStatus(ctx,
+ volFileInfoResp, err = client.ReadVolumeFileStatus(context.Background(),
&volume_server_pb.ReadVolumeFileStatusRequest{
VolumeId: req.VolumeId,
})
@@ -55,11 +55,15 @@ func (vs *VolumeServer) VolumeCopy(ctx context.Context, req *volume_server_pb.Vo
// println("source:", volFileInfoResp.String())
// copy ecx file
- if err := vs.doCopyFile(ctx, client, false, req.Collection, req.VolumeId, volFileInfoResp.CompactionRevision, volFileInfoResp.IdxFileSize, volumeFileName, ".idx", false); err != nil {
+ if err := vs.doCopyFile(client, false, req.Collection, req.VolumeId, volFileInfoResp.CompactionRevision, volFileInfoResp.IdxFileSize, volumeFileName, ".idx", false, false); err != nil {
return err
}
- if err := vs.doCopyFile(ctx, client, false, req.Collection, req.VolumeId, volFileInfoResp.CompactionRevision, volFileInfoResp.DatFileSize, volumeFileName, ".dat", false); err != nil {
+ if err := vs.doCopyFile(client, false, req.Collection, req.VolumeId, volFileInfoResp.CompactionRevision, volFileInfoResp.DatFileSize, volumeFileName, ".dat", false, true); err != nil {
+ return err
+ }
+
+ if err := vs.doCopyFile(client, false, req.Collection, req.VolumeId, volFileInfoResp.CompactionRevision, volFileInfoResp.DatFileSize, volumeFileName, ".vif", false, true); err != nil {
return err
}
@@ -70,12 +74,9 @@ func (vs *VolumeServer) VolumeCopy(ctx context.Context, req *volume_server_pb.Vo
datFileName = volumeFileName + ".dat"
if err != nil && volumeFileName != "" {
- if idxFileName != "" {
- os.Remove(idxFileName)
- }
- if datFileName != "" {
- os.Remove(datFileName)
- }
+ os.Remove(idxFileName)
+ os.Remove(datFileName)
+ os.Remove(volumeFileName + ".vif")
return nil, err
}
@@ -94,16 +95,16 @@ func (vs *VolumeServer) VolumeCopy(ctx context.Context, req *volume_server_pb.Vo
}, err
}
-func (vs *VolumeServer) doCopyFile(ctx context.Context, client volume_server_pb.VolumeServerClient, isEcVolume bool, collection string, vid uint32,
- compactRevision uint32, stopOffset uint64, baseFileName, ext string, isAppend bool) error {
+func (vs *VolumeServer) doCopyFile(client volume_server_pb.VolumeServerClient, isEcVolume bool, collection string, vid, compactRevision uint32, stopOffset uint64, baseFileName, ext string, isAppend, ignoreSourceFileNotFound bool) error {
- copyFileClient, err := client.CopyFile(ctx, &volume_server_pb.CopyFileRequest{
- VolumeId: vid,
- Ext: ext,
- CompactionRevision: compactRevision,
- StopOffset: stopOffset,
- Collection: collection,
- IsEcVolume: isEcVolume,
+ copyFileClient, err := client.CopyFile(context.Background(), &volume_server_pb.CopyFileRequest{
+ VolumeId: vid,
+ Ext: ext,
+ CompactionRevision: compactRevision,
+ StopOffset: stopOffset,
+ Collection: collection,
+ IsEcVolume: isEcVolume,
+ IgnoreSourceFileNotFound: ignoreSourceFileNotFound,
})
if err != nil {
return fmt.Errorf("failed to start copying volume %d %s file: %v", vid, ext, err)
@@ -213,6 +214,9 @@ func (vs *VolumeServer) CopyFile(req *volume_server_pb.CopyFileRequest, stream v
}
}
if fileName == "" {
+ if req.IgnoreSourceFileNotFound {
+ return nil
+ }
return fmt.Errorf("CopyFile not found ec volume id %d", req.VolumeId)
}
}
@@ -221,6 +225,9 @@ func (vs *VolumeServer) CopyFile(req *volume_server_pb.CopyFileRequest, stream v
file, err := os.Open(fileName)
if err != nil {
+ if req.IgnoreSourceFileNotFound && err == os.ErrNotExist {
+ return nil
+ }
return err
}
defer file.Close()
diff --git a/weed/server/volume_grpc_copy_incremental.go b/weed/server/volume_grpc_copy_incremental.go
index 6c5bb8a62..6d6c3daa3 100644
--- a/weed/server/volume_grpc_copy_incremental.go
+++ b/weed/server/volume_grpc_copy_incremental.go
@@ -47,7 +47,7 @@ func (vs *VolumeServer) VolumeSyncStatus(ctx context.Context, req *volume_server
}
-func sendFileContent(datBackend backend.DataStorageBackend, buf []byte, startOffset, stopOffset int64, stream volume_server_pb.VolumeServer_VolumeIncrementalCopyServer) error {
+func sendFileContent(datBackend backend.BackendStorageFile, buf []byte, startOffset, stopOffset int64, stream volume_server_pb.VolumeServer_VolumeIncrementalCopyServer) error {
var blockSizeLimit = int64(len(buf))
for i := int64(0); i < stopOffset-startOffset; i += blockSizeLimit {
n, readErr := datBackend.ReadAt(buf, startOffset+i)
diff --git a/weed/server/volume_grpc_erasure_coding.go b/weed/server/volume_grpc_erasure_coding.go
index 242480197..66dd5bf8d 100644
--- a/weed/server/volume_grpc_erasure_coding.go
+++ b/weed/server/volume_grpc_erasure_coding.go
@@ -8,10 +8,12 @@ import (
"math"
"os"
"path"
+ "path/filepath"
"strings"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/operation"
+ "github.com/chrislusf/seaweedfs/weed/pb"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
@@ -24,7 +26,7 @@ import (
Steps to apply erasure coding to .dat .idx files
0. ensure the volume is readonly
-1. client call VolumeEcShardsGenerate to generate the .ecx and .ec01~.ec14 files
+1. client call VolumeEcShardsGenerate to generate the .ecx and .ec00 ~ .ec13 files
2. client ask master for possible servers to hold the ec files, at least 4 servers
3. client call VolumeEcShardsCopy on above target servers to copy ec files from the source server
4. target servers report the new ec files to the master
@@ -33,7 +35,7 @@ Steps to apply erasure coding to .dat .idx files
*/
-// VolumeEcShardsGenerate generates the .ecx and .ec01 ~ .ec14 files
+// VolumeEcShardsGenerate generates the .ecx and .ec00 ~ .ec13 files
func (vs *VolumeServer) VolumeEcShardsGenerate(ctx context.Context, req *volume_server_pb.VolumeEcShardsGenerateRequest) (*volume_server_pb.VolumeEcShardsGenerateResponse, error) {
v := vs.store.GetVolume(needle.VolumeId(req.VolumeId))
@@ -47,19 +49,24 @@ func (vs *VolumeServer) VolumeEcShardsGenerate(ctx context.Context, req *volume_
}
// write .ecx file
- if err := erasure_coding.WriteSortedEcxFile(baseFileName); err != nil {
- return nil, fmt.Errorf("WriteSortedEcxFile %s: %v", baseFileName, err)
+ if err := erasure_coding.WriteSortedFileFromIdx(baseFileName, ".ecx"); err != nil {
+ return nil, fmt.Errorf("WriteSortedFileFromIdx %s: %v", baseFileName, err)
}
- // write .ec01 ~ .ec14 files
+ // write .ec00 ~ .ec13 files
if err := erasure_coding.WriteEcFiles(baseFileName); err != nil {
return nil, fmt.Errorf("WriteEcFiles %s: %v", baseFileName, err)
}
+ // write .vif files
+ if err := pb.SaveVolumeInfo(baseFileName+".vif", &volume_server_pb.VolumeInfo{Version: uint32(v.Version())}); err != nil {
+ return nil, fmt.Errorf("WriteEcFiles %s: %v", baseFileName, err)
+ }
+
return &volume_server_pb.VolumeEcShardsGenerateResponse{}, nil
}
-// VolumeEcShardsRebuild generates the any of the missing .ec01 ~ .ec14 files
+// VolumeEcShardsRebuild generates the any of the missing .ec00 ~ .ec13 files
func (vs *VolumeServer) VolumeEcShardsRebuild(ctx context.Context, req *volume_server_pb.VolumeEcShardsRebuildRequest) (*volume_server_pb.VolumeEcShardsRebuildResponse, error) {
baseFileName := erasure_coding.EcShardBaseFileName(req.Collection, int(req.VolumeId))
@@ -68,7 +75,7 @@ func (vs *VolumeServer) VolumeEcShardsRebuild(ctx context.Context, req *volume_s
for _, location := range vs.store.Locations {
if util.FileExists(path.Join(location.Directory, baseFileName+".ecx")) {
- // write .ec01 ~ .ec14 files
+ // write .ec00 ~ .ec13 files
baseFileName = path.Join(location.Directory, baseFileName)
if generatedShardIds, err := erasure_coding.RebuildEcFiles(baseFileName); err != nil {
return nil, fmt.Errorf("RebuildEcFiles %s: %v", baseFileName, err)
@@ -103,23 +110,32 @@ func (vs *VolumeServer) VolumeEcShardsCopy(ctx context.Context, req *volume_serv
// copy ec data slices
for _, shardId := range req.ShardIds {
- if err := vs.doCopyFile(ctx, client, true, req.Collection, req.VolumeId, math.MaxUint32, math.MaxInt64, baseFileName, erasure_coding.ToExt(int(shardId)), false); err != nil {
+ if err := vs.doCopyFile(client, true, req.Collection, req.VolumeId, math.MaxUint32, math.MaxInt64, baseFileName, erasure_coding.ToExt(int(shardId)), false, false); err != nil {
return err
}
}
- if !req.CopyEcxFile {
+ if req.CopyEcxFile {
+
+ // copy ecx file
+ if err := vs.doCopyFile(client, true, req.Collection, req.VolumeId, math.MaxUint32, math.MaxInt64, baseFileName, ".ecx", false, false); err != nil {
+ return err
+ }
return nil
}
- // copy ecx file
- if err := vs.doCopyFile(ctx, client, true, req.Collection, req.VolumeId, math.MaxUint32, math.MaxInt64, baseFileName, ".ecx", false); err != nil {
- return err
+ if req.CopyEcjFile {
+ // copy ecj file
+ if err := vs.doCopyFile(client, true, req.Collection, req.VolumeId, math.MaxUint32, math.MaxInt64, baseFileName, ".ecj", true, true); err != nil {
+ return err
+ }
}
- // copy ecj file
- if err := vs.doCopyFile(ctx, client, true, req.Collection, req.VolumeId, math.MaxUint32, math.MaxInt64, baseFileName, ".ecj", true); err != nil {
- return err
+ if req.CopyVifFile {
+ // copy vif file
+ if err := vs.doCopyFile(client, true, req.Collection, req.VolumeId, math.MaxUint32, math.MaxInt64, baseFileName, ".vif", false, true); err != nil {
+ return err
+ }
}
return nil
@@ -137,6 +153,8 @@ func (vs *VolumeServer) VolumeEcShardsDelete(ctx context.Context, req *volume_se
baseFilename := erasure_coding.EcShardBaseFileName(req.Collection, int(req.VolumeId))
+ glog.V(0).Infof("ec volume %d shard delete %v", req.VolumeId, req.ShardIds)
+
found := false
for _, location := range vs.store.Locations {
if util.FileExists(path.Join(location.Directory, baseFilename+".ecx")) {
@@ -153,21 +171,27 @@ func (vs *VolumeServer) VolumeEcShardsDelete(ctx context.Context, req *volume_se
return nil, nil
}
- // check whether to delete the ecx file also
+ // check whether to delete the .ecx and .ecj file also
hasEcxFile := false
+ hasIdxFile := false
existingShardCount := 0
+ bName := filepath.Base(baseFilename)
for _, location := range vs.store.Locations {
fileInfos, err := ioutil.ReadDir(location.Directory)
if err != nil {
continue
}
for _, fileInfo := range fileInfos {
- if fileInfo.Name() == baseFilename+".ecx" {
+ if fileInfo.Name() == bName+".ecx" || fileInfo.Name() == bName+".ecj" {
hasEcxFile = true
continue
}
- if strings.HasPrefix(fileInfo.Name(), baseFilename+".ec") {
+ if fileInfo.Name() == bName+".idx" {
+ hasIdxFile = true
+ continue
+ }
+ if strings.HasPrefix(fileInfo.Name(), bName+".ec") {
existingShardCount++
}
}
@@ -181,6 +205,10 @@ func (vs *VolumeServer) VolumeEcShardsDelete(ctx context.Context, req *volume_se
return nil, err
}
}
+ if !hasIdxFile {
+ // .vif is used for ec volumes and normal volumes
+ os.Remove(baseFilename + ".vif")
+ }
return &volume_server_pb.VolumeEcShardsDeleteResponse{}, nil
}
@@ -317,3 +345,35 @@ func (vs *VolumeServer) VolumeEcBlobDelete(ctx context.Context, req *volume_serv
return resp, nil
}
+
+// VolumeEcShardsToVolume generates the .idx, .dat files from .ecx, .ecj and .ec01 ~ .ec14 files
+func (vs *VolumeServer) VolumeEcShardsToVolume(ctx context.Context, req *volume_server_pb.VolumeEcShardsToVolumeRequest) (*volume_server_pb.VolumeEcShardsToVolumeResponse, error) {
+
+ v, found := vs.store.FindEcVolume(needle.VolumeId(req.VolumeId))
+ if !found {
+ return nil, fmt.Errorf("ec volume %d not found", req.VolumeId)
+ }
+ baseFileName := v.FileName()
+
+ if v.Collection != req.Collection {
+ return nil, fmt.Errorf("existing collection:%v unexpected input: %v", v.Collection, req.Collection)
+ }
+
+ // calculate .dat file size
+ datFileSize, err := erasure_coding.FindDatFileSize(baseFileName)
+ if err != nil {
+ return nil, fmt.Errorf("FindDatFileSize %s: %v", baseFileName, err)
+ }
+
+ // write .dat file from .ec00 ~ .ec09 files
+ if err := erasure_coding.WriteDatFile(baseFileName, datFileSize); err != nil {
+ return nil, fmt.Errorf("WriteEcFiles %s: %v", baseFileName, err)
+ }
+
+ // write .idx file from .ecx and .ecj files
+ if err := erasure_coding.WriteIdxFileFromEcIndex(baseFileName); err != nil {
+ return nil, fmt.Errorf("WriteIdxFileFromEcIndex %s: %v", baseFileName, err)
+ }
+
+ return &volume_server_pb.VolumeEcShardsToVolumeResponse{}, nil
+}
diff --git a/weed/server/volume_grpc_file.go b/weed/server/volume_grpc_file.go
new file mode 100644
index 000000000..4d71ddeb1
--- /dev/null
+++ b/weed/server/volume_grpc_file.go
@@ -0,0 +1,129 @@
+package weed_server
+
+import (
+ "encoding/json"
+ "net/http"
+ "strings"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/util"
+)
+
+func (vs *VolumeServer) FileGet(req *volume_server_pb.FileGetRequest, stream volume_server_pb.VolumeServer_FileGetServer) error {
+
+ headResponse := &volume_server_pb.FileGetResponse{}
+ n := new(needle.Needle)
+
+ commaIndex := strings.LastIndex(req.FileId, ",")
+ vid := req.FileId[:commaIndex]
+ fid := req.FileId[commaIndex+1:]
+
+ volumeId, err := needle.NewVolumeId(vid)
+ if err != nil {
+ headResponse.ErrorCode = http.StatusBadRequest
+ return stream.Send(headResponse)
+ }
+ err = n.ParsePath(fid)
+ if err != nil {
+ headResponse.ErrorCode = http.StatusBadRequest
+ return stream.Send(headResponse)
+ }
+
+ hasVolume := vs.store.HasVolume(volumeId)
+ _, hasEcVolume := vs.store.FindEcVolume(volumeId)
+
+ if !hasVolume && !hasEcVolume {
+ headResponse.ErrorCode = http.StatusMovedPermanently
+ return stream.Send(headResponse)
+ }
+
+ cookie := n.Cookie
+ var count int
+ if hasVolume {
+ count, err = vs.store.ReadVolumeNeedle(volumeId, n)
+ } else if hasEcVolume {
+ count, err = vs.store.ReadEcShardNeedle(volumeId, n)
+ }
+
+ if err != nil || count < 0 {
+ headResponse.ErrorCode = http.StatusNotFound
+ return stream.Send(headResponse)
+ }
+ if n.Cookie != cookie {
+ headResponse.ErrorCode = http.StatusNotFound
+ return stream.Send(headResponse)
+ }
+
+ if n.LastModified != 0 {
+ headResponse.LastModified = n.LastModified
+ }
+
+ headResponse.Etag = n.Etag()
+
+ if n.HasPairs() {
+ pairMap := make(map[string]string)
+ err = json.Unmarshal(n.Pairs, &pairMap)
+ if err != nil {
+ glog.V(0).Infoln("Unmarshal pairs error:", err)
+ }
+ headResponse.Headers = pairMap
+ }
+
+ /*
+ // skip this, no redirection
+ if vs.tryHandleChunkedFile(n, filename, w, r) {
+ return
+ }
+ */
+
+ if n.NameSize > 0 {
+ headResponse.Filename = string(n.Name)
+ }
+ mtype := ""
+ if n.MimeSize > 0 {
+ mt := string(n.Mime)
+ if !strings.HasPrefix(mt, "application/octet-stream") {
+ mtype = mt
+ }
+ }
+ headResponse.ContentType = mtype
+
+ headResponse.IsGzipped = n.IsGzipped()
+
+ if n.IsGzipped() && req.AcceptGzip {
+ if n.Data, err = util.UnGzipData(n.Data); err != nil {
+ glog.V(0).Infof("ungzip %s error: %v", req.FileId, err)
+ }
+ }
+
+ headResponse.ContentLength = uint32(len(n.Data))
+ bytesToRead := len(n.Data)
+ bytesRead := 0
+
+ t := headResponse
+
+ for bytesRead < bytesToRead {
+
+ stopIndex := bytesRead + BufferSizeLimit
+ if stopIndex > bytesToRead {
+ stopIndex = bytesToRead
+ }
+
+ if t == nil {
+ t = &volume_server_pb.FileGetResponse{}
+ }
+ t.Data = n.Data[bytesRead:stopIndex]
+
+ err = stream.Send(t)
+ t = nil
+ if err != nil {
+ return err
+ }
+
+ bytesRead = stopIndex
+ }
+
+ return nil
+}
diff --git a/weed/server/volume_grpc_tail.go b/weed/server/volume_grpc_tail.go
index 1bf61e1c7..c26d6ed8f 100644
--- a/weed/server/volume_grpc_tail.go
+++ b/weed/server/volume_grpc_tail.go
@@ -10,6 +10,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
)
func (vs *VolumeServer) VolumeTailSender(req *volume_server_pb.VolumeTailSenderRequest, stream volume_server_pb.VolumeServer_VolumeTailSenderServer) error {
@@ -101,7 +102,7 @@ type VolumeFileScanner4Tailing struct {
lastProcessedTimestampNs uint64
}
-func (scanner *VolumeFileScanner4Tailing) VisitSuperBlock(superBlock storage.SuperBlock) error {
+func (scanner *VolumeFileScanner4Tailing) VisitSuperBlock(superBlock super_block.SuperBlock) error {
return nil
}
diff --git a/weed/server/volume_grpc_tier_download.go b/weed/server/volume_grpc_tier_download.go
new file mode 100644
index 000000000..7b3982e40
--- /dev/null
+++ b/weed/server/volume_grpc_tier_download.go
@@ -0,0 +1,85 @@
+package weed_server
+
+import (
+ "fmt"
+ "time"
+
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+)
+
+// VolumeTierMoveDatFromRemote copy dat file from a remote tier to local volume server
+func (vs *VolumeServer) VolumeTierMoveDatFromRemote(req *volume_server_pb.VolumeTierMoveDatFromRemoteRequest, stream volume_server_pb.VolumeServer_VolumeTierMoveDatFromRemoteServer) error {
+
+ // find existing volume
+ v := vs.store.GetVolume(needle.VolumeId(req.VolumeId))
+ if v == nil {
+ return fmt.Errorf("volume %d not found", req.VolumeId)
+ }
+
+ // verify the collection
+ if v.Collection != req.Collection {
+ return fmt.Errorf("existing collection:%v unexpected input: %v", v.Collection, req.Collection)
+ }
+
+ // locate the disk file
+ storageName, storageKey := v.RemoteStorageNameKey()
+ if storageName == "" || storageKey == "" {
+ return fmt.Errorf("volume %d is already on local disk", req.VolumeId)
+ }
+
+ // check whether the local .dat already exists
+ _, ok := v.DataBackend.(*backend.DiskFile)
+ if ok {
+ return fmt.Errorf("volume %d is already on local disk", req.VolumeId)
+ }
+
+ // check valid storage backend type
+ backendStorage, found := backend.BackendStorages[storageName]
+ if !found {
+ var keys []string
+ for key := range backend.BackendStorages {
+ keys = append(keys, key)
+ }
+ return fmt.Errorf("remote storage %s not found from suppported: %v", storageName, keys)
+ }
+
+ startTime := time.Now()
+ fn := func(progressed int64, percentage float32) error {
+ now := time.Now()
+ if now.Sub(startTime) < time.Second {
+ return nil
+ }
+ startTime = now
+ return stream.Send(&volume_server_pb.VolumeTierMoveDatFromRemoteResponse{
+ Processed: progressed,
+ ProcessedPercentage: percentage,
+ })
+ }
+ // copy the data file
+ _, err := backendStorage.DownloadFile(v.FileName()+".dat", storageKey, fn)
+ if err != nil {
+ return fmt.Errorf("backend %s copy file %s: %v", storageName, v.FileName()+".dat", err)
+ }
+
+ if req.KeepRemoteDatFile {
+ return nil
+ }
+
+ // remove remote file
+ if err := backendStorage.DeleteFile(storageKey); err != nil {
+ return fmt.Errorf("volume %d fail to delete remote file %s: %v", v.Id, storageKey, err)
+ }
+
+ // forget remote file
+ v.GetVolumeInfo().Files = v.GetVolumeInfo().Files[1:]
+ if err := v.SaveVolumeInfo(); err != nil {
+ return fmt.Errorf("volume %d fail to save remote file info: %v", v.Id, err)
+ }
+
+ v.DataBackend.Close()
+ v.DataBackend = nil
+
+ return nil
+}
diff --git a/weed/server/volume_grpc_tier_upload.go b/weed/server/volume_grpc_tier_upload.go
new file mode 100644
index 000000000..c9694df59
--- /dev/null
+++ b/weed/server/volume_grpc_tier_upload.go
@@ -0,0 +1,100 @@
+package weed_server
+
+import (
+ "fmt"
+ "os"
+ "time"
+
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+)
+
+// VolumeTierMoveDatToRemote copy dat file to a remote tier
+func (vs *VolumeServer) VolumeTierMoveDatToRemote(req *volume_server_pb.VolumeTierMoveDatToRemoteRequest, stream volume_server_pb.VolumeServer_VolumeTierMoveDatToRemoteServer) error {
+
+ // find existing volume
+ v := vs.store.GetVolume(needle.VolumeId(req.VolumeId))
+ if v == nil {
+ return fmt.Errorf("volume %d not found", req.VolumeId)
+ }
+
+ // verify the collection
+ if v.Collection != req.Collection {
+ return fmt.Errorf("existing collection:%v unexpected input: %v", v.Collection, req.Collection)
+ }
+
+ // locate the disk file
+ diskFile, ok := v.DataBackend.(*backend.DiskFile)
+ if !ok {
+ return fmt.Errorf("volume %d is not on local disk", req.VolumeId)
+ }
+
+ // check valid storage backend type
+ backendStorage, found := backend.BackendStorages[req.DestinationBackendName]
+ if !found {
+ var keys []string
+ for key := range backend.BackendStorages {
+ keys = append(keys, key)
+ }
+ return fmt.Errorf("destination %s not found, suppported: %v", req.DestinationBackendName, keys)
+ }
+
+ // check whether the existing backend storage is the same as requested
+ // if same, skip
+ backendType, backendId := backend.BackendNameToTypeId(req.DestinationBackendName)
+ for _, remoteFile := range v.GetVolumeInfo().GetFiles() {
+ if remoteFile.BackendType == backendType && remoteFile.BackendId == backendId {
+ return fmt.Errorf("destination %s already exists", req.DestinationBackendName)
+ }
+ }
+
+ startTime := time.Now()
+ fn := func(progressed int64, percentage float32) error {
+ now := time.Now()
+ if now.Sub(startTime) < time.Second {
+ return nil
+ }
+ startTime = now
+ return stream.Send(&volume_server_pb.VolumeTierMoveDatToRemoteResponse{
+ Processed: progressed,
+ ProcessedPercentage: percentage,
+ })
+ }
+
+ // remember the file original source
+ attributes := make(map[string]string)
+ attributes["volumeId"] = v.Id.String()
+ attributes["collection"] = v.Collection
+ attributes["ext"] = ".dat"
+ // copy the data file
+ key, size, err := backendStorage.CopyFile(diskFile.File, attributes, fn)
+ if err != nil {
+ return fmt.Errorf("backend %s copy file %s: %v", req.DestinationBackendName, diskFile.Name(), err)
+ }
+
+ // save the remote file to volume tier info
+ v.GetVolumeInfo().Files = append(v.GetVolumeInfo().GetFiles(), &volume_server_pb.RemoteFile{
+ BackendType: backendType,
+ BackendId: backendId,
+ Key: key,
+ Offset: 0,
+ FileSize: uint64(size),
+ ModifiedTime: uint64(time.Now().Unix()),
+ Extension: ".dat",
+ })
+
+ if err := v.SaveVolumeInfo(); err != nil {
+ return fmt.Errorf("volume %d fail to save remote file info: %v", v.Id, err)
+ }
+
+ if err := v.LoadRemoteFile(); err != nil {
+ return fmt.Errorf("volume %d fail to load remote file: %v", v.Id, err)
+ }
+
+ if !req.KeepLocalDatFile {
+ os.Remove(v.FileName() + ".dat")
+ }
+
+ return nil
+}
diff --git a/weed/server/volume_server.go b/weed/server/volume_server.go
index 6cf654738..2d716edc1 100644
--- a/weed/server/volume_server.go
+++ b/weed/server/volume_server.go
@@ -4,13 +4,14 @@ import (
"fmt"
"net/http"
- "github.com/chrislusf/seaweedfs/weed/stats"
"google.golang.org/grpc"
+ "github.com/chrislusf/seaweedfs/weed/stats"
+ "github.com/chrislusf/seaweedfs/weed/util"
+
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/security"
"github.com/chrislusf/seaweedfs/weed/storage"
- "github.com/spf13/viper"
)
type VolumeServer struct {
@@ -29,6 +30,7 @@ type VolumeServer struct {
compactionBytePerSecond int64
MetricsAddress string
MetricsIntervalSec int
+ fileSizeLimitBytes int64
}
func NewVolumeServer(adminMux, publicMux *http.ServeMux, ip string,
@@ -41,9 +43,10 @@ func NewVolumeServer(adminMux, publicMux *http.ServeMux, ip string,
fixJpgOrientation bool,
readRedirect bool,
compactionMBPerSecond int,
+ fileSizeLimitMB int,
) *VolumeServer {
- v := viper.GetViper()
+ v := util.GetViper()
signingKey := v.GetString("jwt.signing.key")
v.SetDefault("jwt.signing.expires_after_seconds", 10)
expiresAfterSec := v.GetInt("jwt.signing.expires_after_seconds")
@@ -60,8 +63,9 @@ func NewVolumeServer(adminMux, publicMux *http.ServeMux, ip string,
needleMapKind: needleMapKind,
FixJpgOrientation: fixJpgOrientation,
ReadRedirect: readRedirect,
- grpcDialOption: security.LoadClientTLS(viper.Sub("grpc"), "volume"),
+ grpcDialOption: security.LoadClientTLS(util.GetViper(), "grpc.volume"),
compactionBytePerSecond: int64(compactionMBPerSecond) * 1024 * 1024,
+ fileSizeLimitBytes: int64(fileSizeLimitMB) * 1024 * 1024,
}
vs.SeedMasterNodes = masterNodes
vs.store = storage.NewStore(vs.grpcDialOption, port, ip, publicUrl, folders, maxCounts, vs.needleMapKind)
@@ -73,9 +77,11 @@ func NewVolumeServer(adminMux, publicMux *http.ServeMux, ip string,
// only expose the volume server details for safe environments
adminMux.HandleFunc("/ui/index.html", vs.uiStatusHandler)
adminMux.HandleFunc("/status", vs.guard.WhiteList(vs.statusHandler))
- adminMux.HandleFunc("/stats/counter", vs.guard.WhiteList(statsCounterHandler))
- adminMux.HandleFunc("/stats/memory", vs.guard.WhiteList(statsMemoryHandler))
- adminMux.HandleFunc("/stats/disk", vs.guard.WhiteList(vs.statsDiskHandler))
+ /*
+ adminMux.HandleFunc("/stats/counter", vs.guard.WhiteList(statsCounterHandler))
+ adminMux.HandleFunc("/stats/memory", vs.guard.WhiteList(statsMemoryHandler))
+ adminMux.HandleFunc("/stats/disk", vs.guard.WhiteList(vs.statsDiskHandler))
+ */
}
adminMux.HandleFunc("/", vs.privateStoreHandler)
if publicMux != adminMux {
diff --git a/weed/server/volume_server_handlers_admin.go b/weed/server/volume_server_handlers_admin.go
index 25b6582f7..89bc051c5 100644
--- a/weed/server/volume_server_handlers_admin.go
+++ b/weed/server/volume_server_handlers_admin.go
@@ -12,7 +12,14 @@ import (
func (vs *VolumeServer) statusHandler(w http.ResponseWriter, r *http.Request) {
m := make(map[string]interface{})
m["Version"] = util.VERSION
- m["Volumes"] = vs.store.Status()
+ var ds []*volume_server_pb.DiskStatus
+ for _, loc := range vs.store.Locations {
+ if dir, e := filepath.Abs(loc.Directory); e == nil {
+ ds = append(ds, stats.NewDiskStatus(dir))
+ }
+ }
+ m["DiskStatuses"] = ds
+ m["Volumes"] = vs.store.VolumeInfos()
writeJsonQuiet(w, r, http.StatusOK, m)
}
diff --git a/weed/server/volume_server_handlers_read.go b/weed/server/volume_server_handlers_read.go
index cd11356b9..6e603d158 100644
--- a/weed/server/volume_server_handlers_read.go
+++ b/weed/server/volume_server_handlers_read.go
@@ -2,21 +2,18 @@ package weed_server
import (
"bytes"
- "context"
+ "encoding/json"
"errors"
"fmt"
"io"
"mime"
- "mime/multipart"
"net/http"
"net/url"
- "path"
+ "path/filepath"
"strconv"
"strings"
"time"
- "encoding/json"
-
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/images"
"github.com/chrislusf/seaweedfs/weed/operation"
@@ -54,7 +51,7 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
return
}
- glog.V(4).Infoln("volume", volumeId, "reading", n)
+ // glog.V(4).Infoln("volume", volumeId, "reading", n)
hasVolume := vs.store.HasVolume(volumeId)
_, hasEcVolume := vs.store.FindEcVolume(volumeId)
if !hasVolume && !hasEcVolume {
@@ -86,9 +83,9 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
if hasVolume {
count, err = vs.store.ReadVolumeNeedle(volumeId, n)
} else if hasEcVolume {
- count, err = vs.store.ReadEcShardNeedle(context.Background(), volumeId, n)
+ count, err = vs.store.ReadEcShardNeedle(volumeId, n)
}
- glog.V(4).Infoln("read bytes", count, "error", err)
+ // glog.V(4).Infoln("read bytes", count, "error", err)
if err != nil || count < 0 {
glog.V(0).Infof("read %s isNormalVolume %v error: %v", r.URL.Path, hasVolume, err)
w.WriteHeader(http.StatusNotFound)
@@ -114,11 +111,7 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
w.WriteHeader(http.StatusNotModified)
return
}
- if r.Header.Get("ETag-MD5") == "True" {
- setEtag(w, n.MD5())
- } else {
- setEtag(w, n.Etag())
- }
+ setEtag(w, n.Etag())
if n.HasPairs() {
pairMap := make(map[string]string)
@@ -138,7 +131,7 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
if n.NameSize > 0 && filename == "" {
filename = string(n.Name)
if ext == "" {
- ext = path.Ext(filename)
+ ext = filepath.Ext(filename)
}
}
mtype := ""
@@ -182,7 +175,7 @@ func (vs *VolumeServer) tryHandleChunkedFile(n *needle.Needle, fileName string,
fileName = chunkManifest.Name
}
- ext := path.Ext(fileName)
+ ext := filepath.Ext(fileName)
mType := ""
if chunkManifest.Mime != "" {
@@ -229,113 +222,28 @@ func conditionallyResizeImages(originalDataReaderSeeker io.ReadSeeker, ext strin
func writeResponseContent(filename, mimeType string, rs io.ReadSeeker, w http.ResponseWriter, r *http.Request) error {
totalSize, e := rs.Seek(0, 2)
if mimeType == "" {
- if ext := path.Ext(filename); ext != "" {
+ if ext := filepath.Ext(filename); ext != "" {
mimeType = mime.TypeByExtension(ext)
}
}
if mimeType != "" {
w.Header().Set("Content-Type", mimeType)
}
- if filename != "" {
- contentDisposition := "inline"
- if r.FormValue("dl") != "" {
- if dl, _ := strconv.ParseBool(r.FormValue("dl")); dl {
- contentDisposition = "attachment"
- }
- }
- w.Header().Set("Content-Disposition", contentDisposition+`; filename="`+fileNameEscaper.Replace(filename)+`"`)
- }
w.Header().Set("Accept-Ranges", "bytes")
+
if r.Method == "HEAD" {
w.Header().Set("Content-Length", strconv.FormatInt(totalSize, 10))
return nil
}
- rangeReq := r.Header.Get("Range")
- if rangeReq == "" {
- w.Header().Set("Content-Length", strconv.FormatInt(totalSize, 10))
- if _, e = rs.Seek(0, 0); e != nil {
- return e
- }
- _, e = io.Copy(w, rs)
- return e
- }
- //the rest is dealing with partial content request
- //mostly copy from src/pkg/net/http/fs.go
- ranges, err := parseRange(rangeReq, totalSize)
- if err != nil {
- http.Error(w, err.Error(), http.StatusRequestedRangeNotSatisfiable)
- return nil
- }
- if sumRangesSize(ranges) > totalSize {
- // The total number of bytes in all the ranges
- // is larger than the size of the file by
- // itself, so this is probably an attack, or a
- // dumb client. Ignore the range request.
- return nil
- }
- if len(ranges) == 0 {
- return nil
- }
- if len(ranges) == 1 {
- // RFC 2616, Section 14.16:
- // "When an HTTP message includes the content of a single
- // range (for example, a response to a request for a
- // single range, or to a request for a set of ranges
- // that overlap without any holes), this content is
- // transmitted with a Content-Range header, and a
- // Content-Length header showing the number of bytes
- // actually transferred.
- // ...
- // A response to a request for a single range MUST NOT
- // be sent using the multipart/byteranges media type."
- ra := ranges[0]
- w.Header().Set("Content-Length", strconv.FormatInt(ra.length, 10))
- w.Header().Set("Content-Range", ra.contentRange(totalSize))
- w.WriteHeader(http.StatusPartialContent)
- if _, e = rs.Seek(ra.start, 0); e != nil {
+ adjustHeadersAfterHEAD(w, r, filename)
+
+ processRangeRequst(r, w, totalSize, mimeType, func(writer io.Writer, offset int64, size int64) error {
+ if _, e = rs.Seek(offset, 0); e != nil {
return e
}
-
- _, e = io.CopyN(w, rs, ra.length)
+ _, e = io.CopyN(writer, rs, size)
return e
- }
- // process multiple ranges
- for _, ra := range ranges {
- if ra.start > totalSize {
- http.Error(w, "Out of Range", http.StatusRequestedRangeNotSatisfiable)
- return nil
- }
- }
- sendSize := rangesMIMESize(ranges, mimeType, totalSize)
- pr, pw := io.Pipe()
- mw := multipart.NewWriter(pw)
- w.Header().Set("Content-Type", "multipart/byteranges; boundary="+mw.Boundary())
- sendContent := pr
- defer pr.Close() // cause writing goroutine to fail and exit if CopyN doesn't finish.
- go func() {
- for _, ra := range ranges {
- part, e := mw.CreatePart(ra.mimeHeader(mimeType, totalSize))
- if e != nil {
- pw.CloseWithError(e)
- return
- }
- if _, e = rs.Seek(ra.start, 0); e != nil {
- pw.CloseWithError(e)
- return
- }
- if _, e = io.CopyN(part, rs, ra.length); e != nil {
- pw.CloseWithError(e)
- return
- }
- }
- mw.Close()
- pw.Close()
- }()
- if w.Header().Get("Content-Encoding") == "" {
- w.Header().Set("Content-Length", strconv.FormatInt(sendSize, 10))
- }
- w.WriteHeader(http.StatusPartialContent)
- _, e = io.CopyN(w, sendContent, sendSize)
- return e
+ })
+ return nil
}
diff --git a/weed/server/volume_server_handlers_ui.go b/weed/server/volume_server_handlers_ui.go
index 852f0b751..8d35c9c8b 100644
--- a/weed/server/volume_server_handlers_ui.go
+++ b/weed/server/volume_server_handlers_ui.go
@@ -8,6 +8,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
ui "github.com/chrislusf/seaweedfs/weed/server/volume_server_ui"
"github.com/chrislusf/seaweedfs/weed/stats"
+ "github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/util"
)
@@ -20,19 +21,30 @@ func (vs *VolumeServer) uiStatusHandler(w http.ResponseWriter, r *http.Request)
ds = append(ds, stats.NewDiskStatus(dir))
}
}
+ volumeInfos := vs.store.VolumeInfos()
+ var normalVolumeInfos, remoteVolumeInfos []*storage.VolumeInfo
+ for _, vinfo := range volumeInfos {
+ if vinfo.IsRemote() {
+ remoteVolumeInfos = append(remoteVolumeInfos, vinfo)
+ } else {
+ normalVolumeInfos = append(normalVolumeInfos, vinfo)
+ }
+ }
args := struct {
- Version string
- Masters []string
- Volumes interface{}
- EcVolumes interface{}
- DiskStatuses interface{}
- Stats interface{}
- Counters *stats.ServerStats
+ Version string
+ Masters []string
+ Volumes interface{}
+ EcVolumes interface{}
+ RemoteVolumes interface{}
+ DiskStatuses interface{}
+ Stats interface{}
+ Counters *stats.ServerStats
}{
util.VERSION,
vs.SeedMasterNodes,
- vs.store.Status(),
+ normalVolumeInfos,
vs.store.EcVolumes(),
+ remoteVolumeInfos,
ds,
infos,
serverStats,
diff --git a/weed/server/volume_server_handlers_write.go b/weed/server/volume_server_handlers_write.go
index db8fcb555..101be4c43 100644
--- a/weed/server/volume_server_handlers_write.go
+++ b/weed/server/volume_server_handlers_write.go
@@ -1,7 +1,6 @@
package weed_server
import (
- "context"
"errors"
"fmt"
"net/http"
@@ -43,7 +42,7 @@ func (vs *VolumeServer) PostHandler(w http.ResponseWriter, r *http.Request) {
return
}
- needle, originalSize, ne := needle.CreateNeedleFromRequest(r, vs.FixJpgOrientation)
+ needle, originalSize, ne := needle.CreateNeedleFromRequest(r, vs.FixJpgOrientation, vs.fileSizeLimitBytes)
if ne != nil {
writeJsonError(w, r, http.StatusBadRequest, ne)
return
@@ -51,10 +50,15 @@ func (vs *VolumeServer) PostHandler(w http.ResponseWriter, r *http.Request) {
ret := operation.UploadResult{}
_, isUnchanged, writeError := topology.ReplicatedWrite(vs.GetMaster(), vs.store, volumeId, needle, r)
- httpStatus := http.StatusCreated
- if isUnchanged {
- httpStatus = http.StatusNotModified
+
+ // http 204 status code does not allow body
+ if writeError == nil && isUnchanged {
+ setEtag(w, needle.Etag())
+ w.WriteHeader(http.StatusNoContent)
+ return
}
+
+ httpStatus := http.StatusCreated
if writeError != nil {
httpStatus = http.StatusInternalServerError
ret.Error = writeError.Error()
@@ -64,6 +68,7 @@ func (vs *VolumeServer) PostHandler(w http.ResponseWriter, r *http.Request) {
}
ret.Size = uint32(originalSize)
ret.ETag = needle.Etag()
+ ret.Mime = string(needle.Mime)
setEtag(w, ret.ETag)
writeJsonQuiet(w, r, httpStatus, ret)
}
@@ -93,7 +98,7 @@ func (vs *VolumeServer) DeleteHandler(w http.ResponseWriter, r *http.Request) {
ecVolume, hasEcVolume := vs.store.FindEcVolume(volumeId)
if hasEcVolume {
- count, err := vs.store.DeleteEcShardNeedle(context.Background(), ecVolume, n, cookie)
+ count, err := vs.store.DeleteEcShardNeedle(ecVolume, n, cookie)
writeDeleteResult(err, count, w, r)
return
}
diff --git a/weed/server/volume_server_ui/templates.go b/weed/server/volume_server_ui/templates.go
index eafc0aaeb..1c1394369 100644
--- a/weed/server/volume_server_ui/templates.go
+++ b/weed/server/volume_server_ui/templates.go
@@ -1,11 +1,29 @@
package master_ui
import (
+ "fmt"
"html/template"
"strconv"
"strings"
)
+func bytesToHumanReadble(b uint64) string {
+ const unit = 1024
+ if b < unit {
+ return fmt.Sprintf("%d B", b)
+ }
+ div, exp := uint64(unit), 0
+ for n := b / unit; n >= unit; n /= unit {
+ div *= unit
+ exp++
+ }
+ return fmt.Sprintf("%.2f %ciB", float64(b)/float64(div), "KMGTPE"[exp])
+}
+
+func percentFrom(total uint64, part_of uint64) string {
+ return fmt.Sprintf("%.2f", (float64(part_of)/float64(total))*100)
+}
+
func join(data []int64) string {
var ret []string
for _, d := range data {
@@ -15,7 +33,9 @@ func join(data []int64) string {
}
var funcMap = template.FuncMap{
- "join": join,
+ "join": join,
+ "bytesToHumanReadble": bytesToHumanReadble,
+ "percentFrom": percentFrom,
}
var StatusTpl = template.Must(template.New("status").Funcs(funcMap).Parse(`
@@ -57,13 +77,25 @@ var StatusTpl = template.Must(template.New("status").Funcs(funcMap).Parse(`
Disk Stats
-
+
+
+
+ Path |
+ Total |
+ Free |
+ Usage |
+
+
+
{{ range .DiskStatuses }}
- {{ .Dir }} |
- {{ .Free }} Bytes Free |
+ {{ .Dir }} |
+ {{ bytesToHumanReadble .All }} |
+ {{ bytesToHumanReadble .Free }} |
+ {{ percentFrom .All .Used}}% |
{{ end }}
+
@@ -107,10 +139,11 @@ var StatusTpl = template.Must(template.New("status").Funcs(funcMap).Parse(`
Id |
Collection |
- Size |
+ Data Size |
Files |
Trash |
TTL |
+ ReadOnly |
@@ -122,6 +155,37 @@ var StatusTpl = template.Must(template.New("status").Funcs(funcMap).Parse(`{{ .FileCount }}
{{ .DeleteCount }} / {{.DeletedByteCount}} Bytes |
{{ .Ttl }} |
+ {{ .ReadOnly }} |
+
+ {{ end }}
+
+
+
+
+
+
Remote Volumes
+
+
+
+ Id |
+ Collection |
+ Size |
+ Files |
+ Trash |
+ Remote |
+ Key |
+
+
+
+ {{ range .RemoteVolumes }}
+
+ {{ .Id }} |
+ {{ .Collection }} |
+ {{ .Size }} Bytes |
+ {{ .FileCount }} |
+ {{ .DeleteCount }} / {{.DeletedByteCount}} Bytes |
+ {{ .RemoteStorageName }} |
+ {{ .RemoteStorageKey }} |
{{ end }}
diff --git a/weed/server/webdav_server.go b/weed/server/webdav_server.go
index 151b48a78..1fb0912c5 100644
--- a/weed/server/webdav_server.go
+++ b/weed/server/webdav_server.go
@@ -1,7 +1,6 @@
package weed_server
import (
- "bytes"
"context"
"fmt"
"io"
@@ -10,16 +9,17 @@ import (
"strings"
"time"
+ "golang.org/x/net/webdav"
+ "google.golang.org/grpc"
+
"github.com/chrislusf/seaweedfs/weed/operation"
+ "github.com/chrislusf/seaweedfs/weed/pb"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/util"
- "golang.org/x/net/webdav"
- "google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/filer2"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/security"
- "github.com/spf13/viper"
)
type WebDavOption struct {
@@ -31,6 +31,7 @@ type WebDavOption struct {
Collection string
Uid uint32
Gid uint32
+ Cipher bool
}
type WebDavServer struct {
@@ -47,7 +48,7 @@ func NewWebDavServer(option *WebDavOption) (ws *WebDavServer, err error) {
ws = &WebDavServer{
option: option,
- grpcDialOption: security.LoadClientTLS(viper.Sub("grpc"), "filer"),
+ grpcDialOption: security.LoadClientTLS(util.GetViper(), "grpc.filer"),
Handler: &webdav.Handler{
FileSystem: fs,
LockSystem: webdav.NewMemLS(),
@@ -96,14 +97,17 @@ func NewWebDavFileSystem(option *WebDavOption) (webdav.FileSystem, error) {
}, nil
}
-func (fs *WebDavFileSystem) WithFilerClient(ctx context.Context, fn func(filer_pb.SeaweedFilerClient) error) error {
+func (fs *WebDavFileSystem) WithFilerClient(fn func(filer_pb.SeaweedFilerClient) error) error {
- return util.WithCachedGrpcClient(ctx, func(grpcConnection *grpc.ClientConn) error {
+ return pb.WithCachedGrpcClient(func(grpcConnection *grpc.ClientConn) error {
client := filer_pb.NewSeaweedFilerClient(grpcConnection)
return fn(client)
}, fs.option.FilerGrpcAddress, fs.option.GrpcDialOption)
}
+func (fs *WebDavFileSystem) AdjustedUrl(hostAndPort string) string {
+ return hostAndPort
+}
func clearName(name string) (string, error) {
slashed := strings.HasSuffix(name, "/")
@@ -135,7 +139,7 @@ func (fs *WebDavFileSystem) Mkdir(ctx context.Context, fullDirPath string, perm
return os.ErrExist
}
- return fs.WithFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
+ return fs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
dir, name := filer2.FullPath(fullDirPath).DirAndName()
request := &filer_pb.CreateEntryRequest{
Directory: dir,
@@ -153,7 +157,7 @@ func (fs *WebDavFileSystem) Mkdir(ctx context.Context, fullDirPath string, perm
}
glog.V(1).Infof("mkdir: %v", request)
- if _, err := client.CreateEntry(ctx, request); err != nil {
+ if err := filer_pb.CreateEntry(client, request); err != nil {
return fmt.Errorf("mkdir %s/%s: %v", dir, name, err)
}
@@ -163,7 +167,7 @@ func (fs *WebDavFileSystem) Mkdir(ctx context.Context, fullDirPath string, perm
func (fs *WebDavFileSystem) OpenFile(ctx context.Context, fullFilePath string, flag int, perm os.FileMode) (webdav.File, error) {
- glog.V(2).Infof("WebDavFileSystem.OpenFile %v", fullFilePath)
+ glog.V(2).Infof("WebDavFileSystem.OpenFile %v %x", fullFilePath, flag)
var err error
if fullFilePath, err = clearName(fullFilePath); err != nil {
@@ -175,12 +179,6 @@ func (fs *WebDavFileSystem) OpenFile(ctx context.Context, fullFilePath string, f
if strings.HasSuffix(fullFilePath, "/") {
return nil, os.ErrInvalid
}
- // based directory should be exists.
- dir, _ := path.Split(fullFilePath)
- _, err := fs.stat(ctx, dir)
- if err != nil {
- return nil, os.ErrInvalid
- }
_, err = fs.stat(ctx, fullFilePath)
if err == nil {
if flag&os.O_EXCL != 0 {
@@ -190,8 +188,8 @@ func (fs *WebDavFileSystem) OpenFile(ctx context.Context, fullFilePath string, f
}
dir, name := filer2.FullPath(fullFilePath).DirAndName()
- err = fs.WithFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
- if _, err := client.CreateEntry(ctx, &filer_pb.CreateEntryRequest{
+ err = fs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
+ if err := filer_pb.CreateEntry(client, &filer_pb.CreateEntryRequest{
Directory: dir,
Entry: &filer_pb.Entry{
Name: name,
@@ -255,7 +253,7 @@ func (fs *WebDavFileSystem) removeAll(ctx context.Context, fullFilePath string)
//_, err = fs.db.Exec(`delete from filesystem where fullFilePath = ?`, fullFilePath)
}
dir, name := filer2.FullPath(fullFilePath).DirAndName()
- err = fs.WithFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
+ err = fs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.DeleteEntryRequest{
Directory: dir,
@@ -314,7 +312,7 @@ func (fs *WebDavFileSystem) Rename(ctx context.Context, oldName, newName string)
oldDir, oldBaseName := filer2.FullPath(oldName).DirAndName()
newDir, newBaseName := filer2.FullPath(newName).DirAndName()
- return fs.WithFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
+ return fs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.AtomicRenameEntryRequest{
OldDirectory: oldDir,
@@ -339,8 +337,10 @@ func (fs *WebDavFileSystem) stat(ctx context.Context, fullFilePath string) (os.F
return nil, err
}
+ fullpath := filer2.FullPath(fullFilePath)
+
var fi FileInfo
- entry, err := filer2.GetEntry(ctx, fs, fullFilePath)
+ entry, err := filer2.GetEntry(fs, fullpath)
if entry == nil {
return nil, os.ErrNotExist
}
@@ -348,14 +348,12 @@ func (fs *WebDavFileSystem) stat(ctx context.Context, fullFilePath string) (os.F
return nil, err
}
fi.size = int64(filer2.TotalSize(entry.GetChunks()))
- fi.name = fullFilePath
+ fi.name = string(fullpath)
fi.mode = os.FileMode(entry.Attributes.FileMode)
fi.modifiledTime = time.Unix(entry.Attributes.Mtime, 0)
fi.isDirectory = entry.IsDirectory
- _, fi.name = path.Split(path.Clean(fi.name))
- if fi.name == "" {
- fi.name = "/"
+ if fi.name == "/" {
fi.modifiledTime = time.Now()
fi.isDirectory = true
}
@@ -373,10 +371,12 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
glog.V(2).Infof("WebDavFileSystem.Write %v", f.name)
+ dir, _ := filer2.FullPath(f.name).DirAndName()
+
var err error
ctx := context.Background()
if f.entry == nil {
- f.entry, err = filer2.GetEntry(ctx, f.fs, f.name)
+ f.entry, err = filer2.GetEntry(f.fs, filer2.FullPath(f.name))
}
if f.entry == nil {
@@ -388,13 +388,15 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
var fileId, host string
var auth security.EncodedJwt
+ var collection, replication string
- if err = f.fs.WithFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
+ if err = f.fs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.AssignVolumeRequest{
Count: 1,
- Replication: "000",
+ Replication: "",
Collection: f.fs.option.Collection,
+ ParentPath: dir,
}
resp, err := client.AssignVolume(ctx, request)
@@ -402,8 +404,12 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
glog.V(0).Infof("assign volume failure %v: %v", request, err)
return err
}
+ if resp.Error != "" {
+ return fmt.Errorf("assign volume failure %v: %v", request, resp.Error)
+ }
fileId, host, auth = resp.FileId, resp.Url, security.EncodedJwt(resp.Auth)
+ collection, replication = resp.Collection, resp.Replication
return nil
}); err != nil {
@@ -411,8 +417,7 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
}
fileUrl := fmt.Sprintf("http://%s/%s", host, fileId)
- bufReader := bytes.NewReader(buf)
- uploadResult, err := operation.Upload(fileUrl, f.name, bufReader, false, "application/octet-stream", nil, auth)
+ uploadResult, err := operation.UploadData(fileUrl, f.name, f.fs.option.Cipher, buf, false, "", nil, auth)
if err != nil {
glog.V(0).Infof("upload data %v to %s: %v", f.name, fileUrl, err)
return 0, fmt.Errorf("upload data: %v", err)
@@ -423,18 +428,21 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
}
chunk := &filer_pb.FileChunk{
- FileId: fileId,
- Offset: f.off,
- Size: uint64(len(buf)),
- Mtime: time.Now().UnixNano(),
- ETag: uploadResult.ETag,
+ FileId: fileId,
+ Offset: f.off,
+ Size: uint64(len(buf)),
+ Mtime: time.Now().UnixNano(),
+ ETag: uploadResult.ETag,
+ CipherKey: uploadResult.CipherKey,
+ IsGzipped: uploadResult.Gzip > 0,
}
f.entry.Chunks = append(f.entry.Chunks, chunk)
- dir, _ := filer2.FullPath(f.name).DirAndName()
- err = f.fs.WithFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
+ err = f.fs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
f.entry.Attributes.Mtime = time.Now().Unix()
+ f.entry.Attributes.Collection = collection
+ f.entry.Attributes.Replication = replication
request := &filer_pb.UpdateEntryRequest{
Directory: dir,
@@ -448,9 +456,11 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
return nil
})
- if err != nil {
+ if err == nil {
+ glog.V(3).Infof("WebDavFileSystem.Write %v: written [%d,%d)", f.name, f.off, f.off+int64(len(buf)))
f.off += int64(len(buf))
}
+
return len(buf), err
}
@@ -469,10 +479,9 @@ func (f *WebDavFile) Close() error {
func (f *WebDavFile) Read(p []byte) (readSize int, err error) {
glog.V(2).Infof("WebDavFileSystem.Read %v", f.name)
- ctx := context.Background()
if f.entry == nil {
- f.entry, err = filer2.GetEntry(ctx, f.fs, f.name)
+ f.entry, err = filer2.GetEntry(f.fs, filer2.FullPath(f.name))
}
if f.entry == nil {
return 0, err
@@ -488,30 +497,29 @@ func (f *WebDavFile) Read(p []byte) (readSize int, err error) {
}
chunkViews := filer2.ViewFromVisibleIntervals(f.entryViewCache, f.off, len(p))
- totalRead, err := filer2.ReadIntoBuffer(ctx, f.fs, f.name, p, chunkViews, f.off)
+ totalRead, err := filer2.ReadIntoBuffer(f.fs, filer2.FullPath(f.name), p, chunkViews, f.off)
if err != nil {
return 0, err
}
readSize = int(totalRead)
+ glog.V(3).Infof("WebDavFileSystem.Read %v: [%d,%d)", f.name, f.off, f.off+totalRead)
+
f.off += totalRead
if readSize == 0 {
return 0, io.EOF
}
+
return
}
func (f *WebDavFile) Readdir(count int) (ret []os.FileInfo, err error) {
glog.V(2).Infof("WebDavFileSystem.Readdir %v count %d", f.name, count)
- ctx := context.Background()
- dir := f.name
- if dir != "/" && strings.HasSuffix(dir, "/") {
- dir = dir[:len(dir)-1]
- }
+ dir, _ := filer2.FullPath(f.name).DirAndName()
- err = filer2.ReadDirAllEntries(ctx, f.fs, dir, func(entry *filer_pb.Entry) {
+ err = filer2.ReadDirAllEntries(f.fs, filer2.FullPath(dir), "", func(entry *filer_pb.Entry, isLast bool) {
fi := FileInfo{
size: int64(filer2.TotalSize(entry.GetChunks())),
name: entry.Name,
diff --git a/weed/shell/command_bucket_create.go b/weed/shell/command_bucket_create.go
new file mode 100644
index 000000000..3546528aa
--- /dev/null
+++ b/weed/shell/command_bucket_create.go
@@ -0,0 +1,88 @@
+package shell
+
+import (
+ "context"
+ "flag"
+ "fmt"
+ "io"
+ "os"
+ "time"
+
+ "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
+)
+
+func init() {
+ Commands = append(Commands, &commandBucketCreate{})
+}
+
+type commandBucketCreate struct {
+}
+
+func (c *commandBucketCreate) Name() string {
+ return "bucket.create"
+}
+
+func (c *commandBucketCreate) Help() string {
+ return `create a bucket with a given name
+
+ Example:
+ bucket.create -name -replication 001
+`
+}
+
+func (c *commandBucketCreate) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+
+ bucketCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
+ bucketName := bucketCommand.String("name", "", "bucket name")
+ replication := bucketCommand.String("replication", "", "replication setting for the bucket")
+ if err = bucketCommand.Parse(args); err != nil {
+ return nil
+ }
+
+ if *bucketName == "" {
+ return fmt.Errorf("empty bucket name")
+ }
+
+ filerServer, filerPort, _, parseErr := commandEnv.parseUrl(findInputDirectory(bucketCommand.Args()))
+ if parseErr != nil {
+ return parseErr
+ }
+
+ err = commandEnv.withFilerClient(filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
+
+ resp, err := client.GetFilerConfiguration(context.Background(), &filer_pb.GetFilerConfigurationRequest{})
+ if err != nil {
+ return fmt.Errorf("get filer %s:%d configuration: %v", filerServer, filerPort, err)
+ }
+ filerBucketsPath := resp.DirBuckets
+
+ println("create bucket under", filerBucketsPath)
+
+ entry := &filer_pb.Entry{
+ Name: *bucketName,
+ IsDirectory: true,
+ Attributes: &filer_pb.FuseAttributes{
+ Mtime: time.Now().Unix(),
+ Crtime: time.Now().Unix(),
+ FileMode: uint32(0777 | os.ModeDir),
+ Collection: *bucketName,
+ Replication: *replication,
+ },
+ }
+
+ if err := filer_pb.CreateEntry(client, &filer_pb.CreateEntryRequest{
+ Directory: filerBucketsPath,
+ Entry: entry,
+ }); err != nil {
+ return err
+ }
+
+ println("created bucket", *bucketName)
+
+ return nil
+
+ })
+
+ return err
+
+}
diff --git a/weed/shell/command_bucket_delete.go b/weed/shell/command_bucket_delete.go
new file mode 100644
index 000000000..c57ce7221
--- /dev/null
+++ b/weed/shell/command_bucket_delete.go
@@ -0,0 +1,71 @@
+package shell
+
+import (
+ "context"
+ "flag"
+ "fmt"
+ "io"
+
+ "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
+)
+
+func init() {
+ Commands = append(Commands, &commandBucketDelete{})
+}
+
+type commandBucketDelete struct {
+}
+
+func (c *commandBucketDelete) Name() string {
+ return "bucket.delete"
+}
+
+func (c *commandBucketDelete) Help() string {
+ return `delete a bucket by a given name
+
+ bucket.delete -name
+`
+}
+
+func (c *commandBucketDelete) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+
+ bucketCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
+ bucketName := bucketCommand.String("name", "", "bucket name")
+ if err = bucketCommand.Parse(args); err != nil {
+ return nil
+ }
+
+ if *bucketName == "" {
+ return fmt.Errorf("empty bucket name")
+ }
+
+ filerServer, filerPort, _, parseErr := commandEnv.parseUrl(findInputDirectory(bucketCommand.Args()))
+ if parseErr != nil {
+ return parseErr
+ }
+
+ err = commandEnv.withFilerClient(filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
+
+ resp, err := client.GetFilerConfiguration(context.Background(), &filer_pb.GetFilerConfigurationRequest{})
+ if err != nil {
+ return fmt.Errorf("get filer %s:%d configuration: %v", filerServer, filerPort, err)
+ }
+ filerBucketsPath := resp.DirBuckets
+
+ if _, err := client.DeleteEntry(context.Background(), &filer_pb.DeleteEntryRequest{
+ Directory: filerBucketsPath,
+ Name: *bucketName,
+ IsDeleteData: false,
+ IsRecursive: true,
+ IgnoreRecursiveError: true,
+ }); err != nil {
+ return err
+ }
+
+ return nil
+
+ })
+
+ return err
+
+}
diff --git a/weed/shell/command_bucket_list.go b/weed/shell/command_bucket_list.go
new file mode 100644
index 000000000..5eb5972ce
--- /dev/null
+++ b/weed/shell/command_bucket_list.go
@@ -0,0 +1,81 @@
+package shell
+
+import (
+ "context"
+ "flag"
+ "fmt"
+ "io"
+ "math"
+
+ "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
+)
+
+func init() {
+ Commands = append(Commands, &commandBucketList{})
+}
+
+type commandBucketList struct {
+}
+
+func (c *commandBucketList) Name() string {
+ return "bucket.list"
+}
+
+func (c *commandBucketList) Help() string {
+ return `list all buckets
+
+`
+}
+
+func (c *commandBucketList) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+
+ bucketCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
+ if err = bucketCommand.Parse(args); err != nil {
+ return nil
+ }
+
+ filerServer, filerPort, _, parseErr := commandEnv.parseUrl(findInputDirectory(bucketCommand.Args()))
+ if parseErr != nil {
+ return parseErr
+ }
+
+ err = commandEnv.withFilerClient(filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
+
+ resp, err := client.GetFilerConfiguration(context.Background(), &filer_pb.GetFilerConfigurationRequest{})
+ if err != nil {
+ return fmt.Errorf("get filer %s:%d configuration: %v", filerServer, filerPort, err)
+ }
+ filerBucketsPath := resp.DirBuckets
+
+ stream, err := client.ListEntries(context.Background(), &filer_pb.ListEntriesRequest{
+ Directory: filerBucketsPath,
+ Limit: math.MaxUint32,
+ })
+ if err != nil {
+ return fmt.Errorf("list buckets under %v: %v", filerBucketsPath, err)
+ }
+
+ for {
+ resp, recvErr := stream.Recv()
+ if recvErr != nil {
+ if recvErr == io.EOF {
+ break
+ } else {
+ return recvErr
+ }
+ }
+
+ if resp.Entry.Attributes.Replication == "" || resp.Entry.Attributes.Replication == "000" {
+ fmt.Fprintf(writer, " %s\n", resp.Entry.Name)
+ } else {
+ fmt.Fprintf(writer, " %s\t\t\treplication: %s\n", resp.Entry.Name, resp.Entry.Attributes.Replication)
+ }
+ }
+
+ return nil
+
+ })
+
+ return err
+
+}
diff --git a/weed/shell/command_collection_delete.go b/weed/shell/command_collection_delete.go
index fbaddcd51..4b3d7f0be 100644
--- a/weed/shell/command_collection_delete.go
+++ b/weed/shell/command_collection_delete.go
@@ -34,9 +34,8 @@ func (c *commandCollectionDelete) Do(args []string, commandEnv *CommandEnv, writ
collectionName := args[0]
- ctx := context.Background()
- err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
- _, err = client.CollectionDelete(ctx, &master_pb.CollectionDeleteRequest{
+ err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
+ _, err = client.CollectionDelete(context.Background(), &master_pb.CollectionDeleteRequest{
Name: collectionName,
})
return err
diff --git a/weed/shell/command_collection_list.go b/weed/shell/command_collection_list.go
index c4325c66f..2a114e61b 100644
--- a/weed/shell/command_collection_list.go
+++ b/weed/shell/command_collection_list.go
@@ -41,9 +41,8 @@ func (c *commandCollectionList) Do(args []string, commandEnv *CommandEnv, writer
func ListCollectionNames(commandEnv *CommandEnv, includeNormalVolumes, includeEcVolumes bool) (collections []string, err error) {
var resp *master_pb.CollectionListResponse
- ctx := context.Background()
- err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
- resp, err = client.CollectionList(ctx, &master_pb.CollectionListRequest{
+ err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
+ resp, err = client.CollectionList(context.Background(), &master_pb.CollectionListRequest{
IncludeNormalVolumes: includeNormalVolumes,
IncludeEcVolumes: includeEcVolumes,
})
diff --git a/weed/shell/command_ec_balance.go b/weed/shell/command_ec_balance.go
index 96599372e..299d44fed 100644
--- a/weed/shell/command_ec_balance.go
+++ b/weed/shell/command_ec_balance.go
@@ -1,7 +1,6 @@
package shell
import (
- "context"
"flag"
"fmt"
"io"
@@ -107,10 +106,8 @@ func (c *commandEcBalance) Do(args []string, commandEnv *CommandEnv, writer io.W
return nil
}
- ctx := context.Background()
-
// collect all ec nodes
- allEcNodes, totalFreeEcSlots, err := collectEcNodes(ctx, commandEnv, *dc)
+ allEcNodes, totalFreeEcSlots, err := collectEcNodes(commandEnv, *dc)
if err != nil {
return err
}
@@ -138,7 +135,7 @@ func (c *commandEcBalance) Do(args []string, commandEnv *CommandEnv, writer io.W
}
}
- if err := balanceEcRacks(ctx, commandEnv, racks, *applyBalancing); err != nil {
+ if err := balanceEcRacks(commandEnv, racks, *applyBalancing); err != nil {
return fmt.Errorf("balance ec racks: %v", err)
}
@@ -162,38 +159,36 @@ func collectRacks(allEcNodes []*EcNode) map[RackId]*EcRack {
func balanceEcVolumes(commandEnv *CommandEnv, collection string, allEcNodes []*EcNode, racks map[RackId]*EcRack, applyBalancing bool) error {
- ctx := context.Background()
-
fmt.Printf("balanceEcVolumes %s\n", collection)
- if err := deleteDuplicatedEcShards(ctx, commandEnv, allEcNodes, collection, applyBalancing); err != nil {
+ if err := deleteDuplicatedEcShards(commandEnv, allEcNodes, collection, applyBalancing); err != nil {
return fmt.Errorf("delete duplicated collection %s ec shards: %v", collection, err)
}
- if err := balanceEcShardsAcrossRacks(ctx, commandEnv, allEcNodes, racks, collection, applyBalancing); err != nil {
+ if err := balanceEcShardsAcrossRacks(commandEnv, allEcNodes, racks, collection, applyBalancing); err != nil {
return fmt.Errorf("balance across racks collection %s ec shards: %v", collection, err)
}
- if err := balanceEcShardsWithinRacks(ctx, commandEnv, allEcNodes, racks, collection, applyBalancing); err != nil {
+ if err := balanceEcShardsWithinRacks(commandEnv, allEcNodes, racks, collection, applyBalancing); err != nil {
return fmt.Errorf("balance across racks collection %s ec shards: %v", collection, err)
}
return nil
}
-func deleteDuplicatedEcShards(ctx context.Context, commandEnv *CommandEnv, allEcNodes []*EcNode, collection string, applyBalancing bool) error {
+func deleteDuplicatedEcShards(commandEnv *CommandEnv, allEcNodes []*EcNode, collection string, applyBalancing bool) error {
// vid => []ecNode
vidLocations := collectVolumeIdToEcNodes(allEcNodes)
// deduplicate ec shards
for vid, locations := range vidLocations {
- if err := doDeduplicateEcShards(ctx, commandEnv, collection, vid, locations, applyBalancing); err != nil {
+ if err := doDeduplicateEcShards(commandEnv, collection, vid, locations, applyBalancing); err != nil {
return err
}
}
return nil
}
-func doDeduplicateEcShards(ctx context.Context, commandEnv *CommandEnv, collection string, vid needle.VolumeId, locations []*EcNode, applyBalancing bool) error {
+func doDeduplicateEcShards(commandEnv *CommandEnv, collection string, vid needle.VolumeId, locations []*EcNode, applyBalancing bool) error {
// check whether this volume has ecNodes that are over average
shardToLocations := make([][]*EcNode, erasure_coding.TotalShardsCount)
@@ -215,10 +210,10 @@ func doDeduplicateEcShards(ctx context.Context, commandEnv *CommandEnv, collecti
duplicatedShardIds := []uint32{uint32(shardId)}
for _, ecNode := range ecNodes[1:] {
- if err := unmountEcShards(ctx, commandEnv.option.GrpcDialOption, vid, ecNode.info.Id, duplicatedShardIds); err != nil {
+ if err := unmountEcShards(commandEnv.option.GrpcDialOption, vid, ecNode.info.Id, duplicatedShardIds); err != nil {
return err
}
- if err := sourceServerDeleteEcShards(ctx, commandEnv.option.GrpcDialOption, collection, vid, ecNode.info.Id, duplicatedShardIds); err != nil {
+ if err := sourceServerDeleteEcShards(commandEnv.option.GrpcDialOption, collection, vid, ecNode.info.Id, duplicatedShardIds); err != nil {
return err
}
ecNode.deleteEcVolumeShards(vid, duplicatedShardIds)
@@ -227,19 +222,19 @@ func doDeduplicateEcShards(ctx context.Context, commandEnv *CommandEnv, collecti
return nil
}
-func balanceEcShardsAcrossRacks(ctx context.Context, commandEnv *CommandEnv, allEcNodes []*EcNode, racks map[RackId]*EcRack, collection string, applyBalancing bool) error {
+func balanceEcShardsAcrossRacks(commandEnv *CommandEnv, allEcNodes []*EcNode, racks map[RackId]*EcRack, collection string, applyBalancing bool) error {
// collect vid => []ecNode, since previous steps can change the locations
vidLocations := collectVolumeIdToEcNodes(allEcNodes)
// spread the ec shards evenly
for vid, locations := range vidLocations {
- if err := doBalanceEcShardsAcrossRacks(ctx, commandEnv, collection, vid, locations, racks, applyBalancing); err != nil {
+ if err := doBalanceEcShardsAcrossRacks(commandEnv, collection, vid, locations, racks, applyBalancing); err != nil {
return err
}
}
return nil
}
-func doBalanceEcShardsAcrossRacks(ctx context.Context, commandEnv *CommandEnv, collection string, vid needle.VolumeId, locations []*EcNode, racks map[RackId]*EcRack, applyBalancing bool) error {
+func doBalanceEcShardsAcrossRacks(commandEnv *CommandEnv, collection string, vid needle.VolumeId, locations []*EcNode, racks map[RackId]*EcRack, applyBalancing bool) error {
// calculate average number of shards an ec rack should have for one volume
averageShardsPerEcRack := ceilDivide(erasure_coding.TotalShardsCount, len(racks))
@@ -274,7 +269,7 @@ func doBalanceEcShardsAcrossRacks(ctx context.Context, commandEnv *CommandEnv, c
for _, n := range racks[rackId].ecNodes {
possibleDestinationEcNodes = append(possibleDestinationEcNodes, n)
}
- err := pickOneEcNodeAndMoveOneShard(ctx, commandEnv, averageShardsPerEcRack, ecNode, collection, vid, shardId, possibleDestinationEcNodes, applyBalancing)
+ err := pickOneEcNodeAndMoveOneShard(commandEnv, averageShardsPerEcRack, ecNode, collection, vid, shardId, possibleDestinationEcNodes, applyBalancing)
if err != nil {
return err
}
@@ -306,7 +301,7 @@ func pickOneRack(rackToEcNodes map[RackId]*EcRack, rackToShardCount map[string]i
return ""
}
-func balanceEcShardsWithinRacks(ctx context.Context, commandEnv *CommandEnv, allEcNodes []*EcNode, racks map[RackId]*EcRack, collection string, applyBalancing bool) error {
+func balanceEcShardsWithinRacks(commandEnv *CommandEnv, allEcNodes []*EcNode, racks map[RackId]*EcRack, collection string, applyBalancing bool) error {
// collect vid => []ecNode, since previous steps can change the locations
vidLocations := collectVolumeIdToEcNodes(allEcNodes)
@@ -330,7 +325,7 @@ func balanceEcShardsWithinRacks(ctx context.Context, commandEnv *CommandEnv, all
}
sourceEcNodes := rackEcNodesWithVid[rackId]
averageShardsPerEcNode := ceilDivide(rackToShardCount[rackId], len(possibleDestinationEcNodes))
- if err := doBalanceEcShardsWithinOneRack(ctx, commandEnv, averageShardsPerEcNode, collection, vid, sourceEcNodes, possibleDestinationEcNodes, applyBalancing); err != nil {
+ if err := doBalanceEcShardsWithinOneRack(commandEnv, averageShardsPerEcNode, collection, vid, sourceEcNodes, possibleDestinationEcNodes, applyBalancing); err != nil {
return err
}
}
@@ -338,7 +333,7 @@ func balanceEcShardsWithinRacks(ctx context.Context, commandEnv *CommandEnv, all
return nil
}
-func doBalanceEcShardsWithinOneRack(ctx context.Context, commandEnv *CommandEnv, averageShardsPerEcNode int, collection string, vid needle.VolumeId, existingLocations, possibleDestinationEcNodes []*EcNode, applyBalancing bool) error {
+func doBalanceEcShardsWithinOneRack(commandEnv *CommandEnv, averageShardsPerEcNode int, collection string, vid needle.VolumeId, existingLocations, possibleDestinationEcNodes []*EcNode, applyBalancing bool) error {
for _, ecNode := range existingLocations {
@@ -353,7 +348,7 @@ func doBalanceEcShardsWithinOneRack(ctx context.Context, commandEnv *CommandEnv,
fmt.Printf("%s has %d overlimit, moving ec shard %d.%d\n", ecNode.info.Id, overLimitCount, vid, shardId)
- err := pickOneEcNodeAndMoveOneShard(ctx, commandEnv, averageShardsPerEcNode, ecNode, collection, vid, shardId, possibleDestinationEcNodes, applyBalancing)
+ err := pickOneEcNodeAndMoveOneShard(commandEnv, averageShardsPerEcNode, ecNode, collection, vid, shardId, possibleDestinationEcNodes, applyBalancing)
if err != nil {
return err
}
@@ -365,18 +360,18 @@ func doBalanceEcShardsWithinOneRack(ctx context.Context, commandEnv *CommandEnv,
return nil
}
-func balanceEcRacks(ctx context.Context, commandEnv *CommandEnv, racks map[RackId]*EcRack, applyBalancing bool) error {
+func balanceEcRacks(commandEnv *CommandEnv, racks map[RackId]*EcRack, applyBalancing bool) error {
// balance one rack for all ec shards
for _, ecRack := range racks {
- if err := doBalanceEcRack(ctx, commandEnv, ecRack, applyBalancing); err != nil {
+ if err := doBalanceEcRack(commandEnv, ecRack, applyBalancing); err != nil {
return err
}
}
return nil
}
-func doBalanceEcRack(ctx context.Context, commandEnv *CommandEnv, ecRack *EcRack, applyBalancing bool) error {
+func doBalanceEcRack(commandEnv *CommandEnv, ecRack *EcRack, applyBalancing bool) error {
if len(ecRack.ecNodes) <= 1 {
return nil
@@ -421,7 +416,7 @@ func doBalanceEcRack(ctx context.Context, commandEnv *CommandEnv, ecRack *EcRack
fmt.Printf("%s moves ec shards %d.%d to %s\n", fullNode.info.Id, shards.Id, shardId, emptyNode.info.Id)
- err := moveMountedShardToEcNode(ctx, commandEnv, fullNode, shards.Collection, needle.VolumeId(shards.Id), shardId, emptyNode, applyBalancing)
+ err := moveMountedShardToEcNode(commandEnv, fullNode, shards.Collection, needle.VolumeId(shards.Id), shardId, emptyNode, applyBalancing)
if err != nil {
return err
}
@@ -440,7 +435,7 @@ func doBalanceEcRack(ctx context.Context, commandEnv *CommandEnv, ecRack *EcRack
return nil
}
-func pickOneEcNodeAndMoveOneShard(ctx context.Context, commandEnv *CommandEnv, averageShardsPerEcNode int, existingLocation *EcNode, collection string, vid needle.VolumeId, shardId erasure_coding.ShardId, possibleDestinationEcNodes []*EcNode, applyBalancing bool) error {
+func pickOneEcNodeAndMoveOneShard(commandEnv *CommandEnv, averageShardsPerEcNode int, existingLocation *EcNode, collection string, vid needle.VolumeId, shardId erasure_coding.ShardId, possibleDestinationEcNodes []*EcNode, applyBalancing bool) error {
sortEcNodesByFreeslotsDecending(possibleDestinationEcNodes)
@@ -458,7 +453,7 @@ func pickOneEcNodeAndMoveOneShard(ctx context.Context, commandEnv *CommandEnv, a
fmt.Printf("%s moves ec shard %d.%d to %s\n", existingLocation.info.Id, vid, shardId, destEcNode.info.Id)
- err := moveMountedShardToEcNode(ctx, commandEnv, existingLocation, collection, vid, shardId, destEcNode, applyBalancing)
+ err := moveMountedShardToEcNode(commandEnv, existingLocation, collection, vid, shardId, destEcNode, applyBalancing)
if err != nil {
return err
}
diff --git a/weed/shell/command_ec_common.go b/weed/shell/command_ec_common.go
index cfe14fed5..0db119d3c 100644
--- a/weed/shell/command_ec_common.go
+++ b/weed/shell/command_ec_common.go
@@ -15,26 +15,26 @@ import (
"google.golang.org/grpc"
)
-func moveMountedShardToEcNode(ctx context.Context, commandEnv *CommandEnv, existingLocation *EcNode, collection string, vid needle.VolumeId, shardId erasure_coding.ShardId, destinationEcNode *EcNode, applyBalancing bool) (err error) {
+func moveMountedShardToEcNode(commandEnv *CommandEnv, existingLocation *EcNode, collection string, vid needle.VolumeId, shardId erasure_coding.ShardId, destinationEcNode *EcNode, applyBalancing bool) (err error) {
copiedShardIds := []uint32{uint32(shardId)}
if applyBalancing {
// ask destination node to copy shard and the ecx file from source node, and mount it
- copiedShardIds, err = oneServerCopyAndMountEcShardsFromSource(ctx, commandEnv.option.GrpcDialOption, destinationEcNode, uint32(shardId), 1, vid, collection, existingLocation.info.Id)
+ copiedShardIds, err = oneServerCopyAndMountEcShardsFromSource(commandEnv.option.GrpcDialOption, destinationEcNode, []uint32{uint32(shardId)}, vid, collection, existingLocation.info.Id)
if err != nil {
return err
}
// unmount the to be deleted shards
- err = unmountEcShards(ctx, commandEnv.option.GrpcDialOption, vid, existingLocation.info.Id, copiedShardIds)
+ err = unmountEcShards(commandEnv.option.GrpcDialOption, vid, existingLocation.info.Id, copiedShardIds)
if err != nil {
return err
}
// ask source node to delete the shard, and maybe the ecx file
- err = sourceServerDeleteEcShards(ctx, commandEnv.option.GrpcDialOption, collection, vid, existingLocation.info.Id, copiedShardIds)
+ err = sourceServerDeleteEcShards(commandEnv.option.GrpcDialOption, collection, vid, existingLocation.info.Id, copiedShardIds)
if err != nil {
return err
}
@@ -50,14 +50,10 @@ func moveMountedShardToEcNode(ctx context.Context, commandEnv *CommandEnv, exist
}
-func oneServerCopyAndMountEcShardsFromSource(ctx context.Context, grpcDialOption grpc.DialOption,
- targetServer *EcNode, startFromShardId uint32, shardCount int,
+func oneServerCopyAndMountEcShardsFromSource(grpcDialOption grpc.DialOption,
+ targetServer *EcNode, shardIdsToCopy []uint32,
volumeId needle.VolumeId, collection string, existingLocation string) (copiedShardIds []uint32, err error) {
- var shardIdsToCopy []uint32
- for shardId := startFromShardId; shardId < startFromShardId+uint32(shardCount); shardId++ {
- shardIdsToCopy = append(shardIdsToCopy, shardId)
- }
fmt.Printf("allocate %d.%v %s => %s\n", volumeId, shardIdsToCopy, existingLocation, targetServer.info.Id)
err = operation.WithVolumeServerClient(targetServer.info.Id, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
@@ -65,11 +61,13 @@ func oneServerCopyAndMountEcShardsFromSource(ctx context.Context, grpcDialOption
if targetServer.info.Id != existingLocation {
fmt.Printf("copy %d.%v %s => %s\n", volumeId, shardIdsToCopy, existingLocation, targetServer.info.Id)
- _, copyErr := volumeServerClient.VolumeEcShardsCopy(ctx, &volume_server_pb.VolumeEcShardsCopyRequest{
+ _, copyErr := volumeServerClient.VolumeEcShardsCopy(context.Background(), &volume_server_pb.VolumeEcShardsCopyRequest{
VolumeId: uint32(volumeId),
Collection: collection,
ShardIds: shardIdsToCopy,
CopyEcxFile: true,
+ CopyEcjFile: true,
+ CopyVifFile: true,
SourceDataNode: existingLocation,
})
if copyErr != nil {
@@ -78,7 +76,7 @@ func oneServerCopyAndMountEcShardsFromSource(ctx context.Context, grpcDialOption
}
fmt.Printf("mount %d.%v on %s\n", volumeId, shardIdsToCopy, targetServer.info.Id)
- _, mountErr := volumeServerClient.VolumeEcShardsMount(ctx, &volume_server_pb.VolumeEcShardsMountRequest{
+ _, mountErr := volumeServerClient.VolumeEcShardsMount(context.Background(), &volume_server_pb.VolumeEcShardsMountRequest{
VolumeId: uint32(volumeId),
Collection: collection,
ShardIds: shardIdsToCopy,
@@ -180,12 +178,12 @@ type EcRack struct {
freeEcSlot int
}
-func collectEcNodes(ctx context.Context, commandEnv *CommandEnv, selectedDataCenter string) (ecNodes []*EcNode, totalFreeEcSlots int, err error) {
+func collectEcNodes(commandEnv *CommandEnv, selectedDataCenter string) (ecNodes []*EcNode, totalFreeEcSlots int, err error) {
// list all possible locations
var resp *master_pb.VolumeListResponse
- err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
- resp, err = client.VolumeList(ctx, &master_pb.VolumeListRequest{})
+ err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
+ resp, err = client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
return err
})
if err != nil {
@@ -213,13 +211,12 @@ func collectEcNodes(ctx context.Context, commandEnv *CommandEnv, selectedDataCen
return
}
-func sourceServerDeleteEcShards(ctx context.Context, grpcDialOption grpc.DialOption,
- collection string, volumeId needle.VolumeId, sourceLocation string, toBeDeletedShardIds []uint32) error {
+func sourceServerDeleteEcShards(grpcDialOption grpc.DialOption, collection string, volumeId needle.VolumeId, sourceLocation string, toBeDeletedShardIds []uint32) error {
fmt.Printf("delete %d.%v from %s\n", volumeId, toBeDeletedShardIds, sourceLocation)
return operation.WithVolumeServerClient(sourceLocation, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
- _, deleteErr := volumeServerClient.VolumeEcShardsDelete(ctx, &volume_server_pb.VolumeEcShardsDeleteRequest{
+ _, deleteErr := volumeServerClient.VolumeEcShardsDelete(context.Background(), &volume_server_pb.VolumeEcShardsDeleteRequest{
VolumeId: uint32(volumeId),
Collection: collection,
ShardIds: toBeDeletedShardIds,
@@ -229,13 +226,12 @@ func sourceServerDeleteEcShards(ctx context.Context, grpcDialOption grpc.DialOpt
}
-func unmountEcShards(ctx context.Context, grpcDialOption grpc.DialOption,
- volumeId needle.VolumeId, sourceLocation string, toBeUnmountedhardIds []uint32) error {
+func unmountEcShards(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceLocation string, toBeUnmountedhardIds []uint32) error {
fmt.Printf("unmount %d.%v from %s\n", volumeId, toBeUnmountedhardIds, sourceLocation)
return operation.WithVolumeServerClient(sourceLocation, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
- _, deleteErr := volumeServerClient.VolumeEcShardsUnmount(ctx, &volume_server_pb.VolumeEcShardsUnmountRequest{
+ _, deleteErr := volumeServerClient.VolumeEcShardsUnmount(context.Background(), &volume_server_pb.VolumeEcShardsUnmountRequest{
VolumeId: uint32(volumeId),
ShardIds: toBeUnmountedhardIds,
})
@@ -243,13 +239,12 @@ func unmountEcShards(ctx context.Context, grpcDialOption grpc.DialOption,
})
}
-func mountEcShards(ctx context.Context, grpcDialOption grpc.DialOption,
- collection string, volumeId needle.VolumeId, sourceLocation string, toBeMountedhardIds []uint32) error {
+func mountEcShards(grpcDialOption grpc.DialOption, collection string, volumeId needle.VolumeId, sourceLocation string, toBeMountedhardIds []uint32) error {
fmt.Printf("mount %d.%v on %s\n", volumeId, toBeMountedhardIds, sourceLocation)
return operation.WithVolumeServerClient(sourceLocation, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
- _, mountErr := volumeServerClient.VolumeEcShardsMount(ctx, &volume_server_pb.VolumeEcShardsMountRequest{
+ _, mountErr := volumeServerClient.VolumeEcShardsMount(context.Background(), &volume_server_pb.VolumeEcShardsMountRequest{
VolumeId: uint32(volumeId),
Collection: collection,
ShardIds: toBeMountedhardIds,
diff --git a/weed/shell/command_ec_decode.go b/weed/shell/command_ec_decode.go
new file mode 100644
index 000000000..b69e403cb
--- /dev/null
+++ b/weed/shell/command_ec_decode.go
@@ -0,0 +1,264 @@
+package shell
+
+import (
+ "context"
+ "flag"
+ "fmt"
+ "io"
+
+ "google.golang.org/grpc"
+
+ "github.com/chrislusf/seaweedfs/weed/operation"
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+)
+
+func init() {
+ Commands = append(Commands, &commandEcDecode{})
+}
+
+type commandEcDecode struct {
+}
+
+func (c *commandEcDecode) Name() string {
+ return "ec.decode"
+}
+
+func (c *commandEcDecode) Help() string {
+ return `decode a erasure coded volume into a normal volume
+
+ ec.decode [-collection=""] [-volumeId=]
+
+`
+}
+
+func (c *commandEcDecode) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+
+ encodeCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
+ volumeId := encodeCommand.Int("volumeId", 0, "the volume id")
+ collection := encodeCommand.String("collection", "", "the collection name")
+ if err = encodeCommand.Parse(args); err != nil {
+ return nil
+ }
+
+ vid := needle.VolumeId(*volumeId)
+
+ // collect topology information
+ topologyInfo, err := collectTopologyInfo(commandEnv)
+ if err != nil {
+ return err
+ }
+
+ // volumeId is provided
+ if vid != 0 {
+ return doEcDecode(commandEnv, topologyInfo, *collection, vid)
+ }
+
+ // apply to all volumes in the collection
+ volumeIds := collectEcShardIds(topologyInfo, *collection)
+ fmt.Printf("ec encode volumes: %v\n", volumeIds)
+ for _, vid := range volumeIds {
+ if err = doEcDecode(commandEnv, topologyInfo, *collection, vid); err != nil {
+ return err
+ }
+ }
+
+ return nil
+}
+
+func doEcDecode(commandEnv *CommandEnv, topoInfo *master_pb.TopologyInfo, collection string, vid needle.VolumeId) (err error) {
+ // find volume location
+ nodeToEcIndexBits := collectEcNodeShardBits(topoInfo, vid)
+
+ fmt.Printf("ec volume %d shard locations: %+v\n", vid, nodeToEcIndexBits)
+
+ // collect ec shards to the server with most space
+ targetNodeLocation, err := collectEcShards(commandEnv, nodeToEcIndexBits, collection, vid)
+ if err != nil {
+ return fmt.Errorf("collectEcShards for volume %d: %v", vid, err)
+ }
+
+ // generate a normal volume
+ err = generateNormalVolume(commandEnv.option.GrpcDialOption, needle.VolumeId(vid), collection, targetNodeLocation)
+ if err != nil {
+ return fmt.Errorf("generate normal volume %d on %s: %v", vid, targetNodeLocation, err)
+ }
+
+ // delete the previous ec shards
+ err = mountVolumeAndDeleteEcShards(commandEnv.option.GrpcDialOption, collection, targetNodeLocation, nodeToEcIndexBits, vid)
+ if err != nil {
+ return fmt.Errorf("delete ec shards for volume %d: %v", vid, err)
+ }
+
+ return nil
+}
+
+func mountVolumeAndDeleteEcShards(grpcDialOption grpc.DialOption, collection, targetNodeLocation string, nodeToEcIndexBits map[string]erasure_coding.ShardBits, vid needle.VolumeId) error {
+
+ // mount volume
+ if err := operation.WithVolumeServerClient(targetNodeLocation, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
+ _, mountErr := volumeServerClient.VolumeMount(context.Background(), &volume_server_pb.VolumeMountRequest{
+ VolumeId: uint32(vid),
+ })
+ return mountErr
+ }); err != nil {
+ return fmt.Errorf("mountVolumeAndDeleteEcShards mount volume %d on %s: %v", vid, targetNodeLocation, err)
+ }
+
+ // unmount ec shards
+ for location, ecIndexBits := range nodeToEcIndexBits {
+ fmt.Printf("unmount ec volume %d on %s has shards: %+v\n", vid, location, ecIndexBits.ShardIds())
+ err := unmountEcShards(grpcDialOption, vid, location, ecIndexBits.ToUint32Slice())
+ if err != nil {
+ return fmt.Errorf("mountVolumeAndDeleteEcShards unmount ec volume %d on %s: %v", vid, location, err)
+ }
+ }
+ // delete ec shards
+ for location, ecIndexBits := range nodeToEcIndexBits {
+ fmt.Printf("delete ec volume %d on %s has shards: %+v\n", vid, location, ecIndexBits.ShardIds())
+ err := sourceServerDeleteEcShards(grpcDialOption, collection, vid, location, ecIndexBits.ToUint32Slice())
+ if err != nil {
+ return fmt.Errorf("mountVolumeAndDeleteEcShards delete ec volume %d on %s: %v", vid, location, err)
+ }
+ }
+
+ return nil
+}
+
+func generateNormalVolume(grpcDialOption grpc.DialOption, vid needle.VolumeId, collection string, sourceVolumeServer string) error {
+
+ fmt.Printf("generateNormalVolume from ec volume %d on %s\n", vid, sourceVolumeServer)
+
+ err := operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
+ _, genErr := volumeServerClient.VolumeEcShardsToVolume(context.Background(), &volume_server_pb.VolumeEcShardsToVolumeRequest{
+ VolumeId: uint32(vid),
+ Collection: collection,
+ })
+ return genErr
+ })
+
+ return err
+
+}
+
+func collectEcShards(commandEnv *CommandEnv, nodeToEcIndexBits map[string]erasure_coding.ShardBits, collection string, vid needle.VolumeId) (targetNodeLocation string, err error) {
+
+ maxShardCount := 0
+ var exisitngEcIndexBits erasure_coding.ShardBits
+ for loc, ecIndexBits := range nodeToEcIndexBits {
+ toBeCopiedShardCount := ecIndexBits.MinusParityShards().ShardIdCount()
+ if toBeCopiedShardCount > maxShardCount {
+ maxShardCount = toBeCopiedShardCount
+ targetNodeLocation = loc
+ exisitngEcIndexBits = ecIndexBits
+ }
+ }
+
+ fmt.Printf("collectEcShards: ec volume %d collect shards to %s from: %+v\n", vid, targetNodeLocation, nodeToEcIndexBits)
+
+ var copiedEcIndexBits erasure_coding.ShardBits
+ for loc, ecIndexBits := range nodeToEcIndexBits {
+ if loc == targetNodeLocation {
+ continue
+ }
+
+ needToCopyEcIndexBits := ecIndexBits.Minus(exisitngEcIndexBits).MinusParityShards()
+ if needToCopyEcIndexBits.ShardIdCount() == 0 {
+ continue
+ }
+
+ err = operation.WithVolumeServerClient(targetNodeLocation, commandEnv.option.GrpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
+
+ fmt.Printf("copy %d.%v %s => %s\n", vid, needToCopyEcIndexBits.ShardIds(), loc, targetNodeLocation)
+
+ _, copyErr := volumeServerClient.VolumeEcShardsCopy(context.Background(), &volume_server_pb.VolumeEcShardsCopyRequest{
+ VolumeId: uint32(vid),
+ Collection: collection,
+ ShardIds: needToCopyEcIndexBits.ToUint32Slice(),
+ CopyEcxFile: false,
+ CopyEcjFile: true,
+ CopyVifFile: true,
+ SourceDataNode: loc,
+ })
+ if copyErr != nil {
+ return fmt.Errorf("copy %d.%v %s => %s : %v\n", vid, needToCopyEcIndexBits.ShardIds(), loc, targetNodeLocation, copyErr)
+ }
+
+ return nil
+ })
+
+ if err != nil {
+ break
+ }
+
+ copiedEcIndexBits = copiedEcIndexBits.Plus(needToCopyEcIndexBits)
+
+ }
+
+ nodeToEcIndexBits[targetNodeLocation] = exisitngEcIndexBits.Plus(copiedEcIndexBits)
+
+ return targetNodeLocation, err
+
+}
+
+func collectTopologyInfo(commandEnv *CommandEnv) (topoInfo *master_pb.TopologyInfo, err error) {
+
+ var resp *master_pb.VolumeListResponse
+ err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
+ resp, err = client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
+ return err
+ })
+ if err != nil {
+ return
+ }
+
+ return resp.TopologyInfo, nil
+
+}
+
+func collectEcShardInfos(topoInfo *master_pb.TopologyInfo, selectedCollection string, vid needle.VolumeId) (ecShardInfos []*master_pb.VolumeEcShardInformationMessage) {
+
+ eachDataNode(topoInfo, func(dc string, rack RackId, dn *master_pb.DataNodeInfo) {
+ for _, v := range dn.EcShardInfos {
+ if v.Collection == selectedCollection && v.Id == uint32(vid) {
+ ecShardInfos = append(ecShardInfos, v)
+ }
+ }
+ })
+
+ return
+}
+
+func collectEcShardIds(topoInfo *master_pb.TopologyInfo, selectedCollection string) (vids []needle.VolumeId) {
+
+ vidMap := make(map[uint32]bool)
+ eachDataNode(topoInfo, func(dc string, rack RackId, dn *master_pb.DataNodeInfo) {
+ for _, v := range dn.EcShardInfos {
+ if v.Collection == selectedCollection {
+ vidMap[v.Id] = true
+ }
+ }
+ })
+
+ for vid := range vidMap {
+ vids = append(vids, needle.VolumeId(vid))
+ }
+
+ return
+}
+
+func collectEcNodeShardBits(topoInfo *master_pb.TopologyInfo, vid needle.VolumeId) map[string]erasure_coding.ShardBits {
+
+ nodeToEcIndexBits := make(map[string]erasure_coding.ShardBits)
+ eachDataNode(topoInfo, func(dc string, rack RackId, dn *master_pb.DataNodeInfo) {
+ for _, v := range dn.EcShardInfos {
+ if v.Id == uint32(vid) {
+ nodeToEcIndexBits[dn.Id] = erasure_coding.ShardBits(v.EcIndexBits)
+ }
+ }
+ })
+
+ return nodeToEcIndexBits
+}
diff --git a/weed/shell/command_ec_encode.go b/weed/shell/command_ec_encode.go
index f07cb93f9..6efb05488 100644
--- a/weed/shell/command_ec_encode.go
+++ b/weed/shell/command_ec_encode.go
@@ -8,13 +8,14 @@ import (
"sync"
"time"
+ "google.golang.org/grpc"
+
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/chrislusf/seaweedfs/weed/wdclient"
- "google.golang.org/grpc"
)
func init() {
@@ -62,22 +63,21 @@ func (c *commandEcEncode) Do(args []string, commandEnv *CommandEnv, writer io.Wr
return nil
}
- ctx := context.Background()
vid := needle.VolumeId(*volumeId)
// volumeId is provided
if vid != 0 {
- return doEcEncode(ctx, commandEnv, *collection, vid)
+ return doEcEncode(commandEnv, *collection, vid)
}
// apply to all volumes in the collection
- volumeIds, err := collectVolumeIdsForEcEncode(ctx, commandEnv, *collection, *fullPercentage, *quietPeriod)
+ volumeIds, err := collectVolumeIdsForEcEncode(commandEnv, *collection, *fullPercentage, *quietPeriod)
if err != nil {
return err
}
fmt.Printf("ec encode volumes: %v\n", volumeIds)
for _, vid := range volumeIds {
- if err = doEcEncode(ctx, commandEnv, *collection, vid); err != nil {
+ if err = doEcEncode(commandEnv, *collection, vid); err != nil {
return err
}
}
@@ -85,27 +85,29 @@ func (c *commandEcEncode) Do(args []string, commandEnv *CommandEnv, writer io.Wr
return nil
}
-func doEcEncode(ctx context.Context, commandEnv *CommandEnv, collection string, vid needle.VolumeId) (err error) {
+func doEcEncode(commandEnv *CommandEnv, collection string, vid needle.VolumeId) (err error) {
// find volume location
locations, found := commandEnv.MasterClient.GetLocations(uint32(vid))
if !found {
return fmt.Errorf("volume %d not found", vid)
}
+ // fmt.Printf("found ec %d shards on %v\n", vid, locations)
+
// mark the volume as readonly
- err = markVolumeReadonly(ctx, commandEnv.option.GrpcDialOption, needle.VolumeId(vid), locations)
+ err = markVolumeReadonly(commandEnv.option.GrpcDialOption, needle.VolumeId(vid), locations)
if err != nil {
- return fmt.Errorf("generate ec shards for volume %d on %s: %v", vid, locations[0].Url, err)
+ return fmt.Errorf("mark volume %d as readonly on %s: %v", vid, locations[0].Url, err)
}
// generate ec shards
- err = generateEcShards(ctx, commandEnv.option.GrpcDialOption, needle.VolumeId(vid), collection, locations[0].Url)
+ err = generateEcShards(commandEnv.option.GrpcDialOption, needle.VolumeId(vid), collection, locations[0].Url)
if err != nil {
return fmt.Errorf("generate ec shards for volume %d on %s: %v", vid, locations[0].Url, err)
}
// balance the ec shards to current cluster
- err = spreadEcShards(ctx, commandEnv, vid, collection, locations)
+ err = spreadEcShards(commandEnv, vid, collection, locations)
if err != nil {
return fmt.Errorf("spread ec shards for volume %d from %s: %v", vid, locations[0].Url, err)
}
@@ -113,12 +115,12 @@ func doEcEncode(ctx context.Context, commandEnv *CommandEnv, collection string,
return nil
}
-func markVolumeReadonly(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, locations []wdclient.Location) error {
+func markVolumeReadonly(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, locations []wdclient.Location) error {
for _, location := range locations {
err := operation.WithVolumeServerClient(location.Url, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
- _, markErr := volumeServerClient.VolumeMarkReadonly(ctx, &volume_server_pb.VolumeMarkReadonlyRequest{
+ _, markErr := volumeServerClient.VolumeMarkReadonly(context.Background(), &volume_server_pb.VolumeMarkReadonlyRequest{
VolumeId: uint32(volumeId),
})
return markErr
@@ -133,10 +135,10 @@ func markVolumeReadonly(ctx context.Context, grpcDialOption grpc.DialOption, vol
return nil
}
-func generateEcShards(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, collection string, sourceVolumeServer string) error {
+func generateEcShards(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, collection string, sourceVolumeServer string) error {
err := operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
- _, genErr := volumeServerClient.VolumeEcShardsGenerate(ctx, &volume_server_pb.VolumeEcShardsGenerateRequest{
+ _, genErr := volumeServerClient.VolumeEcShardsGenerate(context.Background(), &volume_server_pb.VolumeEcShardsGenerateRequest{
VolumeId: uint32(volumeId),
Collection: collection,
})
@@ -147,9 +149,9 @@ func generateEcShards(ctx context.Context, grpcDialOption grpc.DialOption, volum
}
-func spreadEcShards(ctx context.Context, commandEnv *CommandEnv, volumeId needle.VolumeId, collection string, existingLocations []wdclient.Location) (err error) {
+func spreadEcShards(commandEnv *CommandEnv, volumeId needle.VolumeId, collection string, existingLocations []wdclient.Location) (err error) {
- allEcNodes, totalFreeEcSlots, err := collectEcNodes(ctx, commandEnv, "")
+ allEcNodes, totalFreeEcSlots, err := collectEcNodes(commandEnv, "")
if err != nil {
return err
}
@@ -163,29 +165,29 @@ func spreadEcShards(ctx context.Context, commandEnv *CommandEnv, volumeId needle
}
// calculate how many shards to allocate for these servers
- allocated := balancedEcDistribution(allocatedDataNodes)
+ allocatedEcIds := balancedEcDistribution(allocatedDataNodes)
// ask the data nodes to copy from the source volume server
- copiedShardIds, err := parallelCopyEcShardsFromSource(ctx, commandEnv.option.GrpcDialOption, allocatedDataNodes, allocated, volumeId, collection, existingLocations[0])
+ copiedShardIds, err := parallelCopyEcShardsFromSource(commandEnv.option.GrpcDialOption, allocatedDataNodes, allocatedEcIds, volumeId, collection, existingLocations[0])
if err != nil {
return err
}
// unmount the to be deleted shards
- err = unmountEcShards(ctx, commandEnv.option.GrpcDialOption, volumeId, existingLocations[0].Url, copiedShardIds)
+ err = unmountEcShards(commandEnv.option.GrpcDialOption, volumeId, existingLocations[0].Url, copiedShardIds)
if err != nil {
return err
}
// ask the source volume server to clean up copied ec shards
- err = sourceServerDeleteEcShards(ctx, commandEnv.option.GrpcDialOption, collection, volumeId, existingLocations[0].Url, copiedShardIds)
+ err = sourceServerDeleteEcShards(commandEnv.option.GrpcDialOption, collection, volumeId, existingLocations[0].Url, copiedShardIds)
if err != nil {
return fmt.Errorf("source delete copied ecShards %s %d.%v: %v", existingLocations[0].Url, volumeId, copiedShardIds, err)
}
// ask the source volume server to delete the original volume
for _, location := range existingLocations {
- err = deleteVolume(ctx, commandEnv.option.GrpcDialOption, volumeId, location.Url)
+ err = deleteVolume(commandEnv.option.GrpcDialOption, volumeId, location.Url)
if err != nil {
return fmt.Errorf("deleteVolume %s volume %d: %v", location.Url, volumeId, err)
}
@@ -195,32 +197,28 @@ func spreadEcShards(ctx context.Context, commandEnv *CommandEnv, volumeId needle
}
-func parallelCopyEcShardsFromSource(ctx context.Context, grpcDialOption grpc.DialOption,
- targetServers []*EcNode, allocated []int,
- volumeId needle.VolumeId, collection string, existingLocation wdclient.Location) (actuallyCopied []uint32, err error) {
+func parallelCopyEcShardsFromSource(grpcDialOption grpc.DialOption, targetServers []*EcNode, allocatedEcIds [][]uint32, volumeId needle.VolumeId, collection string, existingLocation wdclient.Location) (actuallyCopied []uint32, err error) {
// parallelize
shardIdChan := make(chan []uint32, len(targetServers))
var wg sync.WaitGroup
- startFromShardId := uint32(0)
for i, server := range targetServers {
- if allocated[i] <= 0 {
+ if len(allocatedEcIds[i]) <= 0 {
continue
}
wg.Add(1)
- go func(server *EcNode, startFromShardId uint32, shardCount int) {
+ go func(server *EcNode, allocatedEcShardIds []uint32) {
defer wg.Done()
- copiedShardIds, copyErr := oneServerCopyAndMountEcShardsFromSource(ctx, grpcDialOption, server,
- startFromShardId, shardCount, volumeId, collection, existingLocation.Url)
+ copiedShardIds, copyErr := oneServerCopyAndMountEcShardsFromSource(grpcDialOption, server,
+ allocatedEcShardIds, volumeId, collection, existingLocation.Url)
if copyErr != nil {
err = copyErr
} else {
shardIdChan <- copiedShardIds
server.addEcVolumeShards(volumeId, collection, copiedShardIds)
}
- }(server, startFromShardId, allocated[i])
- startFromShardId += uint32(allocated[i])
+ }(server, allocatedEcIds[i])
}
wg.Wait()
close(shardIdChan)
@@ -236,29 +234,29 @@ func parallelCopyEcShardsFromSource(ctx context.Context, grpcDialOption grpc.Dia
return
}
-func balancedEcDistribution(servers []*EcNode) (allocated []int) {
- allocated = make([]int, len(servers))
- allocatedCount := 0
- for allocatedCount < erasure_coding.TotalShardsCount {
- for i, server := range servers {
- if server.freeEcSlot-allocated[i] > 0 {
- allocated[i] += 1
- allocatedCount += 1
- }
- if allocatedCount >= erasure_coding.TotalShardsCount {
- break
- }
+func balancedEcDistribution(servers []*EcNode) (allocated [][]uint32) {
+ allocated = make([][]uint32, len(servers))
+ allocatedShardIdIndex := uint32(0)
+ serverIndex := 0
+ for allocatedShardIdIndex < erasure_coding.TotalShardsCount {
+ if servers[serverIndex].freeEcSlot > 0 {
+ allocated[serverIndex] = append(allocated[serverIndex], allocatedShardIdIndex)
+ allocatedShardIdIndex++
+ }
+ serverIndex++
+ if serverIndex >= len(servers) {
+ serverIndex = 0
}
}
return allocated
}
-func collectVolumeIdsForEcEncode(ctx context.Context, commandEnv *CommandEnv, selectedCollection string, fullPercentage float64, quietPeriod time.Duration) (vids []needle.VolumeId, err error) {
+func collectVolumeIdsForEcEncode(commandEnv *CommandEnv, selectedCollection string, fullPercentage float64, quietPeriod time.Duration) (vids []needle.VolumeId, err error) {
var resp *master_pb.VolumeListResponse
- err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
- resp, err = client.VolumeList(ctx, &master_pb.VolumeListRequest{})
+ err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
+ resp, err = client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
return err
})
if err != nil {
@@ -281,7 +279,7 @@ func collectVolumeIdsForEcEncode(ctx context.Context, commandEnv *CommandEnv, se
}
})
- for vid, _ := range vidMap {
+ for vid := range vidMap {
vids = append(vids, needle.VolumeId(vid))
}
diff --git a/weed/shell/command_ec_rebuild.go b/weed/shell/command_ec_rebuild.go
index b43aed599..d9d943e6d 100644
--- a/weed/shell/command_ec_rebuild.go
+++ b/weed/shell/command_ec_rebuild.go
@@ -64,7 +64,7 @@ func (c *commandEcRebuild) Do(args []string, commandEnv *CommandEnv, writer io.W
}
// collect all ec nodes
- allEcNodes, _, err := collectEcNodes(context.Background(), commandEnv, "")
+ allEcNodes, _, err := collectEcNodes(commandEnv, "")
if err != nil {
return err
}
@@ -92,8 +92,6 @@ func (c *commandEcRebuild) Do(args []string, commandEnv *CommandEnv, writer io.W
func rebuildEcVolumes(commandEnv *CommandEnv, allEcNodes []*EcNode, collection string, writer io.Writer, applyChanges bool) error {
- ctx := context.Background()
-
fmt.Printf("rebuildEcVolumes %s\n", collection)
// collect vid => each shard locations, similar to ecShardMap in topology.go
@@ -117,7 +115,7 @@ func rebuildEcVolumes(commandEnv *CommandEnv, allEcNodes []*EcNode, collection s
return fmt.Errorf("disk space is not enough")
}
- if err := rebuildOneEcVolume(ctx, commandEnv, allEcNodes[0], collection, vid, locations, writer, applyChanges); err != nil {
+ if err := rebuildOneEcVolume(commandEnv, allEcNodes[0], collection, vid, locations, writer, applyChanges); err != nil {
return err
}
}
@@ -125,13 +123,13 @@ func rebuildEcVolumes(commandEnv *CommandEnv, allEcNodes []*EcNode, collection s
return nil
}
-func rebuildOneEcVolume(ctx context.Context, commandEnv *CommandEnv, rebuilder *EcNode, collection string, volumeId needle.VolumeId, locations EcShardLocations, writer io.Writer, applyChanges bool) error {
+func rebuildOneEcVolume(commandEnv *CommandEnv, rebuilder *EcNode, collection string, volumeId needle.VolumeId, locations EcShardLocations, writer io.Writer, applyChanges bool) error {
fmt.Printf("rebuildOneEcVolume %s %d\n", collection, volumeId)
// collect shard files to rebuilder local disk
var generatedShardIds []uint32
- copiedShardIds, _, err := prepareDataToRecover(ctx, commandEnv, rebuilder, collection, volumeId, locations, writer, applyChanges)
+ copiedShardIds, _, err := prepareDataToRecover(commandEnv, rebuilder, collection, volumeId, locations, writer, applyChanges)
if err != nil {
return err
}
@@ -139,7 +137,7 @@ func rebuildOneEcVolume(ctx context.Context, commandEnv *CommandEnv, rebuilder *
// clean up working files
// ask the rebuilder to delete the copied shards
- err = sourceServerDeleteEcShards(ctx, commandEnv.option.GrpcDialOption, collection, volumeId, rebuilder.info.Id, copiedShardIds)
+ err = sourceServerDeleteEcShards(commandEnv.option.GrpcDialOption, collection, volumeId, rebuilder.info.Id, copiedShardIds)
if err != nil {
fmt.Fprintf(writer, "%s delete copied ec shards %s %d.%v\n", rebuilder.info.Id, collection, volumeId, copiedShardIds)
}
@@ -151,13 +149,13 @@ func rebuildOneEcVolume(ctx context.Context, commandEnv *CommandEnv, rebuilder *
}
// generate ec shards, and maybe ecx file
- generatedShardIds, err = generateMissingShards(ctx, commandEnv.option.GrpcDialOption, collection, volumeId, rebuilder.info.Id)
+ generatedShardIds, err = generateMissingShards(commandEnv.option.GrpcDialOption, collection, volumeId, rebuilder.info.Id)
if err != nil {
return err
}
// mount the generated shards
- err = mountEcShards(ctx, commandEnv.option.GrpcDialOption, collection, volumeId, rebuilder.info.Id, generatedShardIds)
+ err = mountEcShards(commandEnv.option.GrpcDialOption, collection, volumeId, rebuilder.info.Id, generatedShardIds)
if err != nil {
return err
}
@@ -167,11 +165,10 @@ func rebuildOneEcVolume(ctx context.Context, commandEnv *CommandEnv, rebuilder *
return nil
}
-func generateMissingShards(ctx context.Context, grpcDialOption grpc.DialOption,
- collection string, volumeId needle.VolumeId, sourceLocation string) (rebuiltShardIds []uint32, err error) {
+func generateMissingShards(grpcDialOption grpc.DialOption, collection string, volumeId needle.VolumeId, sourceLocation string) (rebuiltShardIds []uint32, err error) {
err = operation.WithVolumeServerClient(sourceLocation, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
- resp, rebultErr := volumeServerClient.VolumeEcShardsRebuild(ctx, &volume_server_pb.VolumeEcShardsRebuildRequest{
+ resp, rebultErr := volumeServerClient.VolumeEcShardsRebuild(context.Background(), &volume_server_pb.VolumeEcShardsRebuildRequest{
VolumeId: uint32(volumeId),
Collection: collection,
})
@@ -183,7 +180,7 @@ func generateMissingShards(ctx context.Context, grpcDialOption grpc.DialOption,
return
}
-func prepareDataToRecover(ctx context.Context, commandEnv *CommandEnv, rebuilder *EcNode, collection string, volumeId needle.VolumeId, locations EcShardLocations, writer io.Writer, applyBalancing bool) (copiedShardIds []uint32, localShardIds []uint32, err error) {
+func prepareDataToRecover(commandEnv *CommandEnv, rebuilder *EcNode, collection string, volumeId needle.VolumeId, locations EcShardLocations, writer io.Writer, applyBalancing bool) (copiedShardIds []uint32, localShardIds []uint32, err error) {
needEcxFile := true
var localShardBits erasure_coding.ShardBits
@@ -210,11 +207,13 @@ func prepareDataToRecover(ctx context.Context, commandEnv *CommandEnv, rebuilder
var copyErr error
if applyBalancing {
copyErr = operation.WithVolumeServerClient(rebuilder.info.Id, commandEnv.option.GrpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
- _, copyErr := volumeServerClient.VolumeEcShardsCopy(ctx, &volume_server_pb.VolumeEcShardsCopyRequest{
+ _, copyErr := volumeServerClient.VolumeEcShardsCopy(context.Background(), &volume_server_pb.VolumeEcShardsCopyRequest{
VolumeId: uint32(volumeId),
Collection: collection,
ShardIds: []uint32{uint32(shardId)},
CopyEcxFile: needEcxFile,
+ CopyEcjFile: needEcxFile,
+ CopyVifFile: needEcxFile,
SourceDataNode: ecNodes[0].info.Id,
})
return copyErr
diff --git a/weed/shell/command_ec_test.go b/weed/shell/command_ec_test.go
index 9e578ed28..4fddcbea5 100644
--- a/weed/shell/command_ec_test.go
+++ b/weed/shell/command_ec_test.go
@@ -1,13 +1,25 @@
package shell
import (
- "context"
+ "fmt"
"testing"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
)
+func TestCommandEcDistribution(t *testing.T) {
+
+ allEcNodes := []*EcNode{
+ newEcNode("dc1", "rack1", "dn1", 100),
+ newEcNode("dc1", "rack2", "dn2", 100),
+ }
+
+ allocated := balancedEcDistribution(allEcNodes)
+
+ fmt.Printf("allocated: %+v", allocated)
+}
+
func TestCommandEcBalanceSmall(t *testing.T) {
allEcNodes := []*EcNode{
@@ -108,7 +120,7 @@ func TestCommandEcBalanceVolumeEvenButRackUneven(t *testing.T) {
racks := collectRacks(allEcNodes)
balanceEcVolumes(nil, "c1", allEcNodes, racks, false)
- balanceEcRacks(context.Background(), nil, racks, false)
+ balanceEcRacks(nil, racks, false)
}
func newEcNode(dc string, rack string, dataNodeId string, freeEcSlot int) *EcNode {
diff --git a/weed/shell/command_fs_cat.go b/weed/shell/command_fs_cat.go
index 66ced46c5..3db487979 100644
--- a/weed/shell/command_fs_cat.go
+++ b/weed/shell/command_fs_cat.go
@@ -1,7 +1,6 @@
package shell
import (
- "context"
"fmt"
"io"
"math"
@@ -24,12 +23,8 @@ func (c *commandFsCat) Name() string {
func (c *commandFsCat) Help() string {
return `stream the file content on to the screen
- fs.cat /dir/
fs.cat /dir/file_name
- fs.cat /dir/file_prefix
- fs.cat http://:/dir/
fs.cat http://:/dir/file_name
- fs.cat http://:/dir/file_prefix
`
}
@@ -42,21 +37,19 @@ func (c *commandFsCat) Do(args []string, commandEnv *CommandEnv, writer io.Write
return err
}
- ctx := context.Background()
-
- if commandEnv.isDirectory(ctx, filerServer, filerPort, path) {
+ if commandEnv.isDirectory(filerServer, filerPort, path) {
return fmt.Errorf("%s is a directory", path)
}
dir, name := filer2.FullPath(path).DirAndName()
- return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
+ return commandEnv.withFilerClient(filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.LookupDirectoryEntryRequest{
Name: name,
Directory: dir,
}
- respLookupEntry, err := client.LookupDirectoryEntry(ctx, request)
+ respLookupEntry, err := filer_pb.LookupEntry(client, request)
if err != nil {
return err
}
diff --git a/weed/shell/command_fs_cd.go b/weed/shell/command_fs_cd.go
index 408ec86c8..df42cd516 100644
--- a/weed/shell/command_fs_cd.go
+++ b/weed/shell/command_fs_cd.go
@@ -1,7 +1,6 @@
package shell
import (
- "context"
"io"
)
@@ -45,9 +44,7 @@ func (c *commandFsCd) Do(args []string, commandEnv *CommandEnv, writer io.Writer
return nil
}
- ctx := context.Background()
-
- err = commandEnv.checkDirectory(ctx, filerServer, filerPort, path)
+ err = commandEnv.checkDirectory(filerServer, filerPort, path)
if err == nil {
commandEnv.option.FilerHost = filerServer
diff --git a/weed/shell/command_fs_du.go b/weed/shell/command_fs_du.go
index 5e634c82a..ca2f22b57 100644
--- a/weed/shell/command_fs_du.go
+++ b/weed/shell/command_fs_du.go
@@ -1,13 +1,12 @@
package shell
import (
- "context"
"fmt"
+ "io"
+
"github.com/chrislusf/seaweedfs/weed/filer2"
+ "github.com/chrislusf/seaweedfs/weed/pb"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
- "github.com/chrislusf/seaweedfs/weed/util"
- "google.golang.org/grpc"
- "io"
)
func init() {
@@ -37,81 +36,70 @@ func (c *commandFsDu) Do(args []string, commandEnv *CommandEnv, writer io.Writer
return err
}
- ctx := context.Background()
-
- if commandEnv.isDirectory(ctx, filerServer, filerPort, path) {
+ if commandEnv.isDirectory(filerServer, filerPort, path) {
path = path + "/"
}
+ var blockCount, byteCount uint64
dir, name := filer2.FullPath(path).DirAndName()
+ blockCount, byteCount, err = duTraverseDirectory(writer, commandEnv.getFilerClient(filerServer, filerPort), dir, name)
- return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
-
- _, _, err = paginateDirectory(ctx, writer, client, dir, name, 1000)
-
- return err
+ if name == "" && err == nil {
+ fmt.Fprintf(writer, "block:%4d\tbyte:%10d\t%s\n", blockCount, byteCount, dir)
+ }
- })
+ return
}
-func paginateDirectory(ctx context.Context, writer io.Writer, client filer_pb.SeaweedFilerClient, dir, name string, paginateSize int) (blockCount uint64, byteCount uint64, err error) {
-
- paginatedCount := -1
- startFromFileName := ""
-
- for paginatedCount == -1 || paginatedCount == paginateSize {
- resp, listErr := client.ListEntries(ctx, &filer_pb.ListEntriesRequest{
- Directory: dir,
- Prefix: name,
- StartFromFileName: startFromFileName,
- InclusiveStartFrom: false,
- Limit: uint32(paginateSize),
- })
- if listErr != nil {
- err = listErr
- return
- }
+func duTraverseDirectory(writer io.Writer, filerClient filer2.FilerClient, dir, name string) (blockCount, byteCount uint64, err error) {
- paginatedCount = len(resp.Entries)
-
- for _, entry := range resp.Entries {
- if entry.IsDirectory {
- subDir := fmt.Sprintf("%s/%s", dir, entry.Name)
- if dir == "/" {
- subDir = "/" + entry.Name
- }
- numBlock, numByte, err := paginateDirectory(ctx, writer, client, subDir, "", paginateSize)
- if err == nil {
- blockCount += numBlock
- byteCount += numByte
- }
- } else {
- blockCount += uint64(len(entry.Chunks))
- byteCount += filer2.TotalSize(entry.Chunks)
+ err = filer2.ReadDirAllEntries(filerClient, filer2.FullPath(dir), name, func(entry *filer_pb.Entry, isLast bool) {
+ if entry.IsDirectory {
+ subDir := fmt.Sprintf("%s/%s", dir, entry.Name)
+ if dir == "/" {
+ subDir = "/" + entry.Name
}
- startFromFileName = entry.Name
-
- if name != "" && !entry.IsDirectory {
- fmt.Fprintf(writer, "block:%4d\tbyte:%10d\t%s/%s\n", blockCount, byteCount, dir, name)
+ numBlock, numByte, err := duTraverseDirectory(writer, filerClient, subDir, "")
+ if err == nil {
+ blockCount += numBlock
+ byteCount += numByte
}
+ } else {
+ blockCount += uint64(len(entry.Chunks))
+ byteCount += filer2.TotalSize(entry.Chunks)
}
- }
-
- if name == "" {
- fmt.Fprintf(writer, "block:%4d\tbyte:%10d\t%s\n", blockCount, byteCount, dir)
- }
+ if name != "" && !entry.IsDirectory {
+ fmt.Fprintf(writer, "block:%4d\tbyte:%10d\t%s/%s\n", blockCount, byteCount, dir, name)
+ }
+ })
return
-
}
-func (env *CommandEnv) withFilerClient(ctx context.Context, filerServer string, filerPort int64, fn func(filer_pb.SeaweedFilerClient) error) error {
+func (env *CommandEnv) withFilerClient(filerServer string, filerPort int64, fn func(filer_pb.SeaweedFilerClient) error) error {
filerGrpcAddress := fmt.Sprintf("%s:%d", filerServer, filerPort+10000)
- return util.WithCachedGrpcClient(ctx, func(grpcConnection *grpc.ClientConn) error {
- client := filer_pb.NewSeaweedFilerClient(grpcConnection)
- return fn(client)
- }, filerGrpcAddress, env.option.GrpcDialOption)
+ return pb.WithGrpcFilerClient(filerGrpcAddress, env.option.GrpcDialOption, fn)
}
+
+type commandFilerClient struct {
+ env *CommandEnv
+ filerServer string
+ filerPort int64
+}
+
+func (env *CommandEnv) getFilerClient(filerServer string, filerPort int64) *commandFilerClient {
+ return &commandFilerClient{
+ env: env,
+ filerServer: filerServer,
+ filerPort: filerPort,
+ }
+}
+func (c *commandFilerClient) WithFilerClient(fn func(filer_pb.SeaweedFilerClient) error) error {
+ return c.env.withFilerClient(c.filerServer, c.filerPort, fn)
+}
+func (c *commandFilerClient) AdjustedUrl(hostAndPort string) string {
+ return hostAndPort
+}
diff --git a/weed/shell/command_fs_ls.go b/weed/shell/command_fs_ls.go
index 6979635e1..69ebe1b30 100644
--- a/weed/shell/command_fs_ls.go
+++ b/weed/shell/command_fs_ls.go
@@ -1,15 +1,15 @@
package shell
import (
- "context"
"fmt"
- "github.com/chrislusf/seaweedfs/weed/filer2"
- "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"io"
"os"
"os/user"
"strconv"
"strings"
+
+ "github.com/chrislusf/seaweedfs/weed/filer2"
+ "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
)
func init() {
@@ -59,90 +59,56 @@ func (c *commandFsLs) Do(args []string, commandEnv *CommandEnv, writer io.Writer
return err
}
- ctx := context.Background()
-
- if commandEnv.isDirectory(ctx, filerServer, filerPort, path) {
+ if commandEnv.isDirectory(filerServer, filerPort, path) {
path = path + "/"
}
dir, name := filer2.FullPath(path).DirAndName()
+ entryCount := 0
- return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
-
- return paginateOneDirectory(ctx, writer, client, dir, name, 1000, isLongFormat, showHidden)
-
- })
-
-}
-
-func paginateOneDirectory(ctx context.Context, writer io.Writer, client filer_pb.SeaweedFilerClient, dir, name string, paginateSize int, isLongFormat, showHidden bool) (err error) {
+ err = filer2.ReadDirAllEntries(commandEnv.getFilerClient(filerServer, filerPort), filer2.FullPath(dir), name, func(entry *filer_pb.Entry, isLast bool) {
- entryCount := 0
- paginatedCount := -1
- startFromFileName := ""
-
- for paginatedCount == -1 || paginatedCount == paginateSize {
- resp, listErr := client.ListEntries(ctx, &filer_pb.ListEntriesRequest{
- Directory: dir,
- Prefix: name,
- StartFromFileName: startFromFileName,
- InclusiveStartFrom: false,
- Limit: uint32(paginateSize),
- })
- if listErr != nil {
- err = listErr
+ if !showHidden && strings.HasPrefix(entry.Name, ".") {
return
}
- paginatedCount = len(resp.Entries)
-
- for _, entry := range resp.Entries {
+ entryCount++
- if !showHidden && strings.HasPrefix(entry.Name, ".") {
- continue
- }
-
- entryCount++
-
- if isLongFormat {
- fileMode := os.FileMode(entry.Attributes.FileMode)
- userName, groupNames := entry.Attributes.UserName, entry.Attributes.GroupName
- if userName == "" {
- if user, userErr := user.LookupId(strconv.Itoa(int(entry.Attributes.Uid))); userErr == nil {
- userName = user.Username
- }
- }
- groupName := ""
- if len(groupNames) > 0 {
- groupName = groupNames[0]
+ if isLongFormat {
+ fileMode := os.FileMode(entry.Attributes.FileMode)
+ userName, groupNames := entry.Attributes.UserName, entry.Attributes.GroupName
+ if userName == "" {
+ if user, userErr := user.LookupId(strconv.Itoa(int(entry.Attributes.Uid))); userErr == nil {
+ userName = user.Username
}
- if groupName == "" {
- if group, groupErr := user.LookupGroupId(strconv.Itoa(int(entry.Attributes.Gid))); groupErr == nil {
- groupName = group.Name
- }
- }
-
- if dir == "/" {
- // just for printing
- dir = ""
+ }
+ groupName := ""
+ if len(groupNames) > 0 {
+ groupName = groupNames[0]
+ }
+ if groupName == "" {
+ if group, groupErr := user.LookupGroupId(strconv.Itoa(int(entry.Attributes.Gid))); groupErr == nil {
+ groupName = group.Name
}
- fmt.Fprintf(writer, "%s %3d %s %s %6d %s/%s\n",
- fileMode, len(entry.Chunks),
- userName, groupName,
- filer2.TotalSize(entry.Chunks), dir, entry.Name)
- } else {
- fmt.Fprintf(writer, "%s\n", entry.Name)
}
- startFromFileName = entry.Name
-
+ if dir == "/" {
+ // just for printing
+ dir = ""
+ }
+ fmt.Fprintf(writer, "%s %3d %s %s %6d %s/%s\n",
+ fileMode, len(entry.Chunks),
+ userName, groupName,
+ filer2.TotalSize(entry.Chunks), dir, entry.Name)
+ } else {
+ fmt.Fprintf(writer, "%s\n", entry.Name)
}
- }
- if isLongFormat {
+ })
+
+ if isLongFormat && err == nil {
fmt.Fprintf(writer, "total %d\n", entryCount)
}
return
-
}
diff --git a/weed/shell/command_fs_meta_cat.go b/weed/shell/command_fs_meta_cat.go
new file mode 100644
index 000000000..cd1ffb6fd
--- /dev/null
+++ b/weed/shell/command_fs_meta_cat.go
@@ -0,0 +1,72 @@
+package shell
+
+import (
+ "fmt"
+ "io"
+
+ "github.com/golang/protobuf/jsonpb"
+
+ "github.com/chrislusf/seaweedfs/weed/filer2"
+ "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
+)
+
+func init() {
+ Commands = append(Commands, &commandFsMetaCat{})
+}
+
+type commandFsMetaCat struct {
+}
+
+func (c *commandFsMetaCat) Name() string {
+ return "fs.meta.cat"
+}
+
+func (c *commandFsMetaCat) Help() string {
+ return `print out the meta data content for a file or directory
+
+ fs.meta.cat /dir/
+ fs.meta.cat /dir/file_name
+ fs.meta.cat http://:/dir/
+ fs.meta.cat http://:/dir/file_name
+`
+}
+
+func (c *commandFsMetaCat) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+
+ input := findInputDirectory(args)
+
+ filerServer, filerPort, path, err := commandEnv.parseUrl(input)
+ if err != nil {
+ return err
+ }
+
+ dir, name := filer2.FullPath(path).DirAndName()
+
+ return commandEnv.withFilerClient(filerServer, filerPort, 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
+ }
+
+ m := jsonpb.Marshaler{
+ EmitDefaults: true,
+ Indent: " ",
+ }
+
+ text, marshalErr := m.MarshalToString(respLookupEntry.Entry)
+ if marshalErr != nil {
+ return fmt.Errorf("marshal meta: %v", marshalErr)
+ }
+
+ fmt.Fprintf(writer, "%s\n", text)
+
+ return nil
+
+ })
+
+}
diff --git a/weed/shell/command_fs_meta_load.go b/weed/shell/command_fs_meta_load.go
index 5ea8de9f5..ed92d8011 100644
--- a/weed/shell/command_fs_meta_load.go
+++ b/weed/shell/command_fs_meta_load.go
@@ -1,15 +1,15 @@
package shell
import (
- "context"
"fmt"
"io"
"os"
+ "github.com/golang/protobuf/proto"
+
"github.com/chrislusf/seaweedfs/weed/filer2"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/util"
- "github.com/golang/protobuf/proto"
)
func init() {
@@ -53,9 +53,7 @@ func (c *commandFsMetaLoad) Do(args []string, commandEnv *CommandEnv, writer io.
var dirCount, fileCount uint64
- ctx := context.Background()
-
- err = commandEnv.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
+ err = commandEnv.withFilerClient(filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
sizeBuf := make([]byte, 4)
@@ -80,7 +78,7 @@ func (c *commandFsMetaLoad) Do(args []string, commandEnv *CommandEnv, writer io.
return err
}
- if _, err = client.CreateEntry(ctx, &filer_pb.CreateEntryRequest{
+ if err := filer_pb.CreateEntry(client, &filer_pb.CreateEntryRequest{
Directory: fullEntry.Dir,
Entry: fullEntry.Entry,
}); err != nil {
diff --git a/weed/shell/command_fs_meta_notify.go b/weed/shell/command_fs_meta_notify.go
index 13b272fbf..099e04506 100644
--- a/weed/shell/command_fs_meta_notify.go
+++ b/weed/shell/command_fs_meta_notify.go
@@ -1,7 +1,6 @@
package shell
import (
- "context"
"fmt"
"io"
@@ -9,7 +8,6 @@ import (
"github.com/chrislusf/seaweedfs/weed/notification"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/util"
- "github.com/spf13/viper"
)
func init() {
@@ -41,38 +39,36 @@ func (c *commandFsMetaNotify) Do(args []string, commandEnv *CommandEnv, writer i
}
util.LoadConfiguration("notification", true)
- v := viper.GetViper()
- notification.LoadConfiguration(v.Sub("notification"))
+ v := util.GetViper()
+ notification.LoadConfiguration(v, "notification.")
- ctx := context.Background()
+ var dirCount, fileCount uint64
- return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
+ err = doTraverseBFS(writer, commandEnv.getFilerClient(filerServer, filerPort), filer2.FullPath(path), func(parentPath filer2.FullPath, entry *filer_pb.Entry) {
- var dirCount, fileCount uint64
-
- err = doTraverse(ctx, writer, client, filer2.FullPath(path), func(parentPath filer2.FullPath, entry *filer_pb.Entry) error {
-
- if entry.IsDirectory {
- dirCount++
- } else {
- fileCount++
- }
-
- return notification.Queue.SendMessage(
- string(parentPath.Child(entry.Name)),
- &filer_pb.EventNotification{
- NewEntry: entry,
- },
- )
+ if entry.IsDirectory {
+ dirCount++
+ } else {
+ fileCount++
+ }
- })
+ notifyErr := notification.Queue.SendMessage(
+ string(parentPath.Child(entry.Name)),
+ &filer_pb.EventNotification{
+ NewEntry: entry,
+ },
+ )
- if err == nil {
- fmt.Fprintf(writer, "\ntotal notified %d directories, %d files\n", dirCount, fileCount)
+ if notifyErr != nil {
+ fmt.Fprintf(writer, "fail to notify new entry event for %s: %v\n", parentPath.Child(entry.Name), notifyErr)
}
- return err
-
})
+ if err == nil {
+ fmt.Fprintf(writer, "\ntotal notified %d directories, %d files\n", dirCount, fileCount)
+ }
+
+ return err
+
}
diff --git a/weed/shell/command_fs_meta_save.go b/weed/shell/command_fs_meta_save.go
index e710fe297..b51fdd0f6 100644
--- a/weed/shell/command_fs_meta_save.go
+++ b/weed/shell/command_fs_meta_save.go
@@ -1,17 +1,19 @@
package shell
import (
- "context"
"flag"
"fmt"
"io"
"os"
+ "sync"
+ "sync/atomic"
"time"
+ "github.com/golang/protobuf/proto"
+
"github.com/chrislusf/seaweedfs/weed/filer2"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/util"
- "github.com/golang/protobuf/proto"
)
func init() {
@@ -51,114 +53,127 @@ func (c *commandFsMetaSave) Do(args []string, commandEnv *CommandEnv, writer io.
return nil
}
- filerServer, filerPort, path, err := commandEnv.parseUrl(findInputDirectory(fsMetaSaveCommand.Args()))
- if err != nil {
- return err
+ filerServer, filerPort, path, parseErr := commandEnv.parseUrl(findInputDirectory(fsMetaSaveCommand.Args()))
+ if parseErr != nil {
+ return parseErr
}
- ctx := context.Background()
+ t := time.Now()
+ fileName := *outputFileName
+ if fileName == "" {
+ fileName = fmt.Sprintf("%s-%d-%4d%02d%02d-%02d%02d%02d.meta",
+ filerServer, filerPort, t.Year(), t.Month(), t.Day(), t.Hour(), t.Minute(), t.Second())
+ }
- return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
+ dst, openErr := os.OpenFile(fileName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644)
+ if openErr != nil {
+ return fmt.Errorf("failed to create file %s: %v", fileName, openErr)
+ }
+ defer dst.Close()
- t := time.Now()
- fileName := *outputFileName
- if fileName == "" {
- fileName = fmt.Sprintf("%s-%d-%4d%02d%02d-%02d%02d%02d.meta",
- filerServer, filerPort, t.Year(), t.Month(), t.Day(), t.Hour(), t.Minute(), t.Second())
+ var wg sync.WaitGroup
+ wg.Add(1)
+ outputChan := make(chan []byte, 1024)
+ go func() {
+ sizeBuf := make([]byte, 4)
+ for b := range outputChan {
+ util.Uint32toBytes(sizeBuf, uint32(len(b)))
+ dst.Write(sizeBuf)
+ dst.Write(b)
}
+ wg.Done()
+ }()
- dst, err := os.OpenFile(fileName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644)
- if err != nil {
- return nil
- }
- defer dst.Close()
+ var dirCount, fileCount uint64
- var dirCount, fileCount uint64
+ err = doTraverseBFS(writer, commandEnv.getFilerClient(filerServer, filerPort), filer2.FullPath(path), func(parentPath filer2.FullPath, entry *filer_pb.Entry) {
- sizeBuf := make([]byte, 4)
+ protoMessage := &filer_pb.FullEntry{
+ Dir: string(parentPath),
+ Entry: entry,
+ }
- err = doTraverse(ctx, writer, client, filer2.FullPath(path), func(parentPath filer2.FullPath, entry *filer_pb.Entry) error {
+ bytes, err := proto.Marshal(protoMessage)
+ if err != nil {
+ fmt.Fprintf(writer, "marshall error: %v\n", err)
+ return
+ }
- protoMessage := &filer_pb.FullEntry{
- Dir: string(parentPath),
- Entry: entry,
- }
+ outputChan <- bytes
- bytes, err := proto.Marshal(protoMessage)
- if err != nil {
- return fmt.Errorf("marshall error: %v", err)
- }
+ if entry.IsDirectory {
+ atomic.AddUint64(&dirCount, 1)
+ } else {
+ atomic.AddUint64(&fileCount, 1)
+ }
- util.Uint32toBytes(sizeBuf, uint32(len(bytes)))
+ if *verbose {
+ println(parentPath.Child(entry.Name))
+ }
- dst.Write(sizeBuf)
- dst.Write(bytes)
+ })
- if entry.IsDirectory {
- dirCount++
- } else {
- fileCount++
- }
+ close(outputChan)
- if *verbose {
- println(parentPath.Child(entry.Name))
- }
+ wg.Wait()
- return nil
+ if err == nil {
+ fmt.Fprintf(writer, "total %d directories, %d files\n", dirCount, fileCount)
+ fmt.Fprintf(writer, "meta data for http://%s:%d%s is saved to %s\n", filerServer, filerPort, path, fileName)
+ }
- })
+ return err
- if err == nil {
- fmt.Fprintf(writer, "\ntotal %d directories, %d files", dirCount, fileCount)
- fmt.Fprintf(writer, "\nmeta data for http://%s:%d%s is saved to %s\n", filerServer, filerPort, path, fileName)
- }
+}
+func doTraverseBFS(writer io.Writer, filerClient filer2.FilerClient, parentPath filer2.FullPath, fn func(parentPath filer2.FullPath, entry *filer_pb.Entry)) (err error) {
- return err
+ K := 5
- })
+ var jobQueueWg sync.WaitGroup
+ queue := util.NewQueue()
+ jobQueueWg.Add(1)
+ queue.Enqueue(parentPath)
+ var isTerminating bool
+ for i := 0; i < K; i++ {
+ go func() {
+ for {
+ if isTerminating {
+ break
+ }
+ t := queue.Dequeue()
+ if t == nil {
+ time.Sleep(329 * time.Millisecond)
+ continue
+ }
+ dir := t.(filer2.FullPath)
+ processErr := processOneDirectory(writer, filerClient, dir, queue, &jobQueueWg, fn)
+ if processErr != nil {
+ err = processErr
+ }
+ jobQueueWg.Done()
+ }
+ }()
+ }
+ jobQueueWg.Wait()
+ isTerminating = true
+ return
}
-func doTraverse(ctx context.Context, writer io.Writer, client filer_pb.SeaweedFilerClient, parentPath filer2.FullPath, fn func(parentPath filer2.FullPath, entry *filer_pb.Entry) error) (err error) {
-
- paginatedCount := -1
- startFromFileName := ""
- paginateSize := 1000
-
- for paginatedCount == -1 || paginatedCount == paginateSize {
- resp, listErr := client.ListEntries(ctx, &filer_pb.ListEntriesRequest{
- Directory: string(parentPath),
- Prefix: "",
- StartFromFileName: startFromFileName,
- InclusiveStartFrom: false,
- Limit: uint32(paginateSize),
- })
- if listErr != nil {
- err = listErr
- return
- }
- paginatedCount = len(resp.Entries)
+func processOneDirectory(writer io.Writer, filerClient filer2.FilerClient, parentPath filer2.FullPath, queue *util.Queue, jobQueueWg *sync.WaitGroup, fn func(parentPath filer2.FullPath, entry *filer_pb.Entry)) (err error) {
- for _, entry := range resp.Entries {
+ return filer2.ReadDirAllEntries(filerClient, parentPath, "", func(entry *filer_pb.Entry, isLast bool) {
- if err = fn(parentPath, entry); err != nil {
- return err
- }
+ fn(parentPath, entry)
- if entry.IsDirectory {
- subDir := fmt.Sprintf("%s/%s", parentPath, entry.Name)
- if parentPath == "/" {
- subDir = "/" + entry.Name
- }
- if err = doTraverse(ctx, writer, client, filer2.FullPath(subDir), fn); err != nil {
- return err
- }
+ if entry.IsDirectory {
+ subDir := fmt.Sprintf("%s/%s", parentPath, entry.Name)
+ if parentPath == "/" {
+ subDir = "/" + entry.Name
}
- startFromFileName = entry.Name
-
+ jobQueueWg.Add(1)
+ queue.Enqueue(filer2.FullPath(subDir))
}
- }
-
- return
+ })
}
diff --git a/weed/shell/command_fs_mv.go b/weed/shell/command_fs_mv.go
index 67606ab53..85275058e 100644
--- a/weed/shell/command_fs_mv.go
+++ b/weed/shell/command_fs_mv.go
@@ -47,20 +47,18 @@ func (c *commandFsMv) Do(args []string, commandEnv *CommandEnv, writer io.Writer
return err
}
- ctx := context.Background()
-
sourceDir, sourceName := filer2.FullPath(sourcePath).DirAndName()
destinationDir, destinationName := filer2.FullPath(destinationPath).DirAndName()
- return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
+ return commandEnv.withFilerClient(filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
// collect destination entry info
destinationRequest := &filer_pb.LookupDirectoryEntryRequest{
Name: destinationDir,
Directory: destinationName,
}
- respDestinationLookupEntry, err := client.LookupDirectoryEntry(ctx, destinationRequest)
+ respDestinationLookupEntry, err := filer_pb.LookupEntry(client, destinationRequest)
var targetDir, targetName string
@@ -82,7 +80,7 @@ func (c *commandFsMv) Do(args []string, commandEnv *CommandEnv, writer io.Writer
NewName: targetName,
}
- _, err = client.AtomicRenameEntry(ctx, request)
+ _, err = client.AtomicRenameEntry(context.Background(), request)
fmt.Fprintf(writer, "move: %s => %s\n", sourcePath, filer2.NewFullPath(targetDir, targetName))
diff --git a/weed/shell/command_fs_tree.go b/weed/shell/command_fs_tree.go
index 8474e43ea..04530571c 100644
--- a/weed/shell/command_fs_tree.go
+++ b/weed/shell/command_fs_tree.go
@@ -1,12 +1,12 @@
package shell
import (
- "context"
"fmt"
- "github.com/chrislusf/seaweedfs/weed/filer2"
- "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"io"
"strings"
+
+ "github.com/chrislusf/seaweedfs/weed/filer2"
+ "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
)
func init() {
@@ -36,77 +36,42 @@ func (c *commandFsTree) Do(args []string, commandEnv *CommandEnv, writer io.Writ
dir, name := filer2.FullPath(path).DirAndName()
- ctx := context.Background()
+ dirCount, fCount, terr := treeTraverseDirectory(writer, commandEnv.getFilerClient(filerServer, filerPort), filer2.FullPath(dir), name, newPrefix(), -1)
- return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
-
- dirCount, fCount, terr := treeTraverseDirectory(ctx, writer, client, dir, name, newPrefix(), -1)
-
- if terr == nil {
- fmt.Fprintf(writer, "%d directories, %d files\n", dirCount, fCount)
- }
+ if terr == nil {
+ fmt.Fprintf(writer, "%d directories, %d files\n", dirCount, fCount)
+ }
- return terr
-
- })
+ return terr
}
-func treeTraverseDirectory(ctx context.Context, writer io.Writer, client filer_pb.SeaweedFilerClient, dir, name string, prefix *Prefix, level int) (directoryCount, fileCount int64, err error) {
-
- paginatedCount := -1
- startFromFileName := ""
- paginateSize := 1000
-
- for paginatedCount == -1 || paginatedCount == paginateSize {
- resp, listErr := client.ListEntries(ctx, &filer_pb.ListEntriesRequest{
- Directory: dir,
- Prefix: name,
- StartFromFileName: startFromFileName,
- InclusiveStartFrom: false,
- Limit: uint32(paginateSize),
- })
- if listErr != nil {
- err = listErr
- return
- }
- paginatedCount = len(resp.Entries)
- if paginatedCount > 0 {
- prefix.addMarker(level)
- }
+func treeTraverseDirectory(writer io.Writer, filerClient filer2.FilerClient, dir filer2.FullPath, name string, prefix *Prefix, level int) (directoryCount, fileCount int64, err error) {
- for i, entry := range resp.Entries {
+ prefix.addMarker(level)
- if level < 0 && name != "" {
- if entry.Name != name {
- break
- }
+ err = filer2.ReadDirAllEntries(filerClient, dir, name, func(entry *filer_pb.Entry, isLast bool) {
+ if level < 0 && name != "" {
+ if entry.Name != name {
+ return
}
+ }
- // 0.1% wrong prefix here, but fixing it would need to paginate to the next batch first
- isLast := paginatedCount < paginateSize && i == paginatedCount-1
- fmt.Fprintf(writer, "%s%s\n", prefix.getPrefix(level, isLast), entry.Name)
-
- if entry.IsDirectory {
- directoryCount++
- subDir := fmt.Sprintf("%s/%s", dir, entry.Name)
- if dir == "/" {
- subDir = "/" + entry.Name
- }
- dirCount, fCount, terr := treeTraverseDirectory(ctx, writer, client, subDir, "", prefix, level+1)
- directoryCount += dirCount
- fileCount += fCount
- err = terr
- } else {
- fileCount++
- }
- startFromFileName = entry.Name
+ fmt.Fprintf(writer, "%s%s\n", prefix.getPrefix(level, isLast), entry.Name)
+ if entry.IsDirectory {
+ directoryCount++
+ subDir := dir.Child(entry.Name)
+ dirCount, fCount, terr := treeTraverseDirectory(writer, filerClient, subDir, "", prefix, level+1)
+ directoryCount += dirCount
+ fileCount += fCount
+ err = terr
+ } else {
+ fileCount++
}
- }
+ })
return
-
}
type Prefix struct {
diff --git a/weed/shell/command_volume_balance.go b/weed/shell/command_volume_balance.go
index d7ef0d005..349f52f1c 100644
--- a/weed/shell/command_volume_balance.go
+++ b/weed/shell/command_volume_balance.go
@@ -27,7 +27,7 @@ func (c *commandVolumeBalance) Name() string {
func (c *commandVolumeBalance) Help() string {
return `balance all volumes among volume servers
- volume.balance [-c ALL|EACH_COLLECTION|] [-force] [-dataCenter=]
+ volume.balance [-collection ALL|EACH_COLLECTION|] [-force] [-dataCenter=]
Algorithm:
@@ -69,9 +69,8 @@ func (c *commandVolumeBalance) Do(args []string, commandEnv *CommandEnv, writer
}
var resp *master_pb.VolumeListResponse
- ctx := context.Background()
- err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
- resp, err = client.VolumeList(ctx, &master_pb.VolumeListRequest{})
+ err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
+ resp, err = client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
return err
})
if err != nil {
@@ -79,8 +78,10 @@ func (c *commandVolumeBalance) Do(args []string, commandEnv *CommandEnv, writer
}
typeToNodes := collectVolumeServersByType(resp.TopologyInfo, *dc)
- for _, volumeServers := range typeToNodes {
+
+ for maxVolumeCount, volumeServers := range typeToNodes {
if len(volumeServers) < 2 {
+ fmt.Printf("only 1 node is configured max %d volumes, skipping balancing\n", maxVolumeCount)
continue
}
if *collection == "EACH_COLLECTION" {
@@ -93,8 +94,8 @@ func (c *commandVolumeBalance) Do(args []string, commandEnv *CommandEnv, writer
return err
}
}
- } else if *collection == "ALL" {
- if err = balanceVolumeServers(commandEnv, volumeServers, resp.VolumeSizeLimitMb*1024*1024, "ALL", *applyBalancing); err != nil {
+ } else if *collection == "ALL_COLLECTIONS" {
+ if err = balanceVolumeServers(commandEnv, volumeServers, resp.VolumeSizeLimitMb*1024*1024, "ALL_COLLECTIONS", *applyBalancing); err != nil {
return err
}
} else {
@@ -107,18 +108,12 @@ func (c *commandVolumeBalance) Do(args []string, commandEnv *CommandEnv, writer
return nil
}
-func balanceVolumeServers(commandEnv *CommandEnv, dataNodeInfos []*master_pb.DataNodeInfo, volumeSizeLimit uint64, collection string, applyBalancing bool) error {
- var nodes []*Node
- for _, dn := range dataNodeInfos {
- nodes = append(nodes, &Node{
- info: dn,
- })
- }
+func balanceVolumeServers(commandEnv *CommandEnv, nodes []*Node, volumeSizeLimit uint64, collection string, applyBalancing bool) error {
// balance writable volumes
for _, n := range nodes {
n.selectVolumes(func(v *master_pb.VolumeInformationMessage) bool {
- if collection != "ALL" {
+ if collection != "ALL_COLLECTIONS" {
if v.Collection != collection {
return false
}
@@ -133,7 +128,7 @@ func balanceVolumeServers(commandEnv *CommandEnv, dataNodeInfos []*master_pb.Dat
// balance readable volumes
for _, n := range nodes {
n.selectVolumes(func(v *master_pb.VolumeInformationMessage) bool {
- if collection != "ALL" {
+ if collection != "ALL_COLLECTIONS" {
if v.Collection != collection {
return false
}
@@ -148,15 +143,19 @@ func balanceVolumeServers(commandEnv *CommandEnv, dataNodeInfos []*master_pb.Dat
return nil
}
-func collectVolumeServersByType(t *master_pb.TopologyInfo, selectedDataCenter string) (typeToNodes map[uint64][]*master_pb.DataNodeInfo) {
- typeToNodes = make(map[uint64][]*master_pb.DataNodeInfo)
+func collectVolumeServersByType(t *master_pb.TopologyInfo, selectedDataCenter string) (typeToNodes map[uint64][]*Node) {
+ typeToNodes = make(map[uint64][]*Node)
for _, dc := range t.DataCenterInfos {
if selectedDataCenter != "" && dc.Id != selectedDataCenter {
continue
}
for _, r := range dc.RackInfos {
for _, dn := range r.DataNodeInfos {
- typeToNodes[dn.MaxVolumeCount] = append(typeToNodes[dn.MaxVolumeCount], dn)
+ typeToNodes[dn.MaxVolumeCount] = append(typeToNodes[dn.MaxVolumeCount], &Node{
+ info: dn,
+ dc: dc.Id,
+ rack: r.Id,
+ })
}
}
}
@@ -166,6 +165,8 @@ func collectVolumeServersByType(t *master_pb.TopologyInfo, selectedDataCenter st
type Node struct {
info *master_pb.DataNodeInfo
selectedVolumes map[uint32]*master_pb.VolumeInformationMessage
+ dc string
+ rack string
}
func sortWritableVolumes(volumes []*master_pb.VolumeInformationMessage) {
@@ -207,6 +208,13 @@ func balanceSelectedVolume(commandEnv *CommandEnv, nodes []*Node, sortCandidates
sortCandidatesFn(candidateVolumes)
for _, v := range candidateVolumes {
+ if v.ReplicaPlacement > 0 {
+ if fullNode.dc != emptyNode.dc && fullNode.rack != emptyNode.rack {
+ // TODO this logic is too simple, but should work most of the time
+ // Need a correct algorithm to handle all different cases
+ continue
+ }
+ }
if _, found := emptyNode.selectedVolumes[v.Id]; !found {
if err := moveVolume(commandEnv, v, fullNode, emptyNode, applyBalancing); err == nil {
delete(fullNode.selectedVolumes, v.Id)
@@ -230,8 +238,7 @@ func moveVolume(commandEnv *CommandEnv, v *master_pb.VolumeInformationMessage, f
}
fmt.Fprintf(os.Stdout, "moving volume %s%d %s => %s\n", collectionPrefix, v.Id, fullNode.info.Id, emptyNode.info.Id)
if applyBalancing {
- ctx := context.Background()
- return LiveMoveVolume(ctx, commandEnv.option.GrpcDialOption, needle.VolumeId(v.Id), fullNode.info.Id, emptyNode.info.Id, 5*time.Second)
+ return LiveMoveVolume(commandEnv.option.GrpcDialOption, needle.VolumeId(v.Id), fullNode.info.Id, emptyNode.info.Id, 5*time.Second)
}
return nil
}
diff --git a/weed/shell/command_volume_configure_replication.go b/weed/shell/command_volume_configure_replication.go
new file mode 100644
index 000000000..133ec62c6
--- /dev/null
+++ b/weed/shell/command_volume_configure_replication.go
@@ -0,0 +1,104 @@
+package shell
+
+import (
+ "context"
+ "errors"
+ "flag"
+ "fmt"
+ "io"
+
+ "github.com/chrislusf/seaweedfs/weed/operation"
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
+)
+
+func init() {
+ Commands = append(Commands, &commandVolumeConfigureReplication{})
+}
+
+type commandVolumeConfigureReplication struct {
+}
+
+func (c *commandVolumeConfigureReplication) Name() string {
+ return "volume.configure.replication"
+}
+
+func (c *commandVolumeConfigureReplication) Help() string {
+ return `change volume replication value
+
+ This command changes a volume replication value. It should be followed by volume.fix.replication.
+
+`
+}
+
+func (c *commandVolumeConfigureReplication) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+
+ configureReplicationCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
+ volumeIdInt := configureReplicationCommand.Int("volumeId", 0, "the volume id")
+ replicationString := configureReplicationCommand.String("replication", "", "the intended replication value")
+ if err = configureReplicationCommand.Parse(args); err != nil {
+ return nil
+ }
+
+ if *replicationString == "" {
+ return fmt.Errorf("empty replication value")
+ }
+
+ replicaPlacement, err := super_block.NewReplicaPlacementFromString(*replicationString)
+ if err != nil {
+ return fmt.Errorf("replication format: %v", err)
+ }
+ replicaPlacementInt32 := uint32(replicaPlacement.Byte())
+
+ var resp *master_pb.VolumeListResponse
+ err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
+ resp, err = client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
+ return err
+ })
+ if err != nil {
+ return err
+ }
+
+ vid := needle.VolumeId(*volumeIdInt)
+
+ // find all data nodes with volumes that needs replication change
+ var allLocations []location
+ eachDataNode(resp.TopologyInfo, func(dc string, rack RackId, dn *master_pb.DataNodeInfo) {
+ loc := newLocation(dc, string(rack), dn)
+ for _, v := range dn.VolumeInfos {
+ if v.Id == uint32(vid) && v.ReplicaPlacement != replicaPlacementInt32 {
+ allLocations = append(allLocations, loc)
+ continue
+ }
+ }
+ })
+
+ if len(allLocations) == 0 {
+ return fmt.Errorf("no volume needs change")
+ }
+
+ for _, dst := range allLocations {
+ err := operation.WithVolumeServerClient(dst.dataNode.Id, commandEnv.option.GrpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
+ resp, configureErr := volumeServerClient.VolumeConfigure(context.Background(), &volume_server_pb.VolumeConfigureRequest{
+ VolumeId: uint32(vid),
+ Replication: replicaPlacement.String(),
+ })
+ if configureErr != nil {
+ return configureErr
+ }
+ if resp.Error != "" {
+ return errors.New(resp.Error)
+ }
+ return nil
+ })
+
+ if err != nil {
+ return err
+ }
+
+ }
+
+ return nil
+}
diff --git a/weed/shell/command_volume_copy.go b/weed/shell/command_volume_copy.go
index 1c83ba655..aecc071ad 100644
--- a/weed/shell/command_volume_copy.go
+++ b/weed/shell/command_volume_copy.go
@@ -1,7 +1,6 @@
package shell
import (
- "context"
"fmt"
"io"
@@ -47,7 +46,6 @@ func (c *commandVolumeCopy) Do(args []string, commandEnv *CommandEnv, writer io.
return fmt.Errorf("source and target volume servers are the same!")
}
- ctx := context.Background()
- _, err = copyVolume(ctx, commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer)
+ _, err = copyVolume(commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer)
return
}
diff --git a/weed/shell/command_volume_delete.go b/weed/shell/command_volume_delete.go
index 17d27ea3a..5869b1621 100644
--- a/weed/shell/command_volume_delete.go
+++ b/weed/shell/command_volume_delete.go
@@ -1,7 +1,6 @@
package shell
import (
- "context"
"fmt"
"io"
@@ -42,7 +41,6 @@ func (c *commandVolumeDelete) Do(args []string, commandEnv *CommandEnv, writer i
return fmt.Errorf("wrong volume id format %s: %v", volumeId, err)
}
- ctx := context.Background()
- return deleteVolume(ctx, commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer)
+ return deleteVolume(commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer)
}
diff --git a/weed/shell/command_volume_fix_replication.go b/weed/shell/command_volume_fix_replication.go
index 4c7a794c0..210f4819d 100644
--- a/weed/shell/command_volume_fix_replication.go
+++ b/weed/shell/command_volume_fix_replication.go
@@ -3,13 +3,14 @@ package shell
import (
"context"
"fmt"
- "github.com/chrislusf/seaweedfs/weed/operation"
- "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
- "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
- "github.com/chrislusf/seaweedfs/weed/storage"
"io"
"math/rand"
"sort"
+
+ "github.com/chrislusf/seaweedfs/weed/operation"
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
)
func init() {
@@ -49,9 +50,8 @@ func (c *commandVolumeFixReplication) Do(args []string, commandEnv *CommandEnv,
}
var resp *master_pb.VolumeListResponse
- ctx := context.Background()
- err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
- resp, err = client.VolumeList(ctx, &master_pb.VolumeListRequest{})
+ err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
+ resp, err = client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
return err
})
if err != nil {
@@ -78,7 +78,7 @@ func (c *commandVolumeFixReplication) Do(args []string, commandEnv *CommandEnv,
underReplicatedVolumeLocations := make(map[uint32][]location)
for vid, locations := range replicatedVolumeLocations {
volumeInfo := replicatedVolumeInfo[vid]
- replicaPlacement, _ := storage.NewReplicaPlacementFromByte(byte(volumeInfo.ReplicaPlacement))
+ replicaPlacement, _ := super_block.NewReplicaPlacementFromByte(byte(volumeInfo.ReplicaPlacement))
if replicaPlacement.GetCopyCount() > len(locations) {
underReplicatedVolumeLocations[vid] = locations
}
@@ -97,7 +97,7 @@ func (c *commandVolumeFixReplication) Do(args []string, commandEnv *CommandEnv,
for vid, locations := range underReplicatedVolumeLocations {
volumeInfo := replicatedVolumeInfo[vid]
- replicaPlacement, _ := storage.NewReplicaPlacementFromByte(byte(volumeInfo.ReplicaPlacement))
+ replicaPlacement, _ := super_block.NewReplicaPlacementFromByte(byte(volumeInfo.ReplicaPlacement))
foundNewLocation := false
for _, dst := range allLocations {
// check whether data nodes satisfy the constraints
@@ -113,7 +113,7 @@ func (c *commandVolumeFixReplication) Do(args []string, commandEnv *CommandEnv,
}
err := operation.WithVolumeServerClient(dst.dataNode.Id, commandEnv.option.GrpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
- _, replicateErr := volumeServerClient.VolumeCopy(ctx, &volume_server_pb.VolumeCopyRequest{
+ _, replicateErr := volumeServerClient.VolumeCopy(context.Background(), &volume_server_pb.VolumeCopyRequest{
VolumeId: volumeInfo.Id,
SourceDataNode: sourceNode.dataNode.Id,
})
@@ -145,7 +145,7 @@ func keepDataNodesSorted(dataNodes []location) {
})
}
-func satisfyReplicaPlacement(replicaPlacement *storage.ReplicaPlacement, existingLocations []location, possibleLocation location) bool {
+func satisfyReplicaPlacement(replicaPlacement *super_block.ReplicaPlacement, existingLocations []location, possibleLocation location) bool {
existingDataCenters := make(map[string]bool)
existingRacks := make(map[string]bool)
diff --git a/weed/shell/command_volume_list.go b/weed/shell/command_volume_list.go
index 91b5a0d32..c5a9388fa 100644
--- a/weed/shell/command_volume_list.go
+++ b/weed/shell/command_volume_list.go
@@ -32,9 +32,8 @@ func (c *commandVolumeList) Help() string {
func (c *commandVolumeList) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
var resp *master_pb.VolumeListResponse
- ctx := context.Background()
- err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
- resp, err = client.VolumeList(ctx, &master_pb.VolumeListRequest{})
+ err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
+ resp, err = client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
return err
})
if err != nil {
@@ -46,7 +45,7 @@ func (c *commandVolumeList) Do(args []string, commandEnv *CommandEnv, writer io.
}
func writeTopologyInfo(writer io.Writer, t *master_pb.TopologyInfo, volumeSizeLimitMb uint64) statistics {
- fmt.Fprintf(writer, "Topology volume:%d/%d active:%d free:%d volumeSizeLimit:%d MB\n", t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount, volumeSizeLimitMb)
+ fmt.Fprintf(writer, "Topology volume:%d/%d active:%d free:%d remote:%d volumeSizeLimit:%d MB\n", t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount, t.RemoteVolumeCount, volumeSizeLimitMb)
sort.Slice(t.DataCenterInfos, func(i, j int) bool {
return t.DataCenterInfos[i].Id < t.DataCenterInfos[j].Id
})
@@ -58,7 +57,7 @@ func writeTopologyInfo(writer io.Writer, t *master_pb.TopologyInfo, volumeSizeLi
return s
}
func writeDataCenterInfo(writer io.Writer, t *master_pb.DataCenterInfo) statistics {
- fmt.Fprintf(writer, " DataCenter %s volume:%d/%d active:%d free:%d\n", t.Id, t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount)
+ fmt.Fprintf(writer, " DataCenter %s volume:%d/%d active:%d free:%d remote:%d\n", t.Id, t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount, t.RemoteVolumeCount)
var s statistics
sort.Slice(t.RackInfos, func(i, j int) bool {
return t.RackInfos[i].Id < t.RackInfos[j].Id
@@ -70,7 +69,7 @@ func writeDataCenterInfo(writer io.Writer, t *master_pb.DataCenterInfo) statisti
return s
}
func writeRackInfo(writer io.Writer, t *master_pb.RackInfo) statistics {
- fmt.Fprintf(writer, " Rack %s volume:%d/%d active:%d free:%d\n", t.Id, t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount)
+ fmt.Fprintf(writer, " Rack %s volume:%d/%d active:%d free:%d remote:%d\n", t.Id, t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount, t.RemoteVolumeCount)
var s statistics
sort.Slice(t.DataNodeInfos, func(i, j int) bool {
return t.DataNodeInfos[i].Id < t.DataNodeInfos[j].Id
@@ -82,7 +81,7 @@ func writeRackInfo(writer io.Writer, t *master_pb.RackInfo) statistics {
return s
}
func writeDataNodeInfo(writer io.Writer, t *master_pb.DataNodeInfo) statistics {
- fmt.Fprintf(writer, " DataNode %s volume:%d/%d active:%d free:%d\n", t.Id, t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount)
+ fmt.Fprintf(writer, " DataNode %s volume:%d/%d active:%d free:%d remote:%d\n", t.Id, t.VolumeCount, t.MaxVolumeCount, t.ActiveVolumeCount, t.FreeVolumeCount, t.RemoteVolumeCount)
var s statistics
sort.Slice(t.VolumeInfos, func(i, j int) bool {
return t.VolumeInfos[i].Id < t.VolumeInfos[j].Id
diff --git a/weed/shell/command_volume_mount.go b/weed/shell/command_volume_mount.go
index 50a307492..cffc7136b 100644
--- a/weed/shell/command_volume_mount.go
+++ b/weed/shell/command_volume_mount.go
@@ -45,14 +45,13 @@ func (c *commandVolumeMount) Do(args []string, commandEnv *CommandEnv, writer io
return fmt.Errorf("wrong volume id format %s: %v", volumeId, err)
}
- ctx := context.Background()
- return mountVolume(ctx, commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer)
+ return mountVolume(commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer)
}
-func mountVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer string) (err error) {
+func mountVolume(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer string) (err error) {
return operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
- _, mountErr := volumeServerClient.VolumeMount(ctx, &volume_server_pb.VolumeMountRequest{
+ _, mountErr := volumeServerClient.VolumeMount(context.Background(), &volume_server_pb.VolumeMountRequest{
VolumeId: uint32(volumeId),
})
return mountErr
diff --git a/weed/shell/command_volume_move.go b/weed/shell/command_volume_move.go
index 08d87c988..c25b953a5 100644
--- a/weed/shell/command_volume_move.go
+++ b/weed/shell/command_volume_move.go
@@ -25,7 +25,7 @@ func (c *commandVolumeMove) Name() string {
}
func (c *commandVolumeMove) Help() string {
- return ` move a live volume from one volume server to another volume server
+ return `move a live volume from one volume server to another volume server
volume.move
@@ -59,26 +59,25 @@ func (c *commandVolumeMove) Do(args []string, commandEnv *CommandEnv, writer io.
return fmt.Errorf("source and target volume servers are the same!")
}
- ctx := context.Background()
- return LiveMoveVolume(ctx, commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer, 5*time.Second)
+ return LiveMoveVolume(commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer, 5*time.Second)
}
// LiveMoveVolume moves one volume from one source volume server to one target volume server, with idleTimeout to drain the incoming requests.
-func LiveMoveVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer, targetVolumeServer string, idleTimeout time.Duration) (err error) {
+func LiveMoveVolume(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer, targetVolumeServer string, idleTimeout time.Duration) (err error) {
log.Printf("copying volume %d from %s to %s", volumeId, sourceVolumeServer, targetVolumeServer)
- lastAppendAtNs, err := copyVolume(ctx, grpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer)
+ lastAppendAtNs, err := copyVolume(grpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer)
if err != nil {
return fmt.Errorf("copy volume %d from %s to %s: %v", volumeId, sourceVolumeServer, targetVolumeServer, err)
}
log.Printf("tailing volume %d from %s to %s", volumeId, sourceVolumeServer, targetVolumeServer)
- if err = tailVolume(ctx, grpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer, lastAppendAtNs, idleTimeout); err != nil {
+ if err = tailVolume(grpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer, lastAppendAtNs, idleTimeout); err != nil {
return fmt.Errorf("tail volume %d from %s to %s: %v", volumeId, sourceVolumeServer, targetVolumeServer, err)
}
log.Printf("deleting volume %d from %s", volumeId, sourceVolumeServer)
- if err = deleteVolume(ctx, grpcDialOption, volumeId, sourceVolumeServer); err != nil {
+ if err = deleteVolume(grpcDialOption, volumeId, sourceVolumeServer); err != nil {
return fmt.Errorf("delete volume %d from %s: %v", volumeId, sourceVolumeServer, err)
}
@@ -86,10 +85,10 @@ func LiveMoveVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeI
return nil
}
-func copyVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer, targetVolumeServer string) (lastAppendAtNs uint64, err error) {
+func copyVolume(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer, targetVolumeServer string) (lastAppendAtNs uint64, err error) {
err = operation.WithVolumeServerClient(targetVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
- resp, replicateErr := volumeServerClient.VolumeCopy(ctx, &volume_server_pb.VolumeCopyRequest{
+ resp, replicateErr := volumeServerClient.VolumeCopy(context.Background(), &volume_server_pb.VolumeCopyRequest{
VolumeId: uint32(volumeId),
SourceDataNode: sourceVolumeServer,
})
@@ -102,10 +101,10 @@ func copyVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId ne
return
}
-func tailVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer, targetVolumeServer string, lastAppendAtNs uint64, idleTimeout time.Duration) (err error) {
+func tailVolume(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer, targetVolumeServer string, lastAppendAtNs uint64, idleTimeout time.Duration) (err error) {
return operation.WithVolumeServerClient(targetVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
- _, replicateErr := volumeServerClient.VolumeTailReceiver(ctx, &volume_server_pb.VolumeTailReceiverRequest{
+ _, replicateErr := volumeServerClient.VolumeTailReceiver(context.Background(), &volume_server_pb.VolumeTailReceiverRequest{
VolumeId: uint32(volumeId),
SinceNs: lastAppendAtNs,
IdleTimeoutSeconds: uint32(idleTimeout.Seconds()),
@@ -116,9 +115,9 @@ func tailVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId ne
}
-func deleteVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer string) (err error) {
+func deleteVolume(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer string) (err error) {
return operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
- _, deleteErr := volumeServerClient.VolumeDelete(ctx, &volume_server_pb.VolumeDeleteRequest{
+ _, deleteErr := volumeServerClient.VolumeDelete(context.Background(), &volume_server_pb.VolumeDeleteRequest{
VolumeId: uint32(volumeId),
})
return deleteErr
diff --git a/weed/shell/command_volume_tier_download.go b/weed/shell/command_volume_tier_download.go
new file mode 100644
index 000000000..756dc4686
--- /dev/null
+++ b/weed/shell/command_volume_tier_download.go
@@ -0,0 +1,166 @@
+package shell
+
+import (
+ "context"
+ "flag"
+ "fmt"
+ "io"
+
+ "google.golang.org/grpc"
+
+ "github.com/chrislusf/seaweedfs/weed/operation"
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+)
+
+func init() {
+ Commands = append(Commands, &commandVolumeTierDownload{})
+}
+
+type commandVolumeTierDownload struct {
+}
+
+func (c *commandVolumeTierDownload) Name() string {
+ return "volume.tier.download"
+}
+
+func (c *commandVolumeTierDownload) Help() string {
+ return `download the dat file of a volume from a remote tier
+
+ volume.tier.download [-collection=""]
+ volume.tier.download [-collection=""] -volumeId=
+
+ e.g.:
+ volume.tier.download -volumeId=7
+ volume.tier.download -volumeId=7
+
+ This command will download the dat file of a volume from a remote tier to a volume server in local cluster.
+
+`
+}
+
+func (c *commandVolumeTierDownload) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+
+ tierCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
+ volumeId := tierCommand.Int("volumeId", 0, "the volume id")
+ collection := tierCommand.String("collection", "", "the collection name")
+ if err = tierCommand.Parse(args); err != nil {
+ return nil
+ }
+
+ vid := needle.VolumeId(*volumeId)
+
+ // collect topology information
+ topologyInfo, err := collectTopologyInfo(commandEnv)
+ if err != nil {
+ return err
+ }
+
+ // volumeId is provided
+ if vid != 0 {
+ return doVolumeTierDownload(commandEnv, writer, *collection, vid)
+ }
+
+ // apply to all volumes in the collection
+ // reusing collectVolumeIdsForEcEncode for now
+ volumeIds := collectRemoteVolumes(topologyInfo, *collection)
+ if err != nil {
+ return err
+ }
+ fmt.Printf("tier download volumes: %v\n", volumeIds)
+ for _, vid := range volumeIds {
+ if err = doVolumeTierDownload(commandEnv, writer, *collection, vid); err != nil {
+ return err
+ }
+ }
+
+ return nil
+}
+
+func collectRemoteVolumes(topoInfo *master_pb.TopologyInfo, selectedCollection string) (vids []needle.VolumeId) {
+
+ vidMap := make(map[uint32]bool)
+ eachDataNode(topoInfo, func(dc string, rack RackId, dn *master_pb.DataNodeInfo) {
+ for _, v := range dn.VolumeInfos {
+ if v.Collection == selectedCollection && v.RemoteStorageKey != "" && v.RemoteStorageName != "" {
+ vidMap[v.Id] = true
+ }
+ }
+ })
+
+ for vid := range vidMap {
+ vids = append(vids, needle.VolumeId(vid))
+ }
+
+ return
+}
+
+func doVolumeTierDownload(commandEnv *CommandEnv, writer io.Writer, collection string, vid needle.VolumeId) (err error) {
+ // find volume location
+ locations, found := commandEnv.MasterClient.GetLocations(uint32(vid))
+ if !found {
+ return fmt.Errorf("volume %d not found", vid)
+ }
+
+ // TODO parallelize this
+ for _, loc := range locations {
+ // copy the .dat file from remote tier to local
+ err = downloadDatFromRemoteTier(commandEnv.option.GrpcDialOption, writer, needle.VolumeId(vid), collection, loc.Url)
+ if err != nil {
+ return fmt.Errorf("download dat file for volume %d to %s: %v", vid, loc.Url, err)
+ }
+ }
+
+ return nil
+}
+
+func downloadDatFromRemoteTier(grpcDialOption grpc.DialOption, writer io.Writer, volumeId needle.VolumeId, collection string, targetVolumeServer string) error {
+
+ err := operation.WithVolumeServerClient(targetVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
+ stream, downloadErr := volumeServerClient.VolumeTierMoveDatFromRemote(context.Background(), &volume_server_pb.VolumeTierMoveDatFromRemoteRequest{
+ VolumeId: uint32(volumeId),
+ Collection: collection,
+ })
+
+ var lastProcessed int64
+ for {
+ resp, recvErr := stream.Recv()
+ if recvErr != nil {
+ if recvErr == io.EOF {
+ break
+ } else {
+ return recvErr
+ }
+ }
+
+ processingSpeed := float64(resp.Processed-lastProcessed) / 1024.0 / 1024.0
+
+ fmt.Fprintf(writer, "downloaded %.2f%%, %d bytes, %.2fMB/s\n", resp.ProcessedPercentage, resp.Processed, processingSpeed)
+
+ lastProcessed = resp.Processed
+ }
+ if downloadErr != nil {
+ return downloadErr
+ }
+
+ _, unmountErr := volumeServerClient.VolumeUnmount(context.Background(), &volume_server_pb.VolumeUnmountRequest{
+ VolumeId: uint32(volumeId),
+ })
+ if unmountErr != nil {
+ return unmountErr
+ }
+
+ _, mountErr := volumeServerClient.VolumeMount(context.Background(), &volume_server_pb.VolumeMountRequest{
+ VolumeId: uint32(volumeId),
+ })
+ if mountErr != nil {
+ return mountErr
+ }
+
+ return nil
+ })
+
+ return err
+
+}
diff --git a/weed/shell/command_volume_tier_upload.go b/weed/shell/command_volume_tier_upload.go
new file mode 100644
index 000000000..5131e8f85
--- /dev/null
+++ b/weed/shell/command_volume_tier_upload.go
@@ -0,0 +1,147 @@
+package shell
+
+import (
+ "context"
+ "flag"
+ "fmt"
+ "io"
+ "time"
+
+ "google.golang.org/grpc"
+
+ "github.com/chrislusf/seaweedfs/weed/operation"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+)
+
+func init() {
+ Commands = append(Commands, &commandVolumeTierUpload{})
+}
+
+type commandVolumeTierUpload struct {
+}
+
+func (c *commandVolumeTierUpload) Name() string {
+ return "volume.tier.upload"
+}
+
+func (c *commandVolumeTierUpload) Help() string {
+ return `upload the dat file of a volume to a remote tier
+
+ volume.tier.upload [-collection=""] [-fullPercent=95] [-quietFor=1h]
+ volume.tier.upload [-collection=""] -volumeId= -dest= [-keepLocalDatFile]
+
+ e.g.:
+ volume.tier.upload -volumeId=7 -dest=s3
+ volume.tier.upload -volumeId=7 -dest=s3.default
+
+ The is defined in master.toml.
+ For example, "s3.default" in [storage.backend.s3.default]
+
+ This command will move the dat file of a volume to a remote tier.
+
+ SeaweedFS enables scalable and fast local access to lots of files,
+ and the cloud storage is slower by cost efficient. How to combine them together?
+
+ Usually the data follows 80/20 rule: only 20% of data is frequently accessed.
+ We can offload the old volumes to the cloud.
+
+ With this, SeaweedFS can be both fast and scalable, and infinite storage space.
+ Just add more local SeaweedFS volume servers to increase the throughput.
+
+ The index file is still local, and the same O(1) disk read is applied to the remote file.
+
+`
+}
+
+func (c *commandVolumeTierUpload) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+
+ tierCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
+ volumeId := tierCommand.Int("volumeId", 0, "the volume id")
+ collection := tierCommand.String("collection", "", "the collection name")
+ fullPercentage := tierCommand.Float64("fullPercent", 95, "the volume reaches the percentage of max volume size")
+ quietPeriod := tierCommand.Duration("quietFor", 24*time.Hour, "select volumes without no writes for this period")
+ dest := tierCommand.String("dest", "", "the target tier name")
+ keepLocalDatFile := tierCommand.Bool("keepLocalDatFile", false, "whether keep local dat file")
+ if err = tierCommand.Parse(args); err != nil {
+ return nil
+ }
+
+ vid := needle.VolumeId(*volumeId)
+
+ // volumeId is provided
+ if vid != 0 {
+ return doVolumeTierUpload(commandEnv, writer, *collection, vid, *dest, *keepLocalDatFile)
+ }
+
+ // apply to all volumes in the collection
+ // reusing collectVolumeIdsForEcEncode for now
+ volumeIds, err := collectVolumeIdsForEcEncode(commandEnv, *collection, *fullPercentage, *quietPeriod)
+ if err != nil {
+ return err
+ }
+ fmt.Printf("tier upload volumes: %v\n", volumeIds)
+ for _, vid := range volumeIds {
+ if err = doVolumeTierUpload(commandEnv, writer, *collection, vid, *dest, *keepLocalDatFile); err != nil {
+ return err
+ }
+ }
+
+ return nil
+}
+
+func doVolumeTierUpload(commandEnv *CommandEnv, writer io.Writer, collection string, vid needle.VolumeId, dest string, keepLocalDatFile bool) (err error) {
+ // find volume location
+ locations, found := commandEnv.MasterClient.GetLocations(uint32(vid))
+ if !found {
+ return fmt.Errorf("volume %d not found", vid)
+ }
+
+ err = markVolumeReadonly(commandEnv.option.GrpcDialOption, needle.VolumeId(vid), locations)
+ if err != nil {
+ return fmt.Errorf("mark volume %d as readonly on %s: %v", vid, locations[0].Url, err)
+ }
+
+ // copy the .dat file to remote tier
+ err = uploadDatToRemoteTier(commandEnv.option.GrpcDialOption, writer, needle.VolumeId(vid), collection, locations[0].Url, dest, keepLocalDatFile)
+ if err != nil {
+ return fmt.Errorf("copy dat file for volume %d on %s to %s: %v", vid, locations[0].Url, dest, err)
+ }
+
+ return nil
+}
+
+func uploadDatToRemoteTier(grpcDialOption grpc.DialOption, writer io.Writer, volumeId needle.VolumeId, collection string, sourceVolumeServer string, dest string, keepLocalDatFile bool) error {
+
+ err := operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
+ stream, copyErr := volumeServerClient.VolumeTierMoveDatToRemote(context.Background(), &volume_server_pb.VolumeTierMoveDatToRemoteRequest{
+ VolumeId: uint32(volumeId),
+ Collection: collection,
+ DestinationBackendName: dest,
+ KeepLocalDatFile: keepLocalDatFile,
+ })
+
+ var lastProcessed int64
+ for {
+ resp, recvErr := stream.Recv()
+ if recvErr != nil {
+ if recvErr == io.EOF {
+ break
+ } else {
+ return recvErr
+ }
+ }
+
+ processingSpeed := float64(resp.Processed-lastProcessed) / 1024.0 / 1024.0
+
+ fmt.Fprintf(writer, "copied %.2f%%, %d bytes, %.2fMB/s\n", resp.ProcessedPercentage, resp.Processed, processingSpeed)
+
+ lastProcessed = resp.Processed
+ }
+
+ return copyErr
+ })
+
+ return err
+
+}
diff --git a/weed/shell/command_volume_unmount.go b/weed/shell/command_volume_unmount.go
index 8096f34d8..6e5bef485 100644
--- a/weed/shell/command_volume_unmount.go
+++ b/weed/shell/command_volume_unmount.go
@@ -45,14 +45,13 @@ func (c *commandVolumeUnmount) Do(args []string, commandEnv *CommandEnv, writer
return fmt.Errorf("wrong volume id format %s: %v", volumeId, err)
}
- ctx := context.Background()
- return unmountVolume(ctx, commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer)
+ return unmountVolume(commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer)
}
-func unmountVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer string) (err error) {
+func unmountVolume(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer string) (err error) {
return operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
- _, unmountErr := volumeServerClient.VolumeUnmount(ctx, &volume_server_pb.VolumeUnmountRequest{
+ _, unmountErr := volumeServerClient.VolumeUnmount(context.Background(), &volume_server_pb.VolumeUnmountRequest{
VolumeId: uint32(volumeId),
})
return unmountErr
diff --git a/weed/shell/commands.go b/weed/shell/commands.go
index b642ec253..b8832ad93 100644
--- a/weed/shell/commands.go
+++ b/weed/shell/commands.go
@@ -1,7 +1,6 @@
package shell
import (
- "context"
"fmt"
"io"
"net/url"
@@ -9,10 +8,11 @@ import (
"strconv"
"strings"
+ "google.golang.org/grpc"
+
"github.com/chrislusf/seaweedfs/weed/filer2"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/wdclient"
- "google.golang.org/grpc"
)
type ShellOptions struct {
@@ -42,10 +42,9 @@ var (
func NewCommandEnv(options ShellOptions) *CommandEnv {
return &CommandEnv{
- env: make(map[string]string),
- MasterClient: wdclient.NewMasterClient(context.Background(),
- options.GrpcDialOption, "shell", strings.Split(*options.Masters, ",")),
- option: options,
+ env: make(map[string]string),
+ MasterClient: wdclient.NewMasterClient(options.GrpcDialOption, "shell", 0, strings.Split(*options.Masters, ",")),
+ option: options,
}
}
@@ -59,38 +58,27 @@ func (ce *CommandEnv) parseUrl(input string) (filerServer string, filerPort int6
return ce.option.FilerHost, ce.option.FilerPort, input, err
}
-func (ce *CommandEnv) isDirectory(ctx context.Context, filerServer string, filerPort int64, path string) bool {
+func (ce *CommandEnv) isDirectory(filerServer string, filerPort int64, path string) bool {
- return ce.checkDirectory(ctx, filerServer, filerPort, path) == nil
+ return ce.checkDirectory(filerServer, filerPort, path) == nil
}
-func (ce *CommandEnv) checkDirectory(ctx context.Context, filerServer string, filerPort int64, path string) error {
+func (ce *CommandEnv) checkDirectory(filerServer string, filerPort int64, path string) error {
dir, name := filer2.FullPath(path).DirAndName()
- return ce.withFilerClient(ctx, filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
+ return ce.withFilerClient(filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
- resp, listErr := client.ListEntries(ctx, &filer_pb.ListEntriesRequest{
- Directory: dir,
- Prefix: name,
- StartFromFileName: name,
- InclusiveStartFrom: true,
- Limit: 1,
+ resp, lookupErr := filer_pb.LookupEntry(client, &filer_pb.LookupDirectoryEntryRequest{
+ Directory: dir,
+ Name: name,
})
- if listErr != nil {
- return listErr
- }
-
- if len(resp.Entries) == 0 {
- return fmt.Errorf("entry not found")
- }
-
- if resp.Entries[0].Name != name {
- return fmt.Errorf("not a valid directory, found %s", resp.Entries[0].Name)
+ if lookupErr != nil {
+ return lookupErr
}
- if !resp.Entries[0].IsDirectory {
+ if !resp.Entry.IsDirectory {
return fmt.Errorf("not a directory")
}
diff --git a/weed/stats/disk_supported.go b/weed/stats/disk_supported.go
index 0537828b0..dff580b5b 100644
--- a/weed/stats/disk_supported.go
+++ b/weed/stats/disk_supported.go
@@ -17,5 +17,7 @@ func fillInDiskStatus(disk *volume_server_pb.DiskStatus) {
disk.All = fs.Blocks * uint64(fs.Bsize)
disk.Free = fs.Bfree * uint64(fs.Bsize)
disk.Used = disk.All - disk.Free
+ disk.PercentFree = float32((float64(disk.Free) / float64(disk.All)) * 100)
+ disk.PercentUsed = float32((float64(disk.Used) / float64(disk.All)) * 100)
return
}
diff --git a/weed/stats/metrics.go b/weed/stats/metrics.go
index a9624cd86..ee8763e84 100644
--- a/weed/stats/metrics.go
+++ b/weed/stats/metrics.go
@@ -136,7 +136,7 @@ func LoopPushingMetric(name, instance string, gatherer *prometheus.Registry, fnG
}
}
-func SourceName(port int) string {
+func SourceName(port uint32) string {
hostname, err := os.Hostname()
if err != nil {
return "unknown"
diff --git a/weed/storage/backend/backend.go b/weed/storage/backend/backend.go
index 3c297f20b..6941ca5a1 100644
--- a/weed/storage/backend/backend.go
+++ b/weed/storage/backend/backend.go
@@ -2,18 +2,134 @@ package backend
import (
"io"
+ "os"
+ "strings"
"time"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/spf13/viper"
)
-type DataStorageBackend interface {
+type BackendStorageFile interface {
io.ReaderAt
io.WriterAt
Truncate(off int64) error
io.Closer
GetStat() (datSize int64, modTime time.Time, err error)
- String() string
+ Name() string
+}
+
+type BackendStorage interface {
+ ToProperties() map[string]string
+ NewStorageFile(key string, tierInfo *volume_server_pb.VolumeInfo) BackendStorageFile
+ CopyFile(f *os.File, attributes map[string]string, fn func(progressed int64, percentage float32) error) (key string, size int64, err error)
+ DownloadFile(fileName string, key string, fn func(progressed int64, percentage float32) error) (size int64, err error)
+ DeleteFile(key string) (err error)
+}
+
+type StringProperties interface {
+ GetString(key string) string
+}
+type StorageType string
+type BackendStorageFactory interface {
+ StorageType() StorageType
+ BuildStorage(configuration StringProperties, configPrefix string, id string) (BackendStorage, error)
}
var (
- StorageBackends []DataStorageBackend
+ BackendStorageFactories = make(map[StorageType]BackendStorageFactory)
+ BackendStorages = make(map[string]BackendStorage)
)
+
+// used by master to load remote storage configurations
+func LoadConfiguration(config *viper.Viper) {
+
+ StorageBackendPrefix := "storage.backend"
+
+ for backendTypeName := range config.GetStringMap(StorageBackendPrefix) {
+ backendStorageFactory, found := BackendStorageFactories[StorageType(backendTypeName)]
+ if !found {
+ glog.Fatalf("backend storage type %s not found", backendTypeName)
+ }
+ for backendStorageId := range config.GetStringMap(StorageBackendPrefix + "." + backendTypeName) {
+ if !config.GetBool(StorageBackendPrefix + "." + backendTypeName + "." + backendStorageId + ".enabled") {
+ continue
+ }
+ backendStorage, buildErr := backendStorageFactory.BuildStorage(config,
+ StorageBackendPrefix+"."+backendTypeName+"."+backendStorageId+".", backendStorageId)
+ if buildErr != nil {
+ glog.Fatalf("fail to create backend storage %s.%s", backendTypeName, backendStorageId)
+ }
+ BackendStorages[backendTypeName+"."+backendStorageId] = backendStorage
+ if backendStorageId == "default" {
+ BackendStorages[backendTypeName] = backendStorage
+ }
+ }
+ }
+
+}
+
+// used by volume server to receive remote storage configurations from master
+func LoadFromPbStorageBackends(storageBackends []*master_pb.StorageBackend) {
+
+ for _, storageBackend := range storageBackends {
+ backendStorageFactory, found := BackendStorageFactories[StorageType(storageBackend.Type)]
+ if !found {
+ glog.Warningf("storage type %s not found", storageBackend.Type)
+ continue
+ }
+ backendStorage, buildErr := backendStorageFactory.BuildStorage(newProperties(storageBackend.Properties), "", storageBackend.Id)
+ if buildErr != nil {
+ glog.Fatalf("fail to create backend storage %s.%s", storageBackend.Type, storageBackend.Id)
+ }
+ BackendStorages[storageBackend.Type+"."+storageBackend.Id] = backendStorage
+ if storageBackend.Id == "default" {
+ BackendStorages[storageBackend.Type] = backendStorage
+ }
+ }
+}
+
+type Properties struct {
+ m map[string]string
+}
+
+func newProperties(m map[string]string) *Properties {
+ return &Properties{m: m}
+}
+
+func (p *Properties) GetString(key string) string {
+ if v, found := p.m[key]; found {
+ return v
+ }
+ return ""
+}
+
+func ToPbStorageBackends() (backends []*master_pb.StorageBackend) {
+ for sName, s := range BackendStorages {
+ sType, sId := BackendNameToTypeId(sName)
+ if sType == "" {
+ continue
+ }
+ backends = append(backends, &master_pb.StorageBackend{
+ Type: sType,
+ Id: sId,
+ Properties: s.ToProperties(),
+ })
+ }
+ return
+}
+
+func BackendNameToTypeId(backendName string) (backendType, backendId string) {
+ parts := strings.Split(backendName, ".")
+ if len(parts) == 1 {
+ return backendName, "default"
+ }
+ if len(parts) != 2 {
+ return
+ }
+
+ backendType, backendId = parts[0], parts[1]
+ return
+}
diff --git a/weed/storage/backend/disk_file.go b/weed/storage/backend/disk_file.go
index 7f2b39d15..c4b3caffb 100644
--- a/weed/storage/backend/disk_file.go
+++ b/weed/storage/backend/disk_file.go
@@ -6,7 +6,7 @@ import (
)
var (
- _ DataStorageBackend = &DiskFile{}
+ _ BackendStorageFile = &DiskFile{}
)
type DiskFile struct {
@@ -45,6 +45,6 @@ func (df *DiskFile) GetStat() (datSize int64, modTime time.Time, err error) {
return 0, time.Time{}, err
}
-func (df *DiskFile) String() string {
+func (df *DiskFile) Name() string {
return df.fullFilePath
}
diff --git a/weed/storage/backend/memory_map/memory_map_backend.go b/weed/storage/backend/memory_map/memory_map_backend.go
index bac105022..03e7308d0 100644
--- a/weed/storage/backend/memory_map/memory_map_backend.go
+++ b/weed/storage/backend/memory_map/memory_map_backend.go
@@ -8,7 +8,7 @@ import (
)
var (
- _ backend.DataStorageBackend = &MemoryMappedFile{}
+ _ backend.BackendStorageFile = &MemoryMappedFile{}
)
type MemoryMappedFile struct {
@@ -55,6 +55,6 @@ func (mmf *MemoryMappedFile) GetStat() (datSize int64, modTime time.Time, err er
return 0, time.Time{}, err
}
-func (mmf *MemoryMappedFile) String() string {
+func (mmf *MemoryMappedFile) Name() string {
return mmf.mm.File.Name()
}
diff --git a/weed/storage/backend/s3_backend/s3_backend.go b/weed/storage/backend/s3_backend/s3_backend.go
index 0ff7eca21..8d71861c2 100644
--- a/weed/storage/backend/s3_backend/s3_backend.go
+++ b/weed/storage/backend/s3_backend/s3_backend.go
@@ -2,119 +2,176 @@ package s3_backend
import (
"fmt"
+ "io"
+ "os"
"strings"
"time"
"github.com/aws/aws-sdk-go/service/s3"
"github.com/aws/aws-sdk-go/service/s3/s3iface"
+ "github.com/google/uuid"
+
"github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/storage/backend"
- "github.com/chrislusf/seaweedfs/weed/storage/needle"
- "github.com/chrislusf/seaweedfs/weed/util"
-)
-
-var (
- _ backend.DataStorageBackend = &S3Backend{}
)
func init() {
- backend.StorageBackends = append(backend.StorageBackends, &S3Backend{})
+ backend.BackendStorageFactories["s3"] = &S3BackendFactory{}
}
-type S3Backend struct {
- conn s3iface.S3API
- region string
- bucket string
- dir string
- vid needle.VolumeId
- key string
+type S3BackendFactory struct {
}
-func (s3backend S3Backend) ReadAt(p []byte, off int64) (n int, err error) {
- bytesRange := fmt.Sprintf("bytes=%d-%d", off, off+int64(len(p))-1)
- getObjectOutput, getObjectErr := s3backend.conn.GetObject(&s3.GetObjectInput{
- Bucket: &s3backend.bucket,
- Key: &s3backend.key,
- Range: &bytesRange,
- })
+func (factory *S3BackendFactory) StorageType() backend.StorageType {
+ return backend.StorageType("s3")
+}
+func (factory *S3BackendFactory) BuildStorage(configuration backend.StringProperties, configPrefix string, id string) (backend.BackendStorage, error) {
+ return newS3BackendStorage(configuration, configPrefix, id)
+}
- if getObjectErr != nil {
- return 0, fmt.Errorf("bucket %s GetObject %s: %v", s3backend.bucket, s3backend.key, getObjectErr)
+type S3BackendStorage struct {
+ id string
+ aws_access_key_id string
+ aws_secret_access_key string
+ region string
+ bucket string
+ conn s3iface.S3API
+}
+
+func newS3BackendStorage(configuration backend.StringProperties, configPrefix string, id string) (s *S3BackendStorage, err error) {
+ s = &S3BackendStorage{}
+ s.id = id
+ s.aws_access_key_id = configuration.GetString(configPrefix + "aws_access_key_id")
+ s.aws_secret_access_key = configuration.GetString(configPrefix + "aws_secret_access_key")
+ s.region = configuration.GetString(configPrefix + "region")
+ s.bucket = configuration.GetString(configPrefix + "bucket")
+ s.conn, err = createSession(s.aws_access_key_id, s.aws_secret_access_key, s.region)
+
+ glog.V(0).Infof("created backend storage s3.%s for region %s bucket %s", s.id, s.region, s.bucket)
+ return
+}
+
+func (s *S3BackendStorage) ToProperties() map[string]string {
+ m := make(map[string]string)
+ m["aws_access_key_id"] = s.aws_access_key_id
+ m["aws_secret_access_key"] = s.aws_secret_access_key
+ m["region"] = s.region
+ m["bucket"] = s.bucket
+ return m
+}
+
+func (s *S3BackendStorage) NewStorageFile(key string, tierInfo *volume_server_pb.VolumeInfo) backend.BackendStorageFile {
+ if strings.HasPrefix(key, "/") {
+ key = key[1:]
}
- defer getObjectOutput.Body.Close()
- return getObjectOutput.Body.Read(p)
+ f := &S3BackendStorageFile{
+ backendStorage: s,
+ key: key,
+ tierInfo: tierInfo,
+ }
+ return f
}
-func (s3backend S3Backend) WriteAt(p []byte, off int64) (n int, err error) {
- panic("implement me")
+func (s *S3BackendStorage) CopyFile(f *os.File, attributes map[string]string, fn func(progressed int64, percentage float32) error) (key string, size int64, err error) {
+ randomUuid, _ := uuid.NewRandom()
+ key = randomUuid.String()
+
+ glog.V(1).Infof("copying dat file of %s to remote s3.%s as %s", f.Name(), s.id, key)
+
+ size, err = uploadToS3(s.conn, f.Name(), s.bucket, key, attributes, fn)
+
+ return
}
-func (s3backend S3Backend) Truncate(off int64) error {
- panic("implement me")
+func (s *S3BackendStorage) DownloadFile(fileName string, key string, fn func(progressed int64, percentage float32) error) (size int64, err error) {
+
+ glog.V(1).Infof("download dat file of %s from remote s3.%s as %s", fileName, s.id, key)
+
+ size, err = downloadFromS3(s.conn, fileName, s.bucket, key, fn)
+
+ return
}
-func (s3backend S3Backend) Close() error {
- return nil
+func (s *S3BackendStorage) DeleteFile(key string) (err error) {
+
+ glog.V(1).Infof("delete dat file %s from remote", key)
+
+ err = deleteFromS3(s.conn, s.bucket, key)
+
+ return
}
-func (s3backend S3Backend) GetStat() (datSize int64, modTime time.Time, err error) {
+type S3BackendStorageFile struct {
+ backendStorage *S3BackendStorage
+ key string
+ tierInfo *volume_server_pb.VolumeInfo
+}
- headObjectOutput, headObjectErr := s3backend.conn.HeadObject(&s3.HeadObjectInput{
- Bucket: &s3backend.bucket,
- Key: &s3backend.key,
+func (s3backendStorageFile S3BackendStorageFile) ReadAt(p []byte, off int64) (n int, err error) {
+
+ bytesRange := fmt.Sprintf("bytes=%d-%d", off, off+int64(len(p))-1)
+
+ // glog.V(0).Infof("read %s %s", s3backendStorageFile.key, bytesRange)
+
+ getObjectOutput, getObjectErr := s3backendStorageFile.backendStorage.conn.GetObject(&s3.GetObjectInput{
+ Bucket: &s3backendStorageFile.backendStorage.bucket,
+ Key: &s3backendStorageFile.key,
+ Range: &bytesRange,
})
- if headObjectErr != nil {
- return 0, time.Now(), fmt.Errorf("bucket %s HeadObject %s: %v", s3backend.bucket, s3backend.key, headObjectErr)
+ if getObjectErr != nil {
+ return 0, fmt.Errorf("bucket %s GetObject %s: %v", s3backendStorageFile.backendStorage.bucket, s3backendStorageFile.key, getObjectErr)
+ }
+ defer getObjectOutput.Body.Close()
+
+ glog.V(4).Infof("read %s %s", s3backendStorageFile.key, bytesRange)
+ glog.V(4).Infof("content range: %s, contentLength: %d", *getObjectOutput.ContentRange, *getObjectOutput.ContentLength)
+
+ for {
+ if n, err = getObjectOutput.Body.Read(p); err == nil && n < len(p) {
+ p = p[n:]
+ } else {
+ break
+ }
}
- datSize = int64(*headObjectOutput.ContentLength)
- modTime = *headObjectOutput.LastModified
+ if err == io.EOF {
+ err = nil
+ }
return
}
-func (s3backend S3Backend) String() string {
- return fmt.Sprintf("%s/%s", s3backend.bucket, s3backend.key)
+func (s3backendStorageFile S3BackendStorageFile) WriteAt(p []byte, off int64) (n int, err error) {
+ panic("not implemented")
}
-func (s3backend *S3Backend) GetName() string {
- return "s3"
+func (s3backendStorageFile S3BackendStorageFile) Truncate(off int64) error {
+ panic("not implemented")
}
-func (s3backend *S3Backend) GetSinkToDirectory() string {
- return s3backend.dir
+func (s3backendStorageFile S3BackendStorageFile) Close() error {
+ return nil
}
-func (s3backend *S3Backend) Initialize(configuration util.Configuration, vid needle.VolumeId) error {
- glog.V(0).Infof("storage.backend.s3.region: %v", configuration.GetString("region"))
- glog.V(0).Infof("storage.backend.s3.bucket: %v", configuration.GetString("bucket"))
- glog.V(0).Infof("storage.backend.s3.directory: %v", configuration.GetString("directory"))
+func (s3backendStorageFile S3BackendStorageFile) GetStat() (datSize int64, modTime time.Time, err error) {
- return s3backend.initialize(
- configuration.GetString("aws_access_key_id"),
- configuration.GetString("aws_secret_access_key"),
- configuration.GetString("region"),
- configuration.GetString("bucket"),
- configuration.GetString("directory"),
- vid,
- )
-}
-
-func (s3backend *S3Backend) initialize(awsAccessKeyId, awsSecretAccessKey, region, bucket, dir string,
- vid needle.VolumeId) (err error) {
- s3backend.region = region
- s3backend.bucket = bucket
- s3backend.dir = dir
- s3backend.conn, err = createSession(awsAccessKeyId, awsSecretAccessKey, region)
+ files := s3backendStorageFile.tierInfo.GetFiles()
- s3backend.vid = vid
- s3backend.key = fmt.Sprintf("%s/%d.dat", dir, vid)
- if strings.HasPrefix(s3backend.key, "/") {
- s3backend.key = s3backend.key[1:]
+ if len(files) == 0 {
+ err = fmt.Errorf("remote file info not found")
+ return
}
- return err
+ datSize = int64(files[0].FileSize)
+ modTime = time.Unix(int64(files[0].ModifiedTime), 0)
+
+ return
+}
+
+func (s3backendStorageFile S3BackendStorageFile) Name() string {
+ return s3backendStorageFile.key
}
diff --git a/weed/storage/backend/s3_backend/s3_download.go b/weed/storage/backend/s3_backend/s3_download.go
new file mode 100644
index 000000000..dbc28446a
--- /dev/null
+++ b/weed/storage/backend/s3_backend/s3_download.go
@@ -0,0 +1,98 @@
+package s3_backend
+
+import (
+ "fmt"
+ "os"
+ "sync/atomic"
+
+ "github.com/aws/aws-sdk-go/aws"
+ "github.com/aws/aws-sdk-go/service/s3"
+ "github.com/aws/aws-sdk-go/service/s3/s3iface"
+ "github.com/aws/aws-sdk-go/service/s3/s3manager"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+)
+
+func downloadFromS3(sess s3iface.S3API, destFileName string, sourceBucket string, sourceKey string,
+ fn func(progressed int64, percentage float32) error) (fileSize int64, err error) {
+
+ fileSize, err = getFileSize(sess, sourceBucket, sourceKey)
+ if err != nil {
+ return
+ }
+
+ //open the file
+ f, err := os.OpenFile(destFileName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644)
+ if err != nil {
+ return 0, fmt.Errorf("failed to open file %q, %v", destFileName, err)
+ }
+ defer f.Close()
+
+ // Create a downloader with the session and custom options
+ downloader := s3manager.NewDownloaderWithClient(sess, func(u *s3manager.Downloader) {
+ u.PartSize = int64(64 * 1024 * 1024)
+ u.Concurrency = 5
+ })
+
+ fileWriter := &s3DownloadProgressedWriter{
+ fp: f,
+ size: fileSize,
+ written: 0,
+ fn: fn,
+ }
+
+ // Download the file from S3.
+ fileSize, err = downloader.Download(fileWriter, &s3.GetObjectInput{
+ Bucket: aws.String(sourceBucket),
+ Key: aws.String(sourceKey),
+ })
+ if err != nil {
+ return fileSize, fmt.Errorf("failed to download file %s: %v", destFileName, err)
+ }
+
+ glog.V(1).Infof("downloaded file %s\n", destFileName)
+
+ return
+}
+
+// adapted from https://github.com/aws/aws-sdk-go/pull/1868
+// and https://petersouter.xyz/s3-download-progress-bar-in-golang/
+type s3DownloadProgressedWriter struct {
+ fp *os.File
+ size int64
+ written int64
+ fn func(progressed int64, percentage float32) error
+}
+
+func (w *s3DownloadProgressedWriter) WriteAt(p []byte, off int64) (int, error) {
+ n, err := w.fp.WriteAt(p, off)
+ if err != nil {
+ return n, err
+ }
+
+ // Got the length have read( or means has uploaded), and you can construct your message
+ atomic.AddInt64(&w.written, int64(n))
+
+ if w.fn != nil {
+ written := w.written
+ if err := w.fn(written, float32(written*100)/float32(w.size)); err != nil {
+ return n, err
+ }
+ }
+
+ return n, err
+}
+
+func getFileSize(svc s3iface.S3API, bucket string, key string) (filesize int64, error error) {
+ params := &s3.HeadObjectInput{
+ Bucket: aws.String(bucket),
+ Key: aws.String(key),
+ }
+
+ resp, err := svc.HeadObject(params)
+ if err != nil {
+ return 0, err
+ }
+
+ return *resp.ContentLength, nil
+}
diff --git a/weed/storage/backend/s3_backend/s3_sessions.go b/weed/storage/backend/s3_backend/s3_sessions.go
index cd7b7ad47..5fdbcb66b 100644
--- a/weed/storage/backend/s3_backend/s3_sessions.go
+++ b/weed/storage/backend/s3_backend/s3_sessions.go
@@ -52,3 +52,11 @@ func createSession(awsAccessKeyId, awsSecretAccessKey, region string) (s3iface.S
return t, nil
}
+
+func deleteFromS3(sess s3iface.S3API, sourceBucket string, sourceKey string) (err error) {
+ _, err = sess.DeleteObject(&s3.DeleteObjectInput{
+ Bucket: aws.String(sourceBucket),
+ Key: aws.String(sourceKey),
+ })
+ return err
+}
diff --git a/weed/storage/backend/s3_backend/s3_upload.go b/weed/storage/backend/s3_backend/s3_upload.go
new file mode 100644
index 000000000..500a85590
--- /dev/null
+++ b/weed/storage/backend/s3_backend/s3_upload.go
@@ -0,0 +1,114 @@
+package s3_backend
+
+import (
+ "fmt"
+ "os"
+ "sync/atomic"
+
+ "github.com/aws/aws-sdk-go/aws"
+ "github.com/aws/aws-sdk-go/service/s3/s3iface"
+ "github.com/aws/aws-sdk-go/service/s3/s3manager"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+)
+
+func uploadToS3(sess s3iface.S3API, filename string, destBucket string, destKey string,
+ attributes map[string]string,
+ fn func(progressed int64, percentage float32) error) (fileSize int64, err error) {
+
+ //open the file
+ f, err := os.Open(filename)
+ if err != nil {
+ return 0, fmt.Errorf("failed to open file %q, %v", filename, err)
+ }
+ defer f.Close()
+
+ info, err := f.Stat()
+ if err != nil {
+ return 0, fmt.Errorf("failed to stat file %q, %v", filename, err)
+ }
+
+ fileSize = info.Size()
+
+ partSize := int64(64 * 1024 * 1024) // The minimum/default allowed part size is 5MB
+ for partSize*1000 < fileSize {
+ partSize *= 4
+ }
+
+ // Create an uploader with the session and custom options
+ uploader := s3manager.NewUploaderWithClient(sess, func(u *s3manager.Uploader) {
+ u.PartSize = partSize
+ u.Concurrency = 5
+ })
+
+ fileReader := &s3UploadProgressedReader{
+ fp: f,
+ size: fileSize,
+ read: -fileSize,
+ fn: fn,
+ }
+
+ // process tagging
+ tags := ""
+ for k, v := range attributes {
+ if len(tags) > 0 {
+ tags = tags + "&"
+ }
+ tags = tags + k + "=" + v
+ }
+
+ // Upload the file to S3.
+ var result *s3manager.UploadOutput
+ result, err = uploader.Upload(&s3manager.UploadInput{
+ Bucket: aws.String(destBucket),
+ Key: aws.String(destKey),
+ Body: fileReader,
+ ACL: aws.String("private"),
+ ServerSideEncryption: aws.String("AES256"),
+ StorageClass: aws.String("STANDARD_IA"),
+ Tagging: aws.String(tags),
+ })
+
+ //in case it fails to upload
+ if err != nil {
+ return 0, fmt.Errorf("failed to upload file %s: %v", filename, err)
+ }
+ glog.V(1).Infof("file %s uploaded to %s\n", filename, result.Location)
+
+ return
+}
+
+// adapted from https://github.com/aws/aws-sdk-go/pull/1868
+type s3UploadProgressedReader struct {
+ fp *os.File
+ size int64
+ read int64
+ fn func(progressed int64, percentage float32) error
+}
+
+func (r *s3UploadProgressedReader) Read(p []byte) (int, error) {
+ return r.fp.Read(p)
+}
+
+func (r *s3UploadProgressedReader) ReadAt(p []byte, off int64) (int, error) {
+ n, err := r.fp.ReadAt(p, off)
+ if err != nil {
+ return n, err
+ }
+
+ // Got the length have read( or means has uploaded), and you can construct your message
+ atomic.AddInt64(&r.read, int64(n))
+
+ if r.fn != nil {
+ read := r.read
+ if err := r.fn(read, float32(read*100)/float32(r.size)); err != nil {
+ return n, err
+ }
+ }
+
+ return n, err
+}
+
+func (r *s3UploadProgressedReader) Seek(offset int64, whence int) (int64, error) {
+ return r.fp.Seek(offset, whence)
+}
diff --git a/weed/storage/disk_location.go b/weed/storage/disk_location.go
index c7faa57a6..f15303282 100644
--- a/weed/storage/disk_location.go
+++ b/weed/storage/disk_location.go
@@ -1,13 +1,12 @@
package storage
import (
+ "fmt"
"io/ioutil"
"os"
"strings"
"sync"
- "fmt"
-
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
@@ -17,7 +16,7 @@ type DiskLocation struct {
Directory string
MaxVolumeCount int
volumes map[needle.VolumeId]*Volume
- sync.RWMutex
+ volumesLock sync.RWMutex
// erasure coding
ecVolumes map[needle.VolumeId]*erasure_coding.EcVolume
@@ -33,8 +32,8 @@ func NewDiskLocation(dir string, maxVolumeCount int) *DiskLocation {
func (l *DiskLocation) volumeIdFromPath(dir os.FileInfo) (needle.VolumeId, string, error) {
name := dir.Name()
- if !dir.IsDir() && strings.HasSuffix(name, ".dat") {
- base := name[:len(name)-len(".dat")]
+ if !dir.IsDir() && strings.HasSuffix(name, ".idx") {
+ base := name[:len(name)-len(".idx")]
collection, volumeId, err := parseCollectionVolumeId(base)
return volumeId, collection, err
}
@@ -51,30 +50,39 @@ func parseCollectionVolumeId(base string) (collection string, vid needle.VolumeI
return collection, vol, err
}
-func (l *DiskLocation) loadExistingVolume(fileInfo os.FileInfo, needleMapKind NeedleMapType) {
+func (l *DiskLocation) loadExistingVolume(fileInfo os.FileInfo, needleMapKind NeedleMapType) bool {
name := fileInfo.Name()
- if !fileInfo.IsDir() && strings.HasSuffix(name, ".dat") {
+ if !fileInfo.IsDir() && strings.HasSuffix(name, ".idx") {
vid, collection, err := l.volumeIdFromPath(fileInfo)
- if err == nil {
- l.RLock()
- _, found := l.volumes[vid]
- l.RUnlock()
- if !found {
- if v, e := NewVolume(l.Directory, collection, vid, needleMapKind, nil, nil, 0, 0); e == nil {
- l.Lock()
- l.volumes[vid] = v
- l.Unlock()
- size, _, _ := v.FileStat()
- glog.V(0).Infof("data file %s, replicaPlacement=%s v=%d size=%d ttl=%s",
- l.Directory+"/"+name, v.ReplicaPlacement, v.Version(), size, v.Ttl.String())
- // println("volume", vid, "last append at", v.lastAppendAtNs)
- } else {
- glog.V(0).Infof("new volume %s error %s", name, e)
- }
+ if err != nil {
+ glog.Warningf("get volume id failed, %s, err : %s", name, err)
+ return false
+ }
- }
+ // void loading one volume more than once
+ l.volumesLock.RLock()
+ _, found := l.volumes[vid]
+ l.volumesLock.RUnlock()
+ if found {
+ glog.V(1).Infof("loaded volume, %v", vid)
+ return true
+ }
+
+ v, e := NewVolume(l.Directory, collection, vid, needleMapKind, nil, nil, 0, 0)
+ if e != nil {
+ glog.V(0).Infof("new volume %s error %s", name, e)
+ return false
}
+
+ l.volumesLock.Lock()
+ l.volumes[vid] = v
+ l.volumesLock.Unlock()
+ size, _, _ := v.FileStat()
+ glog.V(0).Infof("data file %s, replicaPlacement=%s v=%d size=%d ttl=%s",
+ l.Directory+"/"+name, v.ReplicaPlacement, v.Version(), size, v.Ttl.String())
+ return true
}
+ return false
}
func (l *DiskLocation) concurrentLoadingVolumes(needleMapKind NeedleMapType, concurrency int) {
@@ -95,7 +103,7 @@ func (l *DiskLocation) concurrentLoadingVolumes(needleMapKind NeedleMapType, con
go func() {
defer wg.Done()
for dir := range task_queue {
- l.loadExistingVolume(dir, needleMapKind)
+ _ = l.loadExistingVolume(dir, needleMapKind)
}
}()
}
@@ -115,29 +123,46 @@ func (l *DiskLocation) loadExistingVolumes(needleMapKind NeedleMapType) {
func (l *DiskLocation) DeleteCollectionFromDiskLocation(collection string) (e error) {
- l.Lock()
- for k, v := range l.volumes {
- if v.Collection == collection {
- e = l.deleteVolumeById(k)
- if e != nil {
- l.Unlock()
- return
- }
- }
- }
- l.Unlock()
+ l.volumesLock.Lock()
+ delVolsMap := l.unmountVolumeByCollection(collection)
+ l.volumesLock.Unlock()
l.ecVolumesLock.Lock()
- for k, v := range l.ecVolumes {
- if v.Collection == collection {
- e = l.deleteEcVolumeById(k)
- if e != nil {
- l.ecVolumesLock.Unlock()
- return
+ delEcVolsMap := l.unmountEcVolumeByCollection(collection)
+ l.ecVolumesLock.Unlock()
+
+ errChain := make(chan error, 2)
+ var wg sync.WaitGroup
+ wg.Add(2)
+ go func() {
+ for _, v := range delVolsMap {
+ if err := v.Destroy(); err != nil {
+ errChain <- err
}
}
+ wg.Done()
+ }()
+
+ go func() {
+ for _, v := range delEcVolsMap {
+ v.Destroy()
+ }
+ wg.Done()
+ }()
+
+ go func() {
+ wg.Wait()
+ close(errChain)
+ }()
+
+ errBuilder := strings.Builder{}
+ for err := range errChain {
+ errBuilder.WriteString(err.Error())
+ errBuilder.WriteString("; ")
+ }
+ if errBuilder.Len() > 0 {
+ e = fmt.Errorf(errBuilder.String())
}
- l.ecVolumesLock.Unlock()
return
}
@@ -156,22 +181,15 @@ func (l *DiskLocation) deleteVolumeById(vid needle.VolumeId) (e error) {
}
func (l *DiskLocation) LoadVolume(vid needle.VolumeId, needleMapKind NeedleMapType) bool {
- if fileInfos, err := ioutil.ReadDir(l.Directory); err == nil {
- for _, fileInfo := range fileInfos {
- volId, _, err := l.volumeIdFromPath(fileInfo)
- if vid == volId && err == nil {
- l.loadExistingVolume(fileInfo, needleMapKind)
- return true
- }
- }
+ if fileInfo, found := l.LocateVolume(vid); found {
+ return l.loadExistingVolume(fileInfo, needleMapKind)
}
-
return false
}
func (l *DiskLocation) DeleteVolume(vid needle.VolumeId) error {
- l.Lock()
- defer l.Unlock()
+ l.volumesLock.Lock()
+ defer l.volumesLock.Unlock()
_, ok := l.volumes[vid]
if !ok {
@@ -181,8 +199,8 @@ func (l *DiskLocation) DeleteVolume(vid needle.VolumeId) error {
}
func (l *DiskLocation) UnloadVolume(vid needle.VolumeId) error {
- l.Lock()
- defer l.Unlock()
+ l.volumesLock.Lock()
+ defer l.volumesLock.Unlock()
v, ok := l.volumes[vid]
if !ok {
@@ -193,34 +211,48 @@ func (l *DiskLocation) UnloadVolume(vid needle.VolumeId) error {
return nil
}
+func (l *DiskLocation) unmountVolumeByCollection(collectionName string) map[needle.VolumeId]*Volume {
+ deltaVols := make(map[needle.VolumeId]*Volume, 0)
+ for k, v := range l.volumes {
+ if v.Collection == collectionName && !v.isCompacting {
+ deltaVols[k] = v
+ }
+ }
+
+ for k := range deltaVols {
+ delete(l.volumes, k)
+ }
+ return deltaVols
+}
+
func (l *DiskLocation) SetVolume(vid needle.VolumeId, volume *Volume) {
- l.Lock()
- defer l.Unlock()
+ l.volumesLock.Lock()
+ defer l.volumesLock.Unlock()
l.volumes[vid] = volume
}
func (l *DiskLocation) FindVolume(vid needle.VolumeId) (*Volume, bool) {
- l.RLock()
- defer l.RUnlock()
+ l.volumesLock.RLock()
+ defer l.volumesLock.RUnlock()
v, ok := l.volumes[vid]
return v, ok
}
func (l *DiskLocation) VolumesLen() int {
- l.RLock()
- defer l.RUnlock()
+ l.volumesLock.RLock()
+ defer l.volumesLock.RUnlock()
return len(l.volumes)
}
func (l *DiskLocation) Close() {
- l.Lock()
+ l.volumesLock.Lock()
for _, v := range l.volumes {
v.Close()
}
- l.Unlock()
+ l.volumesLock.Unlock()
l.ecVolumesLock.Lock()
for _, ecVolume := range l.ecVolumes {
@@ -230,3 +262,16 @@ func (l *DiskLocation) Close() {
return
}
+
+func (l *DiskLocation) LocateVolume(vid needle.VolumeId) (os.FileInfo, bool) {
+ if fileInfos, err := ioutil.ReadDir(l.Directory); err == nil {
+ for _, fileInfo := range fileInfos {
+ volId, _, err := l.volumeIdFromPath(fileInfo)
+ if vid == volId && err == nil {
+ return fileInfo, true
+ }
+ }
+ }
+
+ return nil, false
+}
diff --git a/weed/storage/disk_location_ec.go b/weed/storage/disk_location_ec.go
index ba0824c6d..f6c44e966 100644
--- a/weed/storage/disk_location_ec.go
+++ b/weed/storage/disk_location_ec.go
@@ -169,3 +169,17 @@ func (l *DiskLocation) deleteEcVolumeById(vid needle.VolumeId) (e error) {
delete(l.ecVolumes, vid)
return
}
+
+func (l *DiskLocation) unmountEcVolumeByCollection(collectionName string) map[needle.VolumeId]*erasure_coding.EcVolume {
+ deltaVols := make(map[needle.VolumeId]*erasure_coding.EcVolume, 0)
+ for k, v := range l.ecVolumes {
+ if v.Collection == collectionName {
+ deltaVols[k] = v
+ }
+ }
+
+ for k, _ := range deltaVols {
+ delete(l.ecVolumes, k)
+ }
+ return deltaVols
+}
diff --git a/weed/storage/erasure_coding/ec_decoder.go b/weed/storage/erasure_coding/ec_decoder.go
new file mode 100644
index 000000000..ae77cee3f
--- /dev/null
+++ b/weed/storage/erasure_coding/ec_decoder.go
@@ -0,0 +1,198 @@
+package erasure_coding
+
+import (
+ "fmt"
+ "io"
+ "os"
+
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/idx"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle_map"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
+ "github.com/chrislusf/seaweedfs/weed/storage/types"
+)
+
+// write .idx file from .ecx and .ecj files
+func WriteIdxFileFromEcIndex(baseFileName string) (err error) {
+
+ ecxFile, openErr := os.OpenFile(baseFileName+".ecx", os.O_RDONLY, 0644)
+ if openErr != nil {
+ return fmt.Errorf("cannot open ec index %s.ecx: %v", baseFileName, openErr)
+ }
+ defer ecxFile.Close()
+
+ idxFile, openErr := os.OpenFile(baseFileName+".idx", os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644)
+ if openErr != nil {
+ return fmt.Errorf("cannot open %s.idx: %v", baseFileName, openErr)
+ }
+ defer idxFile.Close()
+
+ io.Copy(idxFile, ecxFile)
+
+ err = iterateEcjFile(baseFileName, func(key types.NeedleId) error {
+
+ bytes := needle_map.ToBytes(key, types.Offset{}, types.TombstoneFileSize)
+ idxFile.Write(bytes)
+
+ return nil
+ })
+
+ return err
+}
+
+// FindDatFileSize calculate .dat file size from max offset entry
+// there may be extra deletions after that entry
+// but they are deletions anyway
+func FindDatFileSize(baseFileName string) (datSize int64, err error) {
+
+ version, err := readEcVolumeVersion(baseFileName)
+ if err != nil {
+ return 0, fmt.Errorf("read ec volume %s version: %v", baseFileName, err)
+ }
+
+ err = iterateEcxFile(baseFileName, func(key types.NeedleId, offset types.Offset, size uint32) error {
+
+ if size == types.TombstoneFileSize {
+ return nil
+ }
+
+ entryStopOffset := offset.ToAcutalOffset() + needle.GetActualSize(size, version)
+ if datSize < entryStopOffset {
+ datSize = entryStopOffset
+ }
+
+ return nil
+ })
+
+ return
+}
+
+func readEcVolumeVersion(baseFileName string) (version needle.Version, err error) {
+
+ // find volume version
+ datFile, err := os.OpenFile(baseFileName+".ec00", os.O_RDONLY, 0644)
+ if err != nil {
+ return 0, fmt.Errorf("open ec volume %s superblock: %v", baseFileName, err)
+ }
+ datBackend := backend.NewDiskFile(datFile)
+
+ superBlock, err := super_block.ReadSuperBlock(datBackend)
+ datBackend.Close()
+ if err != nil {
+ return 0, fmt.Errorf("read ec volume %s superblock: %v", baseFileName, err)
+ }
+
+ return superBlock.Version, nil
+
+}
+
+func iterateEcxFile(baseFileName string, processNeedleFn func(key types.NeedleId, offset types.Offset, size uint32) error) error {
+ ecxFile, openErr := os.OpenFile(baseFileName+".ecx", os.O_RDONLY, 0644)
+ if openErr != nil {
+ return fmt.Errorf("cannot open ec index %s.ecx: %v", baseFileName, openErr)
+ }
+ defer ecxFile.Close()
+
+ buf := make([]byte, types.NeedleMapEntrySize)
+ for {
+ n, err := ecxFile.Read(buf)
+ if n != types.NeedleMapEntrySize {
+ if err == io.EOF {
+ return nil
+ }
+ return err
+ }
+ key, offset, size := idx.IdxFileEntry(buf)
+ if processNeedleFn != nil {
+ err = processNeedleFn(key, offset, size)
+ }
+ if err != nil {
+ if err != io.EOF {
+ return err
+ }
+ return nil
+ }
+ }
+
+}
+
+func iterateEcjFile(baseFileName string, processNeedleFn func(key types.NeedleId) error) error {
+ ecjFile, openErr := os.OpenFile(baseFileName+".ecj", os.O_RDONLY, 0644)
+ if openErr != nil {
+ return fmt.Errorf("cannot open ec index %s.ecx: %v", baseFileName, openErr)
+ }
+ defer ecjFile.Close()
+
+ buf := make([]byte, types.NeedleIdSize)
+ for {
+ n, err := ecjFile.Read(buf)
+ if n != types.NeedleIdSize {
+ if err == io.EOF {
+ return nil
+ }
+ return err
+ }
+ if processNeedleFn != nil {
+ err = processNeedleFn(types.BytesToNeedleId(buf))
+ }
+ if err != nil {
+ if err == io.EOF {
+ return nil
+ }
+ return err
+ }
+ }
+
+}
+
+// WriteDatFile generates .dat from from .ec00 ~ .ec09 files
+func WriteDatFile(baseFileName string, datFileSize int64) error {
+
+ datFile, openErr := os.OpenFile(baseFileName+".dat", os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644)
+ if openErr != nil {
+ return fmt.Errorf("cannot write volume %s.dat: %v", baseFileName, openErr)
+ }
+ defer datFile.Close()
+
+ inputFiles := make([]*os.File, DataShardsCount)
+
+ for shardId := 0; shardId < DataShardsCount; shardId++ {
+ shardFileName := baseFileName + ToExt(shardId)
+ inputFiles[shardId], openErr = os.OpenFile(shardFileName, os.O_RDONLY, 0)
+ if openErr != nil {
+ return openErr
+ }
+ defer inputFiles[shardId].Close()
+ }
+
+ for datFileSize >= DataShardsCount*ErasureCodingLargeBlockSize {
+ for shardId := 0; shardId < DataShardsCount; shardId++ {
+ w, err := io.CopyN(datFile, inputFiles[shardId], ErasureCodingLargeBlockSize)
+ if w != ErasureCodingLargeBlockSize {
+ return fmt.Errorf("copy %s large block %d: %v", baseFileName, shardId, err)
+ }
+ datFileSize -= ErasureCodingLargeBlockSize
+ }
+ }
+
+ for datFileSize > 0 {
+ for shardId := 0; shardId < DataShardsCount; shardId++ {
+ toRead := min(datFileSize, ErasureCodingSmallBlockSize)
+ w, err := io.CopyN(datFile, inputFiles[shardId], toRead)
+ if w != toRead {
+ return fmt.Errorf("copy %s small block %d: %v", baseFileName, shardId, err)
+ }
+ datFileSize -= toRead
+ }
+ }
+
+ return nil
+}
+
+func min(x, y int64) int64 {
+ if x > y {
+ return y
+ }
+ return x
+}
diff --git a/weed/storage/erasure_coding/ec_encoder.go b/weed/storage/erasure_coding/ec_encoder.go
index 97010a1ed..97c3ccbd9 100644
--- a/weed/storage/erasure_coding/ec_encoder.go
+++ b/weed/storage/erasure_coding/ec_encoder.go
@@ -5,12 +5,13 @@ import (
"io"
"os"
+ "github.com/klauspost/reedsolomon"
+
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/storage/idx"
"github.com/chrislusf/seaweedfs/weed/storage/needle_map"
"github.com/chrislusf/seaweedfs/weed/storage/types"
"github.com/chrislusf/seaweedfs/weed/util"
- "github.com/klauspost/reedsolomon"
)
const (
@@ -21,35 +22,38 @@ const (
ErasureCodingSmallBlockSize = 1024 * 1024 // 1MB
)
-// WriteSortedEcxFile generates .ecx file from existing .idx file
+// WriteSortedFileFromIdx generates .ecx file from existing .idx file
// all keys are sorted in ascending order
-func WriteSortedEcxFile(baseFileName string) (e error) {
+func WriteSortedFileFromIdx(baseFileName string, ext string) (e error) {
- cm, err := readCompactMap(baseFileName)
+ nm, err := readNeedleMap(baseFileName)
+ if nm != nil {
+ defer nm.Close()
+ }
if err != nil {
- return fmt.Errorf("readCompactMap: %v", err)
+ return fmt.Errorf("readNeedleMap: %v", err)
}
- ecxFile, err := os.OpenFile(baseFileName+".ecx", os.O_TRUNC|os.O_CREATE|os.O_WRONLY, 0644)
+ ecxFile, err := os.OpenFile(baseFileName+ext, os.O_TRUNC|os.O_CREATE|os.O_WRONLY, 0644)
if err != nil {
return fmt.Errorf("failed to open ecx file: %v", err)
}
defer ecxFile.Close()
- err = cm.AscendingVisit(func(value needle_map.NeedleValue) error {
+ err = nm.AscendingVisit(func(value needle_map.NeedleValue) error {
bytes := value.ToBytes()
_, writeErr := ecxFile.Write(bytes)
return writeErr
})
if err != nil {
- return fmt.Errorf("failed to visit ecx file: %v", err)
+ return fmt.Errorf("failed to visit idx file: %v", err)
}
return nil
}
-// WriteEcFiles generates .ec01 ~ .ec14 files
+// WriteEcFiles generates .ec00 ~ .ec13 files
func WriteEcFiles(baseFileName string) error {
return generateEcFiles(baseFileName, 256*1024, ErasureCodingLargeBlockSize, ErasureCodingSmallBlockSize)
}
@@ -195,7 +199,7 @@ func encodeDatFile(remainingSize int64, err error, baseFileName string, bufferSi
}
buffers := make([][]byte, TotalShardsCount)
- for i, _ := range buffers {
+ for i := range buffers {
buffers[i] = make([]byte, bufferSize)
}
@@ -232,7 +236,7 @@ func rebuildEcFiles(shardHasData []bool, inputFiles []*os.File, outputFiles []*o
}
buffers := make([][]byte, TotalShardsCount)
- for i, _ := range buffers {
+ for i := range buffers {
if shardHasData[i] {
buffers[i] = make([]byte, ErasureCodingSmallBlockSize)
}
@@ -280,14 +284,14 @@ func rebuildEcFiles(shardHasData []bool, inputFiles []*os.File, outputFiles []*o
}
-func readCompactMap(baseFileName string) (*needle_map.CompactMap, error) {
+func readNeedleMap(baseFileName string) (*needle_map.MemDb, error) {
indexFile, err := os.OpenFile(baseFileName+".idx", os.O_RDONLY, 0644)
if err != nil {
return nil, fmt.Errorf("cannot read Volume Index %s.idx: %v", baseFileName, err)
}
defer indexFile.Close()
- cm := needle_map.NewCompactMap()
+ cm := needle_map.NewMemDb()
err = idx.WalkIndexFile(indexFile, func(key types.NeedleId, offset types.Offset, size uint32) error {
if !offset.IsZero() && size != types.TombstoneFileSize {
cm.Set(key, offset, size)
diff --git a/weed/storage/erasure_coding/ec_test.go b/weed/storage/erasure_coding/ec_test.go
index 57df09525..92b83cdc8 100644
--- a/weed/storage/erasure_coding/ec_test.go
+++ b/weed/storage/erasure_coding/ec_test.go
@@ -7,9 +7,10 @@ import (
"os"
"testing"
+ "github.com/klauspost/reedsolomon"
+
"github.com/chrislusf/seaweedfs/weed/storage/needle_map"
"github.com/chrislusf/seaweedfs/weed/storage/types"
- "github.com/klauspost/reedsolomon"
)
const (
@@ -26,14 +27,14 @@ func TestEncodingDecoding(t *testing.T) {
t.Logf("generateEcFiles: %v", err)
}
- err = WriteSortedEcxFile(baseFileName)
+ err = WriteSortedFileFromIdx(baseFileName, ".ecx")
if err != nil {
- t.Logf("WriteSortedEcxFile: %v", err)
+ t.Logf("WriteSortedFileFromIdx: %v", err)
}
err = validateFiles(baseFileName)
if err != nil {
- t.Logf("WriteSortedEcxFile: %v", err)
+ t.Logf("WriteSortedFileFromIdx: %v", err)
}
removeGeneratedFiles(baseFileName)
@@ -41,9 +42,10 @@ func TestEncodingDecoding(t *testing.T) {
}
func validateFiles(baseFileName string) error {
- cm, err := readCompactMap(baseFileName)
+ nm, err := readNeedleMap(baseFileName)
+ defer nm.Close()
if err != nil {
- return fmt.Errorf("readCompactMap: %v", err)
+ return fmt.Errorf("readNeedleMap: %v", err)
}
datFile, err := os.OpenFile(baseFileName+".dat", os.O_RDONLY, 0)
@@ -60,7 +62,7 @@ func validateFiles(baseFileName string) error {
ecFiles, err := openEcFiles(baseFileName, true)
defer closeEcFiles(ecFiles)
- err = cm.AscendingVisit(func(value needle_map.NeedleValue) error {
+ err = nm.AscendingVisit(func(value needle_map.NeedleValue) error {
return assertSame(datFile, fi.Size(), ecFiles, value.Offset, value.Size)
})
if err != nil {
diff --git a/weed/storage/erasure_coding/ec_volume.go b/weed/storage/erasure_coding/ec_volume.go
index bcae164ca..3d9aa2cff 100644
--- a/weed/storage/erasure_coding/ec_volume.go
+++ b/weed/storage/erasure_coding/ec_volume.go
@@ -9,7 +9,9 @@ import (
"sync"
"time"
+ "github.com/chrislusf/seaweedfs/weed/pb"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/storage/idx"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/chrislusf/seaweedfs/weed/storage/types"
@@ -56,6 +58,14 @@ func NewEcVolume(dir string, collection string, vid needle.VolumeId) (ev *EcVolu
return nil, fmt.Errorf("cannot open ec volume journal %s.ecj: %v", baseFileName, err)
}
+ // read volume info
+ ev.Version = needle.Version3
+ if volumeInfo, found, _ := pb.MaybeLoadVolumeInfo(baseFileName + ".vif"); found {
+ ev.Version = needle.Version(volumeInfo.Version)
+ } else {
+ pb.SaveVolumeInfo(baseFileName+".vif", &volume_server_pb.VolumeInfo{Version: uint32(ev.Version)})
+ }
+
ev.ShardLocations = make(map[ShardId][]string)
return
@@ -126,6 +136,7 @@ func (ev *EcVolume) Destroy() {
}
os.Remove(ev.FileName() + ".ecx")
os.Remove(ev.FileName() + ".ecj")
+ os.Remove(ev.FileName() + ".vif")
}
func (ev *EcVolume) FileName() string {
@@ -186,10 +197,10 @@ func (ev *EcVolume) LocateEcShardNeedle(needleId types.NeedleId, version needle.
}
func (ev *EcVolume) FindNeedleFromEcx(needleId types.NeedleId) (offset types.Offset, size uint32, err error) {
- return searchNeedleFromEcx(ev.ecxFile, ev.ecxFileSize, needleId, nil)
+ return SearchNeedleFromSortedIndex(ev.ecxFile, ev.ecxFileSize, needleId, nil)
}
-func searchNeedleFromEcx(ecxFile *os.File, ecxFileSize int64, needleId types.NeedleId, processNeedleFn func(file *os.File, offset int64) error) (offset types.Offset, size uint32, err error) {
+func SearchNeedleFromSortedIndex(ecxFile *os.File, ecxFileSize int64, needleId types.NeedleId, processNeedleFn func(file *os.File, offset int64) error) (offset types.Offset, size uint32, err error) {
var key types.NeedleId
buf := make([]byte, types.NeedleMapEntrySize)
l, h := int64(0), ecxFileSize/types.NeedleMapEntrySize
diff --git a/weed/storage/erasure_coding/ec_volume_delete.go b/weed/storage/erasure_coding/ec_volume_delete.go
index 04102ec9e..822a9e923 100644
--- a/weed/storage/erasure_coding/ec_volume_delete.go
+++ b/weed/storage/erasure_coding/ec_volume_delete.go
@@ -10,15 +10,15 @@ import (
)
var (
- markNeedleDeleted = func(file *os.File, offset int64) error {
+ MarkNeedleDeleted = func(file *os.File, offset int64) error {
b := make([]byte, types.SizeSize)
util.Uint32toBytes(b, types.TombstoneFileSize)
n, err := file.WriteAt(b, offset+types.NeedleIdSize+types.OffsetSize)
if err != nil {
- return fmt.Errorf("ecx write error: %v", err)
+ return fmt.Errorf("sorted needle write error: %v", err)
}
if n != types.SizeSize {
- return fmt.Errorf("ecx written %d bytes, expecting %d", n, types.SizeSize)
+ return fmt.Errorf("sorted needle written %d bytes, expecting %d", n, types.SizeSize)
}
return nil
}
@@ -26,7 +26,7 @@ var (
func (ev *EcVolume) DeleteNeedleFromEcx(needleId types.NeedleId) (err error) {
- _, _, err = searchNeedleFromEcx(ev.ecxFile, ev.ecxFileSize, needleId, markNeedleDeleted)
+ _, _, err = SearchNeedleFromSortedIndex(ev.ecxFile, ev.ecxFileSize, needleId, MarkNeedleDeleted)
if err != nil {
if err == NotFoundError {
@@ -81,7 +81,7 @@ func RebuildEcxFile(baseFileName string) error {
needleId := types.BytesToNeedleId(buf)
- _, _, err = searchNeedleFromEcx(ecxFile, ecxFileSize, needleId, markNeedleDeleted)
+ _, _, err = SearchNeedleFromSortedIndex(ecxFile, ecxFileSize, needleId, MarkNeedleDeleted)
if err != nil && err != NotFoundError {
ecxFile.Close()
diff --git a/weed/storage/erasure_coding/ec_volume_info.go b/weed/storage/erasure_coding/ec_volume_info.go
index c9e85c662..8ff65bb0f 100644
--- a/weed/storage/erasure_coding/ec_volume_info.go
+++ b/weed/storage/erasure_coding/ec_volume_info.go
@@ -81,6 +81,15 @@ func (b ShardBits) ShardIds() (ret []ShardId) {
return
}
+func (b ShardBits) ToUint32Slice() (ret []uint32) {
+ for i := uint32(0); i < TotalShardsCount; i++ {
+ if b.HasShardId(ShardId(i)) {
+ ret = append(ret, i)
+ }
+ }
+ return
+}
+
func (b ShardBits) ShardIdCount() (count int) {
for count = 0; b > 0; count++ {
b &= b - 1
@@ -95,3 +104,10 @@ func (b ShardBits) Minus(other ShardBits) ShardBits {
func (b ShardBits) Plus(other ShardBits) ShardBits {
return b | other
}
+
+func (b ShardBits) MinusParityShards() ShardBits {
+ for i := DataShardsCount; i < TotalShardsCount; i++ {
+ b = b.RemoveShardId(ShardId(i))
+ }
+ return b
+}
diff --git a/weed/storage/needle/crc.go b/weed/storage/needle/crc.go
index 00ea1db69..6fd910bb7 100644
--- a/weed/storage/needle/crc.go
+++ b/weed/storage/needle/crc.go
@@ -1,11 +1,11 @@
package needle
import (
- "crypto/md5"
"fmt"
- "github.com/chrislusf/seaweedfs/weed/util"
"github.com/klauspost/crc32"
+
+ "github.com/chrislusf/seaweedfs/weed/util"
)
var table = crc32.MakeTable(crc32.Castagnoli)
@@ -29,13 +29,3 @@ func (n *Needle) Etag() string {
util.Uint32toBytes(bits, uint32(n.Checksum))
return fmt.Sprintf("%x", bits)
}
-
-func (n *Needle) MD5() string {
-
- hash := md5.New()
-
- hash.Write(n.Data)
-
- return fmt.Sprintf("%x", hash.Sum(nil))
-
-}
diff --git a/weed/storage/needle/needle.go b/weed/storage/needle/needle.go
index 2f03ba87b..d3969e868 100644
--- a/weed/storage/needle/needle.go
+++ b/weed/storage/needle/needle.go
@@ -8,8 +8,6 @@ import (
"strings"
"time"
- "io/ioutil"
-
"github.com/chrislusf/seaweedfs/weed/images"
. "github.com/chrislusf/seaweedfs/weed/storage/types"
)
@@ -50,53 +48,28 @@ func (n *Needle) String() (str string) {
return
}
-func ParseUpload(r *http.Request) (
- fileName string, data []byte, mimeType string, pairMap map[string]string, isGzipped bool, originalDataSize int,
- modifiedTime uint64, ttl *TTL, isChunkedFile bool, e error) {
- pairMap = make(map[string]string)
- for k, v := range r.Header {
- if len(v) > 0 && strings.HasPrefix(k, PairNamePrefix) {
- pairMap[k] = v[0]
- }
- }
-
- if r.Method == "POST" {
- fileName, data, mimeType, isGzipped, originalDataSize, isChunkedFile, e = parseMultipart(r)
- } else {
- isGzipped = false
- mimeType = r.Header.Get("Content-Type")
- fileName = ""
- data, e = ioutil.ReadAll(r.Body)
- originalDataSize = len(data)
- }
- if e != nil {
- return
- }
-
- modifiedTime, _ = strconv.ParseUint(r.FormValue("ts"), 10, 64)
- ttl, _ = ReadTTL(r.FormValue("ttl"))
-
- return
-}
-func CreateNeedleFromRequest(r *http.Request, fixJpgOrientation bool) (n *Needle, originalSize int, e error) {
- var pairMap map[string]string
- fname, mimeType, isGzipped, isChunkedFile := "", "", false, false
+func CreateNeedleFromRequest(r *http.Request, fixJpgOrientation bool, sizeLimit int64) (n *Needle, originalSize int, e error) {
n = new(Needle)
- fname, n.Data, mimeType, pairMap, isGzipped, originalSize, n.LastModified, n.Ttl, isChunkedFile, e = ParseUpload(r)
+ pu, e := ParseUpload(r, sizeLimit)
if e != nil {
return
}
- if len(fname) < 256 {
- n.Name = []byte(fname)
+ n.Data = pu.Data
+ originalSize = pu.OriginalDataSize
+ n.LastModified = pu.ModifiedTime
+ n.Ttl = pu.Ttl
+
+ if len(pu.FileName) < 256 {
+ n.Name = []byte(pu.FileName)
n.SetHasName()
}
- if len(mimeType) < 256 {
- n.Mime = []byte(mimeType)
+ if len(pu.MimeType) < 256 {
+ n.Mime = []byte(pu.MimeType)
n.SetHasMime()
}
- if len(pairMap) != 0 {
+ if len(pu.PairMap) != 0 {
trimmedPairMap := make(map[string]string)
- for k, v := range pairMap {
+ for k, v := range pu.PairMap {
trimmedPairMap[k[len(PairNamePrefix):]] = v
}
@@ -107,7 +80,7 @@ func CreateNeedleFromRequest(r *http.Request, fixJpgOrientation bool) (n *Needle
n.SetHasPairs()
}
}
- if isGzipped {
+ if pu.IsGzipped {
n.SetGzipped()
}
if n.LastModified == 0 {
@@ -118,13 +91,13 @@ func CreateNeedleFromRequest(r *http.Request, fixJpgOrientation bool) (n *Needle
n.SetHasTtl()
}
- if isChunkedFile {
+ if pu.IsChunkedFile {
n.SetIsChunkManifest()
}
if fixJpgOrientation {
- loweredName := strings.ToLower(fname)
- if mimeType == "image/jpeg" || strings.HasSuffix(loweredName, ".jpg") || strings.HasSuffix(loweredName, ".jpeg") {
+ loweredName := strings.ToLower(pu.FileName)
+ if pu.MimeType == "image/jpeg" || strings.HasSuffix(loweredName, ".jpg") || strings.HasSuffix(loweredName, ".jpeg") {
n.Data = images.FixJpgOrientation(n.Data)
}
}
diff --git a/weed/storage/needle/needle_parse_multipart.go b/weed/storage/needle/needle_parse_multipart.go
deleted file mode 100644
index 8be1a1da4..000000000
--- a/weed/storage/needle/needle_parse_multipart.go
+++ /dev/null
@@ -1,109 +0,0 @@
-package needle
-
-import (
- "github.com/chrislusf/seaweedfs/weed/glog"
- "github.com/chrislusf/seaweedfs/weed/util"
-
- "io"
- "io/ioutil"
- "mime"
- "net/http"
- "path"
- "strconv"
- "strings"
-)
-
-func parseMultipart(r *http.Request) (
- fileName string, data []byte, mimeType string, isGzipped bool, originalDataSize int, isChunkedFile bool, e error) {
- defer func() {
- if e != nil && r.Body != nil {
- io.Copy(ioutil.Discard, r.Body)
- r.Body.Close()
- }
- }()
- form, fe := r.MultipartReader()
- if fe != nil {
- glog.V(0).Infoln("MultipartReader [ERROR]", fe)
- e = fe
- return
- }
-
- //first multi-part item
- part, fe := form.NextPart()
- if fe != nil {
- glog.V(0).Infoln("Reading Multi part [ERROR]", fe)
- e = fe
- return
- }
-
- fileName = part.FileName()
- if fileName != "" {
- fileName = path.Base(fileName)
- }
-
- data, e = ioutil.ReadAll(part)
- if e != nil {
- glog.V(0).Infoln("Reading Content [ERROR]", e)
- return
- }
-
- //if the filename is empty string, do a search on the other multi-part items
- for fileName == "" {
- part2, fe := form.NextPart()
- if fe != nil {
- break // no more or on error, just safely break
- }
-
- fName := part2.FileName()
-
- //found the first multi-part has filename
- if fName != "" {
- data2, fe2 := ioutil.ReadAll(part2)
- if fe2 != nil {
- glog.V(0).Infoln("Reading Content [ERROR]", fe2)
- e = fe2
- return
- }
-
- //update
- data = data2
- fileName = path.Base(fName)
- break
- }
- }
-
- originalDataSize = len(data)
-
- isChunkedFile, _ = strconv.ParseBool(r.FormValue("cm"))
-
- if !isChunkedFile {
-
- dotIndex := strings.LastIndex(fileName, ".")
- ext, mtype := "", ""
- if dotIndex > 0 {
- ext = strings.ToLower(fileName[dotIndex:])
- mtype = mime.TypeByExtension(ext)
- }
- contentType := part.Header.Get("Content-Type")
- if contentType != "" && mtype != contentType {
- mimeType = contentType //only return mime type if not deductable
- mtype = contentType
- }
-
- if part.Header.Get("Content-Encoding") == "gzip" {
- if unzipped, e := util.UnGzipData(data); e == nil {
- originalDataSize = len(unzipped)
- }
- isGzipped = true
- } else if util.IsGzippable(ext, mtype, data) {
- if compressedData, err := util.GzipData(data); err == nil {
- if len(data) > len(compressedData) {
- data = compressedData
- isGzipped = true
- }
- }
- }
- }
-
- return
-}
diff --git a/weed/storage/needle/needle_parse_upload.go b/weed/storage/needle/needle_parse_upload.go
new file mode 100644
index 000000000..85526aaa8
--- /dev/null
+++ b/weed/storage/needle/needle_parse_upload.go
@@ -0,0 +1,166 @@
+package needle
+
+import (
+ "fmt"
+ "io"
+ "io/ioutil"
+ "mime"
+ "net/http"
+ "path"
+ "strconv"
+ "strings"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/util"
+)
+
+type ParsedUpload struct {
+ FileName string
+ Data []byte
+ MimeType string
+ PairMap map[string]string
+ IsGzipped bool
+ OriginalDataSize int
+ ModifiedTime uint64
+ Ttl *TTL
+ IsChunkedFile bool
+ UncompressedData []byte
+}
+
+func ParseUpload(r *http.Request, sizeLimit int64) (pu *ParsedUpload, e error) {
+ pu = &ParsedUpload{}
+ pu.PairMap = make(map[string]string)
+ for k, v := range r.Header {
+ if len(v) > 0 && strings.HasPrefix(k, PairNamePrefix) {
+ pu.PairMap[k] = v[0]
+ }
+ }
+
+ if r.Method == "POST" {
+ e = parseMultipart(r, sizeLimit, pu)
+ } else {
+ e = parsePut(r, sizeLimit, pu)
+ }
+ if e != nil {
+ return
+ }
+
+ pu.ModifiedTime, _ = strconv.ParseUint(r.FormValue("ts"), 10, 64)
+ pu.Ttl, _ = ReadTTL(r.FormValue("ttl"))
+
+ pu.OriginalDataSize = len(pu.Data)
+ pu.UncompressedData = pu.Data
+ if pu.IsGzipped {
+ if unzipped, e := util.UnGzipData(pu.Data); e == nil {
+ pu.OriginalDataSize = len(unzipped)
+ pu.UncompressedData = unzipped
+ }
+ } else if shouldGzip, _ := util.IsGzippableFileType("", pu.MimeType); shouldGzip {
+ if compressedData, err := util.GzipData(pu.Data); err == nil {
+ pu.Data = compressedData
+ pu.IsGzipped = true
+ }
+ }
+
+ return
+}
+
+func parsePut(r *http.Request, sizeLimit int64, pu *ParsedUpload) (e error) {
+ pu.IsGzipped = r.Header.Get("Content-Encoding") == "gzip"
+ pu.MimeType = r.Header.Get("Content-Type")
+ pu.FileName = ""
+ pu.Data, e = ioutil.ReadAll(io.LimitReader(r.Body, sizeLimit+1))
+ if e == io.EOF || int64(pu.OriginalDataSize) == sizeLimit+1 {
+ io.Copy(ioutil.Discard, r.Body)
+ }
+ r.Body.Close()
+ return nil
+}
+
+func parseMultipart(r *http.Request, sizeLimit int64, pu *ParsedUpload) (e error) {
+ defer func() {
+ if e != nil && r.Body != nil {
+ io.Copy(ioutil.Discard, r.Body)
+ r.Body.Close()
+ }
+ }()
+ form, fe := r.MultipartReader()
+ if fe != nil {
+ glog.V(0).Infoln("MultipartReader [ERROR]", fe)
+ e = fe
+ return
+ }
+
+ //first multi-part item
+ part, fe := form.NextPart()
+ if fe != nil {
+ glog.V(0).Infoln("Reading Multi part [ERROR]", fe)
+ e = fe
+ return
+ }
+
+ pu.FileName = part.FileName()
+ if pu.FileName != "" {
+ pu.FileName = path.Base(pu.FileName)
+ }
+
+ pu.Data, e = ioutil.ReadAll(io.LimitReader(part, sizeLimit+1))
+ if e != nil {
+ glog.V(0).Infoln("Reading Content [ERROR]", e)
+ return
+ }
+ if len(pu.Data) == int(sizeLimit)+1 {
+ e = fmt.Errorf("file over the limited %d bytes", sizeLimit)
+ return
+ }
+
+ //if the filename is empty string, do a search on the other multi-part items
+ for pu.FileName == "" {
+ part2, fe := form.NextPart()
+ if fe != nil {
+ break // no more or on error, just safely break
+ }
+
+ fName := part2.FileName()
+
+ //found the first multi-part has filename
+ if fName != "" {
+ data2, fe2 := ioutil.ReadAll(io.LimitReader(part2, sizeLimit+1))
+ if fe2 != nil {
+ glog.V(0).Infoln("Reading Content [ERROR]", fe2)
+ e = fe2
+ return
+ }
+ if len(data2) == int(sizeLimit)+1 {
+ e = fmt.Errorf("file over the limited %d bytes", sizeLimit)
+ return
+ }
+
+ //update
+ pu.Data = data2
+ pu.FileName = path.Base(fName)
+ break
+ }
+ }
+
+ pu.IsChunkedFile, _ = strconv.ParseBool(r.FormValue("cm"))
+
+ if !pu.IsChunkedFile {
+
+ dotIndex := strings.LastIndex(pu.FileName, ".")
+ ext, mtype := "", ""
+ if dotIndex > 0 {
+ ext = strings.ToLower(pu.FileName[dotIndex:])
+ mtype = mime.TypeByExtension(ext)
+ }
+ contentType := part.Header.Get("Content-Type")
+ if contentType != "" && contentType != "application/octet-stream" && mtype != contentType {
+ pu.MimeType = contentType //only return mime type if not deductable
+ mtype = contentType
+ }
+
+ pu.IsGzipped = part.Header.Get("Content-Encoding") == "gzip"
+ }
+
+ return
+}
diff --git a/weed/storage/needle/needle_read_write.go b/weed/storage/needle/needle_read_write.go
index 8e5d18b1a..7f8aa4823 100644
--- a/weed/storage/needle/needle_read_write.go
+++ b/weed/storage/needle/needle_read_write.go
@@ -125,13 +125,13 @@ func (n *Needle) prepareWriteBuffer(version Version) ([]byte, uint32, int64, err
return writeBytes, 0, 0, fmt.Errorf("Unsupported Version! (%d)", version)
}
-func (n *Needle) Append(w backend.DataStorageBackend, version Version) (offset uint64, size uint32, actualSize int64, err error) {
+func (n *Needle) Append(w backend.BackendStorageFile, version Version) (offset uint64, size uint32, actualSize int64, err error) {
if end, _, e := w.GetStat(); e == nil {
- defer func(w backend.DataStorageBackend, off int64) {
+ defer func(w backend.BackendStorageFile, off int64) {
if err != nil {
if te := w.Truncate(end); te != nil {
- glog.V(0).Infof("Failed to truncate %s back to %d with error: %v", w.String(), end, te)
+ glog.V(0).Infof("Failed to truncate %s back to %d with error: %v", w.Name(), end, te)
}
}
}(w, end)
@@ -150,7 +150,7 @@ func (n *Needle) Append(w backend.DataStorageBackend, version Version) (offset u
return offset, size, actualSize, err
}
-func ReadNeedleBlob(r backend.DataStorageBackend, offset int64, size uint32, version Version) (dataSlice []byte, err error) {
+func ReadNeedleBlob(r backend.BackendStorageFile, offset int64, size uint32, version Version) (dataSlice []byte, err error) {
dataSize := GetActualSize(size, version)
dataSlice = make([]byte, int(dataSize))
@@ -191,7 +191,7 @@ func (n *Needle) ReadBytes(bytes []byte, offset int64, size uint32, version Vers
}
// ReadData hydrates the needle from the file, with only n.Id is set.
-func (n *Needle) ReadData(r backend.DataStorageBackend, offset int64, size uint32, version Version) (err error) {
+func (n *Needle) ReadData(r backend.BackendStorageFile, offset int64, size uint32, version Version) (err error) {
bytes, err := ReadNeedleBlob(r, offset, size, version)
if err != nil {
return err
@@ -266,7 +266,7 @@ func (n *Needle) readNeedleDataVersion2(bytes []byte) (err error) {
return nil
}
-func ReadNeedleHeader(r backend.DataStorageBackend, version Version, offset int64) (n *Needle, bytes []byte, bodyLength int64, err error) {
+func ReadNeedleHeader(r backend.BackendStorageFile, version Version, offset int64) (n *Needle, bytes []byte, bodyLength int64, err error) {
n = new(Needle)
if version == Version1 || version == Version2 || version == Version3 {
bytes = make([]byte, NeedleHeaderSize)
@@ -301,7 +301,7 @@ func NeedleBodyLength(needleSize uint32, version Version) int64 {
//n should be a needle already read the header
//the input stream will read until next file entry
-func (n *Needle) ReadNeedleBody(r backend.DataStorageBackend, version Version, offset int64, bodyLength int64) (bytes []byte, err error) {
+func (n *Needle) ReadNeedleBody(r backend.BackendStorageFile, version Version, offset int64, bodyLength int64) (bytes []byte, err error) {
if bodyLength <= 0 {
return nil, nil
diff --git a/weed/storage/needle/volume_ttl.go b/weed/storage/needle/volume_ttl.go
index 4a169870d..179057876 100644
--- a/weed/storage/needle/volume_ttl.go
+++ b/weed/storage/needle/volume_ttl.go
@@ -69,6 +69,9 @@ func (t *TTL) ToBytes(output []byte) {
}
func (t *TTL) ToUint32() (output uint32) {
+ if t == nil || t.Count == 0 {
+ return 0
+ }
output = uint32(t.Count) << 8
output += uint32(t.Unit)
return output
diff --git a/weed/storage/needle_map/btree_map.go b/weed/storage/needle_map/btree_map.go
deleted file mode 100644
index a26c5e068..000000000
--- a/weed/storage/needle_map/btree_map.go
+++ /dev/null
@@ -1,53 +0,0 @@
-package needle_map
-
-import (
- . "github.com/chrislusf/seaweedfs/weed/storage/types"
- "github.com/google/btree"
-)
-
-//This map assumes mostly inserting increasing keys
-type BtreeMap struct {
- tree *btree.BTree
-}
-
-func NewBtreeMap() *BtreeMap {
- return &BtreeMap{
- tree: btree.New(32),
- }
-}
-
-func (cm *BtreeMap) Set(key NeedleId, offset Offset, size uint32) (oldOffset Offset, oldSize uint32) {
- found := cm.tree.ReplaceOrInsert(NeedleValue{key, offset, size})
- if found != nil {
- old := found.(NeedleValue)
- return old.Offset, old.Size
- }
- return
-}
-
-func (cm *BtreeMap) Delete(key NeedleId) (oldSize uint32) {
- found := cm.tree.Delete(NeedleValue{key, Offset{}, 0})
- if found != nil {
- old := found.(NeedleValue)
- return old.Size
- }
- return
-}
-func (cm *BtreeMap) Get(key NeedleId) (*NeedleValue, bool) {
- found := cm.tree.Get(NeedleValue{key, Offset{}, 0})
- if found != nil {
- old := found.(NeedleValue)
- return &old, true
- }
- return nil, false
-}
-
-// Visit visits all entries or stop if any error when visiting
-func (cm *BtreeMap) AscendingVisit(visit func(NeedleValue) error) (ret error) {
- cm.tree.Ascend(func(item btree.Item) bool {
- needle := item.(NeedleValue)
- ret = visit(needle)
- return ret == nil
- })
- return ret
-}
diff --git a/weed/storage/needle_map/compact_map_test.go b/weed/storage/needle_map/compact_map_test.go
index 3bad85727..7eea3969a 100644
--- a/weed/storage/needle_map/compact_map_test.go
+++ b/weed/storage/needle_map/compact_map_test.go
@@ -8,7 +8,14 @@ import (
func TestOverflow2(t *testing.T) {
m := NewCompactMap()
- m.Set(NeedleId(150088), ToOffset(8), 3000073)
+ _, oldSize := m.Set(NeedleId(150088), ToOffset(8), 3000073)
+ if oldSize != 0 {
+ t.Fatalf("expecting no previous data")
+ }
+ _, oldSize = m.Set(NeedleId(150088), ToOffset(8), 3000073)
+ if oldSize != 3000073 {
+ t.Fatalf("expecting previous data size is %d, not %d", 3000073, oldSize)
+ }
m.Set(NeedleId(150073), ToOffset(8), 3000073)
m.Set(NeedleId(150089), ToOffset(8), 3000073)
m.Set(NeedleId(150076), ToOffset(8), 3000073)
diff --git a/weed/storage/needle_map/memdb.go b/weed/storage/needle_map/memdb.go
new file mode 100644
index 000000000..a52d52a10
--- /dev/null
+++ b/weed/storage/needle_map/memdb.go
@@ -0,0 +1,119 @@
+package needle_map
+
+import (
+ "fmt"
+ "os"
+
+ "github.com/syndtr/goleveldb/leveldb"
+ "github.com/syndtr/goleveldb/leveldb/opt"
+ "github.com/syndtr/goleveldb/leveldb/storage"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/storage/idx"
+ . "github.com/chrislusf/seaweedfs/weed/storage/types"
+ "github.com/chrislusf/seaweedfs/weed/util"
+)
+
+//This map uses in memory level db
+type MemDb struct {
+ db *leveldb.DB
+}
+
+func NewMemDb() *MemDb {
+ opts := &opt.Options{}
+
+ var err error
+ t := &MemDb{}
+ if t.db, err = leveldb.Open(storage.NewMemStorage(), opts); err != nil {
+ glog.V(0).Infof("MemDb fails to open: %v", err)
+ return nil
+ }
+
+ return t
+}
+
+func (cm *MemDb) Set(key NeedleId, offset Offset, size uint32) error {
+
+ bytes := ToBytes(key, offset, size)
+
+ if err := cm.db.Put(bytes[0:NeedleIdSize], bytes[NeedleIdSize:NeedleIdSize+OffsetSize+SizeSize], nil); err != nil {
+ return fmt.Errorf("failed to write temp leveldb: %v", err)
+ }
+ return nil
+}
+
+func (cm *MemDb) Delete(key NeedleId) error {
+ bytes := make([]byte, NeedleIdSize)
+ NeedleIdToBytes(bytes, key)
+ return cm.db.Delete(bytes, nil)
+
+}
+func (cm *MemDb) Get(key NeedleId) (*NeedleValue, bool) {
+ bytes := make([]byte, NeedleIdSize)
+ NeedleIdToBytes(bytes[0:NeedleIdSize], key)
+ data, err := cm.db.Get(bytes, nil)
+ if err != nil || len(data) != OffsetSize+SizeSize {
+ return nil, false
+ }
+ offset := BytesToOffset(data[0:OffsetSize])
+ size := util.BytesToUint32(data[OffsetSize : OffsetSize+SizeSize])
+ return &NeedleValue{Key: key, Offset: offset, Size: size}, true
+}
+
+// Visit visits all entries or stop if any error when visiting
+func (cm *MemDb) AscendingVisit(visit func(NeedleValue) error) (ret error) {
+ iter := cm.db.NewIterator(nil, nil)
+ for iter.Next() {
+ key := BytesToNeedleId(iter.Key())
+ data := iter.Value()
+ offset := BytesToOffset(data[0:OffsetSize])
+ size := util.BytesToUint32(data[OffsetSize : OffsetSize+SizeSize])
+
+ needle := NeedleValue{Key: key, Offset: offset, Size: size}
+ ret = visit(needle)
+ if ret != nil {
+ return
+ }
+ }
+ iter.Release()
+ ret = iter.Error()
+
+ return
+}
+
+func (cm *MemDb) SaveToIdx(idxName string) (ret error) {
+ idxFile, err := os.OpenFile(idxName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644)
+ if err != nil {
+ return
+ }
+ defer idxFile.Close()
+
+ return cm.AscendingVisit(func(value NeedleValue) error {
+ if value.Offset.IsZero() || value.Size == TombstoneFileSize {
+ return nil
+ }
+ _, err := idxFile.Write(value.ToBytes())
+ return err
+ })
+
+}
+
+func (cm *MemDb) LoadFromIdx(idxName string) (ret error) {
+ idxFile, err := os.OpenFile(idxName, os.O_RDONLY, 0644)
+ if err != nil {
+ return
+ }
+ defer idxFile.Close()
+
+ return idx.WalkIndexFile(idxFile, func(key NeedleId, offset Offset, size uint32) error {
+ if offset.IsZero() || size == TombstoneFileSize {
+ return cm.Delete(key)
+ }
+ return cm.Set(key, offset, size)
+ })
+
+}
+
+func (cm *MemDb) Close() {
+ cm.db.Close()
+}
diff --git a/weed/storage/needle_map/memdb_test.go b/weed/storage/needle_map/memdb_test.go
new file mode 100644
index 000000000..7b45d23f8
--- /dev/null
+++ b/weed/storage/needle_map/memdb_test.go
@@ -0,0 +1,23 @@
+package needle_map
+
+import (
+ "testing"
+
+ "github.com/chrislusf/seaweedfs/weed/storage/types"
+)
+
+func BenchmarkMemDb(b *testing.B) {
+ b.ReportAllocs()
+ for i := 0; i < b.N; i++ {
+ nm := NewMemDb()
+
+ nid := types.NeedleId(345)
+ offset := types.Offset{
+ OffsetHigher: types.OffsetHigher{},
+ OffsetLower: types.OffsetLower{},
+ }
+ nm.Set(nid, offset, 324)
+ nm.Close()
+ }
+
+}
diff --git a/weed/storage/needle_map_leveldb.go b/weed/storage/needle_map_leveldb.go
index ef8571e83..3bb258559 100644
--- a/weed/storage/needle_map_leveldb.go
+++ b/weed/storage/needle_map_leveldb.go
@@ -128,8 +128,17 @@ func (m *LevelDbNeedleMap) Delete(key NeedleId, offset Offset) error {
}
func (m *LevelDbNeedleMap) Close() {
- m.indexFile.Close()
- m.db.Close()
+ indexFileName := m.indexFile.Name()
+ if err := m.indexFile.Sync(); err != nil {
+ glog.Warningf("sync file %s failed: %v", indexFileName, err)
+ }
+ if err := m.indexFile.Close(); err != nil {
+ glog.Warningf("close index file %s failed: %v", indexFileName, err)
+ }
+
+ if err := m.db.Close(); err != nil {
+ glog.Warningf("close levelDB failed: %v", err)
+ }
}
func (m *LevelDbNeedleMap) Destroy() error {
diff --git a/weed/storage/needle_map_memory.go b/weed/storage/needle_map_memory.go
index ee639a7e6..84197912f 100644
--- a/weed/storage/needle_map_memory.go
+++ b/weed/storage/needle_map_memory.go
@@ -22,24 +22,11 @@ func NewCompactNeedleMap(file *os.File) *NeedleMap {
return nm
}
-func NewBtreeNeedleMap(file *os.File) *NeedleMap {
- nm := &NeedleMap{
- m: needle_map.NewBtreeMap(),
- }
- nm.indexFile = file
- return nm
-}
-
func LoadCompactNeedleMap(file *os.File) (*NeedleMap, error) {
nm := NewCompactNeedleMap(file)
return doLoading(file, nm)
}
-func LoadBtreeNeedleMap(file *os.File) (*NeedleMap, error) {
- nm := NewBtreeNeedleMap(file)
- return doLoading(file, nm)
-}
-
func doLoading(file *os.File, nm *NeedleMap) (*NeedleMap, error) {
e := idx.WalkIndexFile(file, func(key NeedleId, offset Offset, size uint32) error {
nm.MaybeSetMaxFileKey(key)
@@ -47,14 +34,12 @@ func doLoading(file *os.File, nm *NeedleMap) (*NeedleMap, error) {
nm.FileCounter++
nm.FileByteCounter = nm.FileByteCounter + uint64(size)
oldOffset, oldSize := nm.m.Set(NeedleId(key), offset, size)
- // glog.V(3).Infoln("reading key", key, "offset", offset*NeedlePaddingSize, "size", size, "oldSize", oldSize)
if !oldOffset.IsZero() && oldSize != TombstoneFileSize {
nm.DeletionCounter++
nm.DeletionByteCounter = nm.DeletionByteCounter + uint64(oldSize)
}
} else {
oldSize := nm.m.Delete(NeedleId(key))
- // glog.V(3).Infoln("removing key", key, "offset", offset*NeedlePaddingSize, "size", size, "oldSize", oldSize)
nm.DeletionCounter++
nm.DeletionByteCounter = nm.DeletionByteCounter + uint64(oldSize)
}
@@ -79,6 +64,10 @@ func (nm *NeedleMap) Delete(key NeedleId, offset Offset) error {
return nm.appendToIndexFile(key, offset, TombstoneFileSize)
}
func (nm *NeedleMap) Close() {
+ indexFileName := nm.indexFile.Name()
+ if err := nm.indexFile.Sync(); err != nil {
+ glog.Warningf("sync file %s failed, %v", indexFileName, err)
+ }
_ = nm.indexFile.Close()
}
func (nm *NeedleMap) Destroy() error {
diff --git a/weed/storage/needle_map_metric_test.go b/weed/storage/needle_map_metric_test.go
index 539f83a87..ae2177a30 100644
--- a/weed/storage/needle_map_metric_test.go
+++ b/weed/storage/needle_map_metric_test.go
@@ -1,17 +1,18 @@
package storage
import (
- "github.com/chrislusf/seaweedfs/weed/glog"
- . "github.com/chrislusf/seaweedfs/weed/storage/types"
"io/ioutil"
"math/rand"
"testing"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ . "github.com/chrislusf/seaweedfs/weed/storage/types"
)
func TestFastLoadingNeedleMapMetrics(t *testing.T) {
idxFile, _ := ioutil.TempFile("", "tmp.idx")
- nm := NewBtreeNeedleMap(idxFile)
+ nm := NewCompactNeedleMap(idxFile)
for i := 0; i < 10000; i++ {
nm.Put(Uint64ToNeedleId(uint64(i+1)), Uint32ToOffset(uint32(0)), uint32(1))
diff --git a/weed/storage/needle_map_sorted_file.go b/weed/storage/needle_map_sorted_file.go
new file mode 100644
index 000000000..e6f9258f3
--- /dev/null
+++ b/weed/storage/needle_map_sorted_file.go
@@ -0,0 +1,105 @@
+package storage
+
+import (
+ "os"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle_map"
+ . "github.com/chrislusf/seaweedfs/weed/storage/types"
+)
+
+type SortedFileNeedleMap struct {
+ baseNeedleMapper
+ baseFileName string
+ dbFile *os.File
+ dbFileSize int64
+}
+
+func NewSortedFileNeedleMap(baseFileName string, indexFile *os.File) (m *SortedFileNeedleMap, err error) {
+ m = &SortedFileNeedleMap{baseFileName: baseFileName}
+ m.indexFile = indexFile
+ fileName := baseFileName + ".sdx"
+ if !isSortedFileFresh(fileName, indexFile) {
+ glog.V(0).Infof("Start to Generate %s from %s", fileName, indexFile.Name())
+ erasure_coding.WriteSortedFileFromIdx(baseFileName, ".sdx")
+ glog.V(0).Infof("Finished Generating %s from %s", fileName, indexFile.Name())
+ }
+ glog.V(1).Infof("Opening %s...", fileName)
+
+ if m.dbFile, err = os.Open(baseFileName + ".sdx"); err != nil {
+ return
+ }
+ dbStat, _ := m.dbFile.Stat()
+ m.dbFileSize = dbStat.Size()
+ glog.V(1).Infof("Loading %s...", indexFile.Name())
+ mm, indexLoadError := newNeedleMapMetricFromIndexFile(indexFile)
+ if indexLoadError != nil {
+ return nil, indexLoadError
+ }
+ m.mapMetric = *mm
+ return
+}
+
+func isSortedFileFresh(dbFileName string, indexFile *os.File) bool {
+ // normally we always write to index file first
+ dbFile, err := os.Open(dbFileName)
+ if err != nil {
+ return false
+ }
+ defer dbFile.Close()
+ dbStat, dbStatErr := dbFile.Stat()
+ indexStat, indexStatErr := indexFile.Stat()
+ if dbStatErr != nil || indexStatErr != nil {
+ glog.V(0).Infof("Can not stat file: %v and %v", dbStatErr, indexStatErr)
+ return false
+ }
+
+ return dbStat.ModTime().After(indexStat.ModTime())
+}
+
+func (m *SortedFileNeedleMap) Get(key NeedleId) (element *needle_map.NeedleValue, ok bool) {
+ offset, size, err := erasure_coding.SearchNeedleFromSortedIndex(m.dbFile, m.dbFileSize, key, nil)
+ ok = err == nil
+ return &needle_map.NeedleValue{Key: key, Offset: offset, Size: size}, ok
+
+}
+
+func (m *SortedFileNeedleMap) Put(key NeedleId, offset Offset, size uint32) error {
+ return os.ErrInvalid
+}
+
+func (m *SortedFileNeedleMap) Delete(key NeedleId, offset Offset) error {
+
+ _, size, err := erasure_coding.SearchNeedleFromSortedIndex(m.dbFile, m.dbFileSize, key, nil)
+
+ if err != nil {
+ if err == erasure_coding.NotFoundError {
+ return nil
+ }
+ return err
+ }
+
+ if size == TombstoneFileSize {
+ return nil
+ }
+
+ // write to index file first
+ if err := m.appendToIndexFile(key, offset, TombstoneFileSize); err != nil {
+ return err
+ }
+ _, _, err = erasure_coding.SearchNeedleFromSortedIndex(m.dbFile, m.dbFileSize, key, erasure_coding.MarkNeedleDeleted)
+
+ return err
+}
+
+func (m *SortedFileNeedleMap) Close() {
+ m.indexFile.Close()
+ m.dbFile.Close()
+}
+
+func (m *SortedFileNeedleMap) Destroy() error {
+ m.Close()
+ os.Remove(m.indexFile.Name())
+ return os.Remove(m.baseFileName + ".sdx")
+}
diff --git a/weed/storage/store.go b/weed/storage/store.go
index 4d1061bed..e29680f6f 100644
--- a/weed/storage/store.go
+++ b/weed/storage/store.go
@@ -2,14 +2,19 @@ package storage
import (
"fmt"
+ "path/filepath"
+ "strings"
"sync/atomic"
+ "google.golang.org/grpc"
+
"github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/stats"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
. "github.com/chrislusf/seaweedfs/weed/storage/types"
- "google.golang.org/grpc"
)
const (
@@ -60,7 +65,7 @@ func NewStore(grpcDialOption grpc.DialOption, port int, ip, publicUrl string, di
return
}
func (s *Store) AddVolume(volumeId needle.VolumeId, collection string, needleMapKind NeedleMapType, replicaPlacement string, ttlString string, preallocate int64, MemoryMapMaxSizeMb uint32) error {
- rt, e := NewReplicaPlacementFromString(replicaPlacement)
+ rt, e := super_block.NewReplicaPlacementFromString(replicaPlacement)
if e != nil {
return e
}
@@ -101,7 +106,7 @@ func (s *Store) FindFreeLocation() (ret *DiskLocation) {
}
return ret
}
-func (s *Store) addVolume(vid needle.VolumeId, collection string, needleMapKind NeedleMapType, replicaPlacement *ReplicaPlacement, ttl *needle.TTL, preallocate int64, memoryMapMaxSizeMb uint32) error {
+func (s *Store) addVolume(vid needle.VolumeId, collection string, needleMapKind NeedleMapType, replicaPlacement *super_block.ReplicaPlacement, ttl *needle.TTL, preallocate int64, memoryMapMaxSizeMb uint32) error {
if s.findVolume(vid) != nil {
return fmt.Errorf("Volume Id %d already exists!", vid)
}
@@ -126,10 +131,10 @@ func (s *Store) addVolume(vid needle.VolumeId, collection string, needleMapKind
return fmt.Errorf("No more free space left")
}
-func (s *Store) Status() []*VolumeInfo {
+func (s *Store) VolumeInfos() []*VolumeInfo {
var stats []*VolumeInfo
for _, location := range s.Locations {
- location.RLock()
+ location.volumesLock.RLock()
for k, v := range location.volumes {
s := &VolumeInfo{
Id: needle.VolumeId(k),
@@ -140,13 +145,14 @@ func (s *Store) Status() []*VolumeInfo {
FileCount: int(v.FileCount()),
DeleteCount: int(v.DeletedCount()),
DeletedByteCount: v.DeletedSize(),
- ReadOnly: v.readOnly,
+ ReadOnly: v.noWriteOrDelete || v.noWriteCanDelete,
Ttl: v.Ttl,
CompactRevision: uint32(v.CompactionRevision),
}
+ s.RemoteStorageName, s.RemoteStorageKey = v.RemoteStorageNameKey()
stats = append(stats, s)
}
- location.RUnlock()
+ location.volumesLock.RUnlock()
}
sortVolumeInfos(stats)
return stats
@@ -167,7 +173,7 @@ func (s *Store) CollectHeartbeat() *master_pb.Heartbeat {
for _, location := range s.Locations {
var deleteVids []needle.VolumeId
maxVolumeCount = maxVolumeCount + location.MaxVolumeCount
- location.RLock()
+ location.volumesLock.RLock()
for _, v := range location.volumes {
if maxFileKey < v.MaxFileKey() {
maxFileKey = v.MaxFileKey()
@@ -184,16 +190,16 @@ func (s *Store) CollectHeartbeat() *master_pb.Heartbeat {
fileSize, _, _ := v.FileStat()
collectionVolumeSize[v.Collection] += fileSize
}
- location.RUnlock()
+ location.volumesLock.RUnlock()
if len(deleteVids) > 0 {
// delete expired volumes.
- location.Lock()
+ location.volumesLock.Lock()
for _, vid := range deleteVids {
location.deleteVolumeById(vid)
glog.V(0).Infoln("volume", vid, "is deleted.")
}
- location.Unlock()
+ location.volumesLock.Unlock()
}
}
@@ -223,11 +229,11 @@ func (s *Store) Close() {
func (s *Store) WriteVolumeNeedle(i needle.VolumeId, n *needle.Needle) (size uint32, isUnchanged bool, err error) {
if v := s.findVolume(i); v != nil {
- if v.readOnly {
+ if v.noWriteOrDelete || v.noWriteCanDelete {
err = fmt.Errorf("volume %d is read only", i)
return
}
- if MaxPossibleVolumeSize >= v.ContentSize()+uint64(needle.GetActualSize(size, v.version)) {
+ if MaxPossibleVolumeSize >= v.ContentSize()+uint64(needle.GetActualSize(size, v.Version())) {
_, size, isUnchanged, err = v.writeNeedle(n)
} else {
err = fmt.Errorf("volume size limit %d exceeded! current size is %d", s.GetVolumeSizeLimit(), v.ContentSize())
@@ -241,10 +247,10 @@ func (s *Store) WriteVolumeNeedle(i needle.VolumeId, n *needle.Needle) (size uin
func (s *Store) DeleteVolumeNeedle(i needle.VolumeId, n *needle.Needle) (uint32, error) {
if v := s.findVolume(i); v != nil {
- if v.readOnly {
+ if v.noWriteOrDelete {
return 0, fmt.Errorf("volume %d is read only", i)
}
- if MaxPossibleVolumeSize >= v.ContentSize()+uint64(needle.GetActualSize(0, v.version)) {
+ if MaxPossibleVolumeSize >= v.ContentSize()+uint64(needle.GetActualSize(0, v.Version())) {
return v.deleteNeedle(n)
} else {
return 0, fmt.Errorf("volume size limit %d exceeded! current size is %d", s.GetVolumeSizeLimit(), v.ContentSize())
@@ -273,7 +279,7 @@ func (s *Store) MarkVolumeReadonly(i needle.VolumeId) error {
if v == nil {
return fmt.Errorf("volume %d not found", i)
}
- v.readOnly = true
+ v.noWriteOrDelete = true
return nil
}
@@ -343,6 +349,31 @@ func (s *Store) DeleteVolume(i needle.VolumeId) error {
return fmt.Errorf("volume %d not found on disk", i)
}
+func (s *Store) ConfigureVolume(i needle.VolumeId, replication string) error {
+
+ for _, location := range s.Locations {
+ fileInfo, found := location.LocateVolume(i)
+ if !found {
+ continue
+ }
+ // load, modify, save
+ baseFileName := strings.TrimSuffix(fileInfo.Name(), filepath.Ext(fileInfo.Name()))
+ vifFile := filepath.Join(location.Directory, baseFileName+".vif")
+ volumeInfo, _, err := pb.MaybeLoadVolumeInfo(vifFile)
+ if err != nil {
+ return fmt.Errorf("volume %d fail to load vif", i)
+ }
+ volumeInfo.Replication = replication
+ err = pb.SaveVolumeInfo(vifFile, volumeInfo)
+ if err != nil {
+ return fmt.Errorf("volume %d fail to save vif", i)
+ }
+ return nil
+ }
+
+ return fmt.Errorf("volume %d not found on disk", i)
+}
+
func (s *Store) SetVolumeSizeLimit(x uint64) {
atomic.StoreUint64(&s.volumeSizeLimit, x)
}
diff --git a/weed/storage/store_ec.go b/weed/storage/store_ec.go
index 7e3f1a46c..e423e7dca 100644
--- a/weed/storage/store_ec.go
+++ b/weed/storage/store_ec.go
@@ -8,6 +8,8 @@ import (
"sync"
"time"
+ "github.com/klauspost/reedsolomon"
+
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
@@ -16,7 +18,6 @@ import (
"github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/chrislusf/seaweedfs/weed/storage/types"
- "github.com/klauspost/reedsolomon"
)
func (s *Store) CollectErasureCodingHeartbeat() *master_pb.Heartbeat {
@@ -115,19 +116,11 @@ func (s *Store) DestroyEcVolume(vid needle.VolumeId) {
}
}
-func (s *Store) ReadEcShardNeedle(ctx context.Context, vid needle.VolumeId, n *needle.Needle) (int, error) {
+func (s *Store) ReadEcShardNeedle(vid needle.VolumeId, n *needle.Needle) (int, error) {
for _, location := range s.Locations {
if localEcVolume, found := location.FindEcVolume(vid); found {
- // read the volume version
- for localEcVolume.Version == 0 {
- err := s.readEcVolumeVersion(ctx, vid, localEcVolume)
- time.Sleep(1357 * time.Millisecond)
- glog.V(0).Infof("ReadEcShardNeedle vid %d version:%v: %v", vid, localEcVolume.Version, err)
- }
- version := localEcVolume.Version
-
- offset, size, intervals, err := localEcVolume.LocateEcShardNeedle(n.Id, version)
+ offset, size, intervals, err := localEcVolume.LocateEcShardNeedle(n.Id, localEcVolume.Version)
if err != nil {
return 0, fmt.Errorf("locate in local ec volume: %v", err)
}
@@ -140,7 +133,7 @@ func (s *Store) ReadEcShardNeedle(ctx context.Context, vid needle.VolumeId, n *n
if len(intervals) > 1 {
glog.V(3).Infof("ReadEcShardNeedle needle id %s intervals:%+v", n.String(), intervals)
}
- bytes, isDeleted, err := s.readEcShardIntervals(ctx, vid, n.Id, localEcVolume, intervals)
+ bytes, isDeleted, err := s.readEcShardIntervals(vid, n.Id, localEcVolume, intervals)
if err != nil {
return 0, fmt.Errorf("ReadEcShardIntervals: %v", err)
}
@@ -148,7 +141,7 @@ func (s *Store) ReadEcShardNeedle(ctx context.Context, vid needle.VolumeId, n *n
return 0, fmt.Errorf("ec entry %s is deleted", n.Id)
}
- err = n.ReadBytes(bytes, offset.ToAcutalOffset(), size, version)
+ err = n.ReadBytes(bytes, offset.ToAcutalOffset(), size, localEcVolume.Version)
if err != nil {
return 0, fmt.Errorf("readbytes: %v", err)
}
@@ -159,30 +152,14 @@ func (s *Store) ReadEcShardNeedle(ctx context.Context, vid needle.VolumeId, n *n
return 0, fmt.Errorf("ec shard %d not found", vid)
}
-func (s *Store) readEcVolumeVersion(ctx context.Context, vid needle.VolumeId, ecVolume *erasure_coding.EcVolume) (err error) {
-
- interval := erasure_coding.Interval{
- BlockIndex: 0,
- InnerBlockOffset: 0,
- Size: _SuperBlockSize,
- IsLargeBlock: true, // it could be large block, but ok in this place
- LargeBlockRowsCount: 0,
- }
- data, _, err := s.readEcShardIntervals(ctx, vid, 0, ecVolume, []erasure_coding.Interval{interval})
- if err == nil {
- ecVolume.Version = needle.Version(data[0])
- }
- return
-}
-
-func (s *Store) readEcShardIntervals(ctx context.Context, vid needle.VolumeId, needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, intervals []erasure_coding.Interval) (data []byte, is_deleted bool, err error) {
+func (s *Store) readEcShardIntervals(vid needle.VolumeId, needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, intervals []erasure_coding.Interval) (data []byte, is_deleted bool, err error) {
- if err = s.cachedLookupEcShardLocations(ctx, ecVolume); err != nil {
+ if err = s.cachedLookupEcShardLocations(ecVolume); err != nil {
return nil, false, fmt.Errorf("failed to locate shard via master grpc %s: %v", s.MasterAddress, err)
}
for i, interval := range intervals {
- if d, isDeleted, e := s.readOneEcShardInterval(ctx, needleId, ecVolume, interval); e != nil {
+ if d, isDeleted, e := s.readOneEcShardInterval(needleId, ecVolume, interval); e != nil {
return nil, isDeleted, e
} else {
if isDeleted {
@@ -198,7 +175,7 @@ func (s *Store) readEcShardIntervals(ctx context.Context, vid needle.VolumeId, n
return
}
-func (s *Store) readOneEcShardInterval(ctx context.Context, needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, interval erasure_coding.Interval) (data []byte, is_deleted bool, err error) {
+func (s *Store) readOneEcShardInterval(needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, interval erasure_coding.Interval) (data []byte, is_deleted bool, err error) {
shardId, actualOffset := interval.ToShardIdAndOffset(erasure_coding.ErasureCodingLargeBlockSize, erasure_coding.ErasureCodingSmallBlockSize)
data = make([]byte, interval.Size)
if shard, found := ecVolume.FindEcVolumeShard(shardId); found {
@@ -213,7 +190,7 @@ func (s *Store) readOneEcShardInterval(ctx context.Context, needleId types.Needl
// try reading directly
if hasShardIdLocation {
- _, is_deleted, err = s.readRemoteEcShardInterval(ctx, sourceDataNodes, needleId, ecVolume.VolumeId, shardId, data, actualOffset)
+ _, is_deleted, err = s.readRemoteEcShardInterval(sourceDataNodes, needleId, ecVolume.VolumeId, shardId, data, actualOffset)
if err == nil {
return
}
@@ -222,7 +199,7 @@ func (s *Store) readOneEcShardInterval(ctx context.Context, needleId types.Needl
}
// try reading by recovering from other shards
- _, is_deleted, err = s.recoverOneRemoteEcShardInterval(ctx, needleId, ecVolume, shardId, data, actualOffset)
+ _, is_deleted, err = s.recoverOneRemoteEcShardInterval(needleId, ecVolume, shardId, data, actualOffset)
if err == nil {
return
}
@@ -238,7 +215,7 @@ func forgetShardId(ecVolume *erasure_coding.EcVolume, shardId erasure_coding.Sha
ecVolume.ShardLocationsLock.Unlock()
}
-func (s *Store) cachedLookupEcShardLocations(ctx context.Context, ecVolume *erasure_coding.EcVolume) (err error) {
+func (s *Store) cachedLookupEcShardLocations(ecVolume *erasure_coding.EcVolume) (err error) {
shardCount := len(ecVolume.ShardLocations)
if shardCount < erasure_coding.DataShardsCount &&
@@ -257,7 +234,7 @@ func (s *Store) cachedLookupEcShardLocations(ctx context.Context, ecVolume *eras
req := &master_pb.LookupEcVolumeRequest{
VolumeId: uint32(ecVolume.VolumeId),
}
- resp, err := masterClient.LookupEcVolume(ctx, req)
+ resp, err := masterClient.LookupEcVolume(context.Background(), req)
if err != nil {
return fmt.Errorf("lookup ec volume %d: %v", ecVolume.VolumeId, err)
}
@@ -281,7 +258,7 @@ func (s *Store) cachedLookupEcShardLocations(ctx context.Context, ecVolume *eras
return
}
-func (s *Store) readRemoteEcShardInterval(ctx context.Context, sourceDataNodes []string, needleId types.NeedleId, vid needle.VolumeId, shardId erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
+func (s *Store) readRemoteEcShardInterval(sourceDataNodes []string, needleId types.NeedleId, vid needle.VolumeId, shardId erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
if len(sourceDataNodes) == 0 {
return 0, false, fmt.Errorf("failed to find ec shard %d.%d", vid, shardId)
@@ -289,7 +266,7 @@ func (s *Store) readRemoteEcShardInterval(ctx context.Context, sourceDataNodes [
for _, sourceDataNode := range sourceDataNodes {
glog.V(3).Infof("read remote ec shard %d.%d from %s", vid, shardId, sourceDataNode)
- n, is_deleted, err = s.doReadRemoteEcShardInterval(ctx, sourceDataNode, needleId, vid, shardId, buf, offset)
+ n, is_deleted, err = s.doReadRemoteEcShardInterval(sourceDataNode, needleId, vid, shardId, buf, offset)
if err == nil {
return
}
@@ -299,12 +276,12 @@ func (s *Store) readRemoteEcShardInterval(ctx context.Context, sourceDataNodes [
return
}
-func (s *Store) doReadRemoteEcShardInterval(ctx context.Context, sourceDataNode string, needleId types.NeedleId, vid needle.VolumeId, shardId erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
+func (s *Store) doReadRemoteEcShardInterval(sourceDataNode string, needleId types.NeedleId, vid needle.VolumeId, shardId erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
err = operation.WithVolumeServerClient(sourceDataNode, s.grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
// copy data slice
- shardReadClient, err := client.VolumeEcShardRead(ctx, &volume_server_pb.VolumeEcShardReadRequest{
+ shardReadClient, err := client.VolumeEcShardRead(context.Background(), &volume_server_pb.VolumeEcShardReadRequest{
VolumeId: uint32(vid),
ShardId: uint32(shardId),
Offset: offset,
@@ -339,7 +316,7 @@ func (s *Store) doReadRemoteEcShardInterval(ctx context.Context, sourceDataNode
return
}
-func (s *Store) recoverOneRemoteEcShardInterval(ctx context.Context, needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, shardIdToRecover erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
+func (s *Store) recoverOneRemoteEcShardInterval(needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, shardIdToRecover erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
glog.V(3).Infof("recover ec shard %d.%d from other locations", ecVolume.VolumeId, shardIdToRecover)
enc, err := reedsolomon.New(erasure_coding.DataShardsCount, erasure_coding.ParityShardsCount)
@@ -367,7 +344,7 @@ func (s *Store) recoverOneRemoteEcShardInterval(ctx context.Context, needleId ty
go func(shardId erasure_coding.ShardId, locations []string) {
defer wg.Done()
data := make([]byte, len(buf))
- nRead, isDeleted, readErr := s.readRemoteEcShardInterval(ctx, locations, needleId, ecVolume.VolumeId, shardId, data, offset)
+ nRead, isDeleted, readErr := s.readRemoteEcShardInterval(locations, needleId, ecVolume.VolumeId, shardId, data, offset)
if readErr != nil {
glog.V(3).Infof("recover: readRemoteEcShardInterval %d.%d %d bytes from %+v: %v", ecVolume.VolumeId, shardId, nRead, locations, readErr)
forgetShardId(ecVolume, shardId)
diff --git a/weed/storage/store_ec_delete.go b/weed/storage/store_ec_delete.go
index e027d2887..4a75fb20b 100644
--- a/weed/storage/store_ec_delete.go
+++ b/weed/storage/store_ec_delete.go
@@ -12,9 +12,9 @@ import (
"github.com/chrislusf/seaweedfs/weed/storage/types"
)
-func (s *Store) DeleteEcShardNeedle(ctx context.Context, ecVolume *erasure_coding.EcVolume, n *needle.Needle, cookie types.Cookie) (int64, error) {
+func (s *Store) DeleteEcShardNeedle(ecVolume *erasure_coding.EcVolume, n *needle.Needle, cookie types.Cookie) (int64, error) {
- count, err := s.ReadEcShardNeedle(ctx, ecVolume.VolumeId, n)
+ count, err := s.ReadEcShardNeedle(ecVolume.VolumeId, n)
if err != nil {
return 0, err
@@ -24,7 +24,7 @@ func (s *Store) DeleteEcShardNeedle(ctx context.Context, ecVolume *erasure_codin
return 0, fmt.Errorf("unexpected cookie %x", cookie)
}
- if err = s.doDeleteNeedleFromAtLeastOneRemoteEcShards(ctx, ecVolume, n.Id); err != nil {
+ if err = s.doDeleteNeedleFromAtLeastOneRemoteEcShards(ecVolume, n.Id); err != nil {
return 0, err
}
@@ -32,7 +32,7 @@ func (s *Store) DeleteEcShardNeedle(ctx context.Context, ecVolume *erasure_codin
}
-func (s *Store) doDeleteNeedleFromAtLeastOneRemoteEcShards(ctx context.Context, ecVolume *erasure_coding.EcVolume, needleId types.NeedleId) error {
+func (s *Store) doDeleteNeedleFromAtLeastOneRemoteEcShards(ecVolume *erasure_coding.EcVolume, needleId types.NeedleId) error {
_, _, intervals, err := ecVolume.LocateEcShardNeedle(needleId, ecVolume.Version)
@@ -43,13 +43,13 @@ func (s *Store) doDeleteNeedleFromAtLeastOneRemoteEcShards(ctx context.Context,
shardId, _ := intervals[0].ToShardIdAndOffset(erasure_coding.ErasureCodingLargeBlockSize, erasure_coding.ErasureCodingSmallBlockSize)
hasDeletionSuccess := false
- err = s.doDeleteNeedleFromRemoteEcShardServers(ctx, shardId, ecVolume, needleId)
+ err = s.doDeleteNeedleFromRemoteEcShardServers(shardId, ecVolume, needleId)
if err == nil {
hasDeletionSuccess = true
}
for shardId = erasure_coding.DataShardsCount; shardId < erasure_coding.TotalShardsCount; shardId++ {
- if parityDeletionError := s.doDeleteNeedleFromRemoteEcShardServers(ctx, shardId, ecVolume, needleId); parityDeletionError == nil {
+ if parityDeletionError := s.doDeleteNeedleFromRemoteEcShardServers(shardId, ecVolume, needleId); parityDeletionError == nil {
hasDeletionSuccess = true
}
}
@@ -62,7 +62,7 @@ func (s *Store) doDeleteNeedleFromAtLeastOneRemoteEcShards(ctx context.Context,
}
-func (s *Store) doDeleteNeedleFromRemoteEcShardServers(ctx context.Context, shardId erasure_coding.ShardId, ecVolume *erasure_coding.EcVolume, needleId types.NeedleId) error {
+func (s *Store) doDeleteNeedleFromRemoteEcShardServers(shardId erasure_coding.ShardId, ecVolume *erasure_coding.EcVolume, needleId types.NeedleId) error {
ecVolume.ShardLocationsLock.RLock()
sourceDataNodes, hasShardLocations := ecVolume.ShardLocations[shardId]
@@ -74,7 +74,7 @@ func (s *Store) doDeleteNeedleFromRemoteEcShardServers(ctx context.Context, shar
for _, sourceDataNode := range sourceDataNodes {
glog.V(4).Infof("delete from remote ec shard %d.%d from %s", ecVolume.VolumeId, shardId, sourceDataNode)
- err := s.doDeleteNeedleFromRemoteEcShard(ctx, sourceDataNode, ecVolume.VolumeId, ecVolume.Collection, ecVolume.Version, needleId)
+ err := s.doDeleteNeedleFromRemoteEcShard(sourceDataNode, ecVolume.VolumeId, ecVolume.Collection, ecVolume.Version, needleId)
if err != nil {
return err
}
@@ -85,12 +85,12 @@ func (s *Store) doDeleteNeedleFromRemoteEcShardServers(ctx context.Context, shar
}
-func (s *Store) doDeleteNeedleFromRemoteEcShard(ctx context.Context, sourceDataNode string, vid needle.VolumeId, collection string, version needle.Version, needleId types.NeedleId) error {
+func (s *Store) doDeleteNeedleFromRemoteEcShard(sourceDataNode string, vid needle.VolumeId, collection string, version needle.Version, needleId types.NeedleId) error {
return operation.WithVolumeServerClient(sourceDataNode, s.grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
// copy data slice
- _, err := client.VolumeEcBlobDelete(ctx, &volume_server_pb.VolumeEcBlobDeleteRequest{
+ _, err := client.VolumeEcBlobDelete(context.Background(), &volume_server_pb.VolumeEcBlobDeleteRequest{
VolumeId: uint32(vid),
Collection: collection,
FileKey: uint64(needleId),
diff --git a/weed/storage/store_vacuum.go b/weed/storage/store_vacuum.go
index b1f1a6277..e94d9b516 100644
--- a/weed/storage/store_vacuum.go
+++ b/weed/storage/store_vacuum.go
@@ -16,7 +16,8 @@ func (s *Store) CheckCompactVolume(volumeId needle.VolumeId) (float64, error) {
}
func (s *Store) CompactVolume(vid needle.VolumeId, preallocate int64, compactionBytePerSecond int64) error {
if v := s.findVolume(vid); v != nil {
- return v.Compact(preallocate, compactionBytePerSecond)
+ return v.Compact2(preallocate) // compactionBytePerSecond
+ // return v.Compact(preallocate, compactionBytePerSecond)
}
return fmt.Errorf("volume id %d is not found during compact", vid)
}
diff --git a/weed/storage/replica_placement.go b/weed/storage/super_block/replica_placement.go
similarity index 98%
rename from weed/storage/replica_placement.go
rename to weed/storage/super_block/replica_placement.go
index c1aca52eb..fcccbba7d 100644
--- a/weed/storage/replica_placement.go
+++ b/weed/storage/super_block/replica_placement.go
@@ -1,4 +1,4 @@
-package storage
+package super_block
import (
"errors"
diff --git a/weed/storage/replica_placement_test.go b/weed/storage/super_block/replica_placement_test.go
similarity index 93%
rename from weed/storage/replica_placement_test.go
rename to weed/storage/super_block/replica_placement_test.go
index 7968af7cb..7742ba548 100644
--- a/weed/storage/replica_placement_test.go
+++ b/weed/storage/super_block/replica_placement_test.go
@@ -1,4 +1,4 @@
-package storage
+package super_block
import (
"testing"
diff --git a/weed/storage/super_block/super_block.go b/weed/storage/super_block/super_block.go
new file mode 100644
index 000000000..f48cd0bdc
--- /dev/null
+++ b/weed/storage/super_block/super_block.go
@@ -0,0 +1,69 @@
+package super_block
+
+import (
+ "github.com/golang/protobuf/proto"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/util"
+)
+
+const (
+ SuperBlockSize = 8
+)
+
+/*
+* Super block currently has 8 bytes allocated for each volume.
+* Byte 0: version, 1 or 2
+* Byte 1: Replica Placement strategy, 000, 001, 002, 010, etc
+* Byte 2 and byte 3: Time to live. See TTL for definition
+* Byte 4 and byte 5: The number of times the volume has been compacted.
+* Rest bytes: Reserved
+ */
+type SuperBlock struct {
+ Version needle.Version
+ ReplicaPlacement *ReplicaPlacement
+ Ttl *needle.TTL
+ CompactionRevision uint16
+ Extra *master_pb.SuperBlockExtra
+ ExtraSize uint16
+}
+
+func (s *SuperBlock) BlockSize() int {
+ switch s.Version {
+ case needle.Version2, needle.Version3:
+ return SuperBlockSize + int(s.ExtraSize)
+ }
+ return SuperBlockSize
+}
+
+func (s *SuperBlock) Bytes() []byte {
+ header := make([]byte, SuperBlockSize)
+ header[0] = byte(s.Version)
+ header[1] = s.ReplicaPlacement.Byte()
+ s.Ttl.ToBytes(header[2:4])
+ util.Uint16toBytes(header[4:6], s.CompactionRevision)
+
+ if s.Extra != nil {
+ extraData, err := proto.Marshal(s.Extra)
+ if err != nil {
+ glog.Fatalf("cannot marshal super block extra %+v: %v", s.Extra, err)
+ }
+ extraSize := len(extraData)
+ if extraSize > 256*256-2 {
+ // reserve a couple of bits for future extension
+ glog.Fatalf("super block extra size is %d bigger than %d", extraSize, 256*256-2)
+ }
+ s.ExtraSize = uint16(extraSize)
+ util.Uint16toBytes(header[6:8], s.ExtraSize)
+
+ header = append(header, extraData...)
+ }
+
+ return header
+}
+
+func (s *SuperBlock) Initialized() bool {
+ return s.ReplicaPlacement != nil && s.Ttl != nil
+}
diff --git a/weed/storage/super_block/super_block_read.go.go b/weed/storage/super_block/super_block_read.go.go
new file mode 100644
index 000000000..9eb12e116
--- /dev/null
+++ b/weed/storage/super_block/super_block_read.go.go
@@ -0,0 +1,44 @@
+package super_block
+
+import (
+ "fmt"
+
+ "github.com/golang/protobuf/proto"
+
+ "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/util"
+)
+
+// ReadSuperBlock reads from data file and load it into volume's super block
+func ReadSuperBlock(datBackend backend.BackendStorageFile) (superBlock SuperBlock, err error) {
+
+ header := make([]byte, SuperBlockSize)
+ if _, e := datBackend.ReadAt(header, 0); e != nil {
+ err = fmt.Errorf("cannot read volume %s super block: %v", datBackend.Name(), e)
+ return
+ }
+
+ superBlock.Version = needle.Version(header[0])
+ if superBlock.ReplicaPlacement, err = NewReplicaPlacementFromByte(header[1]); err != nil {
+ err = fmt.Errorf("cannot read replica type: %s", err.Error())
+ return
+ }
+ superBlock.Ttl = needle.LoadTTLFromBytes(header[2:4])
+ superBlock.CompactionRevision = util.BytesToUint16(header[4:6])
+ superBlock.ExtraSize = util.BytesToUint16(header[6:8])
+
+ if superBlock.ExtraSize > 0 {
+ // read more
+ extraData := make([]byte, int(superBlock.ExtraSize))
+ superBlock.Extra = &master_pb.SuperBlockExtra{}
+ err = proto.Unmarshal(extraData, superBlock.Extra)
+ if err != nil {
+ err = fmt.Errorf("cannot read volume %s super block extra: %v", datBackend.Name(), err)
+ return
+ }
+ }
+
+ return
+}
diff --git a/weed/storage/volume_super_block_test.go b/weed/storage/super_block/super_block_test.go
similarity index 86%
rename from weed/storage/volume_super_block_test.go
rename to weed/storage/super_block/super_block_test.go
index 06ad8a5d3..25699070d 100644
--- a/weed/storage/volume_super_block_test.go
+++ b/weed/storage/super_block/super_block_test.go
@@ -1,4 +1,4 @@
-package storage
+package super_block
import (
"testing"
@@ -10,7 +10,7 @@ func TestSuperBlockReadWrite(t *testing.T) {
rp, _ := NewReplicaPlacementFromByte(byte(001))
ttl, _ := needle.ReadTTL("15d")
s := &SuperBlock{
- version: needle.CurrentVersion,
+ Version: needle.CurrentVersion,
ReplicaPlacement: rp,
Ttl: ttl,
}
diff --git a/weed/storage/volume.go b/weed/storage/volume.go
index e85696eab..7da83de7a 100644
--- a/weed/storage/volume.go
+++ b/weed/storage/volume.go
@@ -2,18 +2,19 @@ package storage
import (
"fmt"
+ "path"
+ "strconv"
+ "sync"
+ "time"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/stats"
"github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"github.com/chrislusf/seaweedfs/weed/storage/types"
- "path"
- "strconv"
- "sync"
- "time"
-
"github.com/chrislusf/seaweedfs/weed/glog"
)
@@ -21,15 +22,17 @@ type Volume struct {
Id needle.VolumeId
dir string
Collection string
- DataBackend backend.DataStorageBackend
+ DataBackend backend.BackendStorageFile
nm NeedleMapper
needleMapKind NeedleMapType
- readOnly bool
+ noWriteOrDelete bool // if readonly, either noWriteOrDelete or noWriteCanDelete
+ noWriteCanDelete bool // if readonly, either noWriteOrDelete or noWriteCanDelete
+ hasRemoteFile bool // if the volume has a remote file
MemoryMapMaxSizeMb uint32
- SuperBlock
+ super_block.SuperBlock
- dataFileAccessLock sync.Mutex
+ dataFileAccessLock sync.RWMutex
lastModifiedTsSeconds uint64 //unix time in seconds
lastAppendAtNs uint64 //unix time in nanoseconds
@@ -37,18 +40,20 @@ type Volume struct {
lastCompactRevision uint16
isCompacting bool
+
+ volumeInfo *volume_server_pb.VolumeInfo
}
-func NewVolume(dirname string, collection string, id needle.VolumeId, needleMapKind NeedleMapType, replicaPlacement *ReplicaPlacement, ttl *needle.TTL, preallocate int64, memoryMapMaxSizeMb uint32) (v *Volume, e error) {
+func NewVolume(dirname string, collection string, id needle.VolumeId, needleMapKind NeedleMapType, replicaPlacement *super_block.ReplicaPlacement, ttl *needle.TTL, preallocate int64, memoryMapMaxSizeMb uint32) (v *Volume, e error) {
// if replicaPlacement is nil, the superblock will be loaded from disk
v = &Volume{dir: dirname, Collection: collection, Id: id, MemoryMapMaxSizeMb: memoryMapMaxSizeMb}
- v.SuperBlock = SuperBlock{ReplicaPlacement: replicaPlacement, Ttl: ttl}
+ v.SuperBlock = super_block.SuperBlock{ReplicaPlacement: replicaPlacement, Ttl: ttl}
v.needleMapKind = needleMapKind
e = v.load(true, true, needleMapKind, preallocate)
return
}
func (v *Volume) String() string {
- return fmt.Sprintf("Id:%v, dir:%s, Collection:%s, dataFile:%v, nm:%v, readOnly:%v", v.Id, v.dir, v.Collection, v.DataBackend, v.nm, v.readOnly)
+ return fmt.Sprintf("Id:%v, dir:%s, Collection:%s, dataFile:%v, nm:%v, noWrite:%v canDelete:%v", v.Id, v.dir, v.Collection, v.DataBackend, v.nm, v.noWriteOrDelete || v.noWriteCanDelete, v.noWriteCanDelete)
}
func VolumeFileName(dir string, collection string, id int) (fileName string) {
@@ -65,12 +70,15 @@ func (v *Volume) FileName() (fileName string) {
}
func (v *Volume) Version() needle.Version {
- return v.SuperBlock.Version()
+ if v.volumeInfo.Version != 0 {
+ v.SuperBlock.Version = needle.Version(v.volumeInfo.Version)
+ }
+ return v.SuperBlock.Version
}
func (v *Volume) FileStat() (datSize uint64, idxSize uint64, modTime time.Time) {
- v.dataFileAccessLock.Lock()
- defer v.dataFileAccessLock.Unlock()
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
if v.DataBackend == nil {
return
@@ -80,13 +88,13 @@ func (v *Volume) FileStat() (datSize uint64, idxSize uint64, modTime time.Time)
if e == nil {
return uint64(datFileSize), v.nm.IndexFileSize(), modTime
}
- glog.V(0).Infof("Failed to read file size %s %v", v.DataBackend.String(), e)
+ glog.V(0).Infof("Failed to read file size %s %v", v.DataBackend.Name(), e)
return // -1 causes integer overflow and the volume to become unwritable.
}
func (v *Volume) ContentSize() uint64 {
- v.dataFileAccessLock.Lock()
- defer v.dataFileAccessLock.Unlock()
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
if v.nm == nil {
return 0
}
@@ -94,8 +102,8 @@ func (v *Volume) ContentSize() uint64 {
}
func (v *Volume) DeletedSize() uint64 {
- v.dataFileAccessLock.Lock()
- defer v.dataFileAccessLock.Unlock()
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
if v.nm == nil {
return 0
}
@@ -103,8 +111,8 @@ func (v *Volume) DeletedSize() uint64 {
}
func (v *Volume) FileCount() uint64 {
- v.dataFileAccessLock.Lock()
- defer v.dataFileAccessLock.Unlock()
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
if v.nm == nil {
return 0
}
@@ -112,8 +120,8 @@ func (v *Volume) FileCount() uint64 {
}
func (v *Volume) DeletedCount() uint64 {
- v.dataFileAccessLock.Lock()
- defer v.dataFileAccessLock.Unlock()
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
if v.nm == nil {
return 0
}
@@ -121,8 +129,8 @@ func (v *Volume) DeletedCount() uint64 {
}
func (v *Volume) MaxFileKey() types.NeedleId {
- v.dataFileAccessLock.Lock()
- defer v.dataFileAccessLock.Unlock()
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
if v.nm == nil {
return 0
}
@@ -130,8 +138,8 @@ func (v *Volume) MaxFileKey() types.NeedleId {
}
func (v *Volume) IndexFileSize() uint64 {
- v.dataFileAccessLock.Lock()
- defer v.dataFileAccessLock.Unlock()
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
if v.nm == nil {
return 0
}
@@ -172,9 +180,9 @@ func (v *Volume) expired(volumeSizeLimit uint64) bool {
if v.Ttl == nil || v.Ttl.Minutes() == 0 {
return false
}
- glog.V(1).Infof("now:%v lastModified:%v", time.Now().Unix(), v.lastModifiedTsSeconds)
+ glog.V(2).Infof("now:%v lastModified:%v", time.Now().Unix(), v.lastModifiedTsSeconds)
livedMinutes := (time.Now().Unix() - int64(v.lastModifiedTsSeconds)) / 60
- glog.V(1).Infof("ttl:%v lived:%v", v.Ttl, livedMinutes)
+ glog.V(2).Infof("ttl:%v lived:%v", v.Ttl, livedMinutes)
if int64(v.Ttl.Minutes()) < livedMinutes {
return true
}
@@ -200,18 +208,32 @@ func (v *Volume) expiredLongEnough(maxDelayMinutes uint32) bool {
func (v *Volume) ToVolumeInformationMessage() *master_pb.VolumeInformationMessage {
size, _, modTime := v.FileStat()
- return &master_pb.VolumeInformationMessage{
+ volumInfo := &master_pb.VolumeInformationMessage{
Id: uint32(v.Id),
Size: size,
Collection: v.Collection,
- FileCount: uint64(v.FileCount()),
- DeleteCount: uint64(v.DeletedCount()),
+ FileCount: v.FileCount(),
+ DeleteCount: v.DeletedCount(),
DeletedByteCount: v.DeletedSize(),
- ReadOnly: v.readOnly,
+ ReadOnly: v.noWriteOrDelete || v.noWriteCanDelete,
ReplicaPlacement: uint32(v.ReplicaPlacement.Byte()),
Version: uint32(v.Version()),
Ttl: v.Ttl.ToUint32(),
CompactRevision: uint32(v.SuperBlock.CompactionRevision),
ModifiedAtSecond: modTime.Unix(),
}
+
+ volumInfo.RemoteStorageName, volumInfo.RemoteStorageKey = v.RemoteStorageNameKey()
+
+ return volumInfo
+}
+
+func (v *Volume) RemoteStorageNameKey() (storageName, storageKey string) {
+ if v.volumeInfo == nil {
+ return
+ }
+ if len(v.volumeInfo.GetFiles()) == 0 {
+ return
+ }
+ return v.volumeInfo.GetFiles()[0].BackendName(), v.volumeInfo.GetFiles()[0].GetKey()
}
diff --git a/weed/storage/volume_backup.go b/weed/storage/volume_backup.go
index fe0506917..f7075fe2b 100644
--- a/weed/storage/volume_backup.go
+++ b/weed/storage/volume_backup.go
@@ -6,17 +6,19 @@ import (
"io"
"os"
+ "google.golang.org/grpc"
+
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/storage/idx"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
. "github.com/chrislusf/seaweedfs/weed/storage/types"
- "google.golang.org/grpc"
)
func (v *Volume) GetVolumeSyncStatus() *volume_server_pb.VolumeSyncStatusResponse {
- v.dataFileAccessLock.Lock()
- defer v.dataFileAccessLock.Unlock()
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
var syncStatus = &volume_server_pb.VolumeSyncStatusResponse{}
if datSize, _, err := v.DataBackend.GetStat(); err == nil {
@@ -62,8 +64,6 @@ update needle map when receiving new .dat bytes. But seems not necessary now.)
func (v *Volume) IncrementalBackup(volumeServer string, grpcDialOption grpc.DialOption) error {
- ctx := context.Background()
-
startFromOffset, _, _ := v.FileStat()
appendAtNs, err := v.findLastAppendAtNs()
if err != nil {
@@ -74,7 +74,7 @@ func (v *Volume) IncrementalBackup(volumeServer string, grpcDialOption grpc.Dial
err = operation.WithVolumeServerClient(volumeServer, grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
- stream, err := client.VolumeIncrementalCopy(ctx, &volume_server_pb.VolumeIncrementalCopyRequest{
+ stream, err := client.VolumeIncrementalCopy(context.Background(), &volume_server_pb.VolumeIncrementalCopyRequest{
VolumeId: uint32(v.Id),
SinceNs: appendAtNs,
})
@@ -108,7 +108,7 @@ func (v *Volume) IncrementalBackup(volumeServer string, grpcDialOption grpc.Dial
}
// add to needle map
- return ScanVolumeFileFrom(v.version, v.DataBackend, int64(startFromOffset), &VolumeFileScanner4GenIdx{v: v})
+ return ScanVolumeFileFrom(v.Version(), v.DataBackend, int64(startFromOffset), &VolumeFileScanner4GenIdx{v: v})
}
@@ -154,11 +154,11 @@ func (v *Volume) locateLastAppendEntry() (Offset, error) {
func (v *Volume) readAppendAtNs(offset Offset) (uint64, error) {
- n, _, bodyLength, err := needle.ReadNeedleHeader(v.DataBackend, v.SuperBlock.version, offset.ToAcutalOffset())
+ n, _, bodyLength, err := needle.ReadNeedleHeader(v.DataBackend, v.SuperBlock.Version, offset.ToAcutalOffset())
if err != nil {
return 0, fmt.Errorf("ReadNeedleHeader: %v", err)
}
- _, err = n.ReadNeedleBody(v.DataBackend, v.SuperBlock.version, offset.ToAcutalOffset()+int64(NeedleHeaderSize), bodyLength)
+ _, err = n.ReadNeedleBody(v.DataBackend, v.SuperBlock.Version, offset.ToAcutalOffset()+int64(NeedleHeaderSize), bodyLength)
if err != nil {
return 0, fmt.Errorf("ReadNeedleBody offset %d, bodyLength %d: %v", offset.ToAcutalOffset(), bodyLength, err)
}
@@ -244,7 +244,7 @@ type VolumeFileScanner4GenIdx struct {
v *Volume
}
-func (scanner *VolumeFileScanner4GenIdx) VisitSuperBlock(superBlock SuperBlock) error {
+func (scanner *VolumeFileScanner4GenIdx) VisitSuperBlock(superBlock super_block.SuperBlock) error {
return nil
}
diff --git a/weed/storage/volume_checking.go b/weed/storage/volume_checking.go
index 61b59e9f7..a65c2a3ff 100644
--- a/weed/storage/volume_checking.go
+++ b/weed/storage/volume_checking.go
@@ -55,7 +55,7 @@ func readIndexEntryAtOffset(indexFile *os.File, offset int64) (bytes []byte, err
return
}
-func verifyNeedleIntegrity(datFile backend.DataStorageBackend, v needle.Version, offset int64, key NeedleId, size uint32) (lastAppendAtNs uint64, err error) {
+func verifyNeedleIntegrity(datFile backend.BackendStorageFile, v needle.Version, offset int64, key NeedleId, size uint32) (lastAppendAtNs uint64, err error) {
n := new(needle.Needle)
if err = n.ReadData(datFile, offset, size, v); err != nil {
return n.AppendAtNs, err
diff --git a/weed/storage/volume_create.go b/weed/storage/volume_create.go
index b27a62990..ffcb246a4 100644
--- a/weed/storage/volume_create.go
+++ b/weed/storage/volume_create.go
@@ -9,10 +9,13 @@ import (
"github.com/chrislusf/seaweedfs/weed/storage/backend"
)
-func createVolumeFile(fileName string, preallocate int64, memoryMapSizeMB uint32) (backend.DataStorageBackend, error) {
+func createVolumeFile(fileName string, preallocate int64, memoryMapSizeMB uint32) (backend.BackendStorageFile, error) {
file, e := os.OpenFile(fileName, os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0644)
+ if e != nil {
+ return nil, e
+ }
if preallocate > 0 {
glog.V(0).Infof("Preallocated disk space for %s is not supported", fileName)
}
- return backend.NewDiskFile(file), e
+ return backend.NewDiskFile(file), nil
}
diff --git a/weed/storage/volume_create_linux.go b/weed/storage/volume_create_linux.go
index e3305d991..ee599ac32 100644
--- a/weed/storage/volume_create_linux.go
+++ b/weed/storage/volume_create_linux.go
@@ -10,11 +10,14 @@ import (
"github.com/chrislusf/seaweedfs/weed/storage/backend"
)
-func createVolumeFile(fileName string, preallocate int64, memoryMapSizeMB uint32) (backend.DataStorageBackend, error) {
+func createVolumeFile(fileName string, preallocate int64, memoryMapSizeMB uint32) (backend.BackendStorageFile, error) {
file, e := os.OpenFile(fileName, os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0644)
+ if e != nil {
+ return nil, e
+ }
if preallocate != 0 {
syscall.Fallocate(int(file.Fd()), 1, 0, preallocate)
glog.V(0).Infof("Preallocated %d bytes disk space for %s", preallocate, fileName)
}
- return backend.NewDiskFile(file), e
+ return backend.NewDiskFile(file), nil
}
diff --git a/weed/storage/volume_create_windows.go b/weed/storage/volume_create_windows.go
index 81536810b..e1c0b961f 100644
--- a/weed/storage/volume_create_windows.go
+++ b/weed/storage/volume_create_windows.go
@@ -11,18 +11,23 @@ import (
"github.com/chrislusf/seaweedfs/weed/storage/backend/memory_map/os_overloads"
)
-func createVolumeFile(fileName string, preallocate int64, memoryMapSizeMB uint32) (backend.DataStorageBackend, error) {
-
+func createVolumeFile(fileName string, preallocate int64, memoryMapSizeMB uint32) (backend.BackendStorageFile, error) {
if preallocate > 0 {
glog.V(0).Infof("Preallocated disk space for %s is not supported", fileName)
}
if memoryMapSizeMB > 0 {
file, e := os_overloads.OpenFile(fileName, windows.O_RDWR|windows.O_CREAT, 0644, true)
- return memory_map.NewMemoryMappedFile(file, memoryMapSizeMB), e
+ if e != nil {
+ return nil, e
+ }
+ return memory_map.NewMemoryMappedFile(file, memoryMapSizeMB), nil
} else {
file, e := os_overloads.OpenFile(fileName, windows.O_RDWR|windows.O_CREAT|windows.O_TRUNC, 0644, false)
- return backend.NewDiskFile(file), e
+ if e != nil {
+ return nil, e
+ }
+ return backend.NewDiskFile(file), nil
}
}
diff --git a/weed/storage/volume_info.go b/weed/storage/volume_info.go
index 111058b6e..313818cde 100644
--- a/weed/storage/volume_info.go
+++ b/weed/storage/volume_info.go
@@ -6,37 +6,42 @@ import (
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
)
type VolumeInfo struct {
- Id needle.VolumeId
- Size uint64
- ReplicaPlacement *ReplicaPlacement
- Ttl *needle.TTL
- Collection string
- Version needle.Version
- FileCount int
- DeleteCount int
- DeletedByteCount uint64
- ReadOnly bool
- CompactRevision uint32
- ModifiedAtSecond int64
+ Id needle.VolumeId
+ Size uint64
+ ReplicaPlacement *super_block.ReplicaPlacement
+ Ttl *needle.TTL
+ Collection string
+ Version needle.Version
+ FileCount int
+ DeleteCount int
+ DeletedByteCount uint64
+ ReadOnly bool
+ CompactRevision uint32
+ ModifiedAtSecond int64
+ RemoteStorageName string
+ RemoteStorageKey string
}
func NewVolumeInfo(m *master_pb.VolumeInformationMessage) (vi VolumeInfo, err error) {
vi = VolumeInfo{
- Id: needle.VolumeId(m.Id),
- Size: m.Size,
- Collection: m.Collection,
- FileCount: int(m.FileCount),
- DeleteCount: int(m.DeleteCount),
- DeletedByteCount: m.DeletedByteCount,
- ReadOnly: m.ReadOnly,
- Version: needle.Version(m.Version),
- CompactRevision: m.CompactRevision,
- ModifiedAtSecond: m.ModifiedAtSecond,
+ Id: needle.VolumeId(m.Id),
+ Size: m.Size,
+ Collection: m.Collection,
+ FileCount: int(m.FileCount),
+ DeleteCount: int(m.DeleteCount),
+ DeletedByteCount: m.DeletedByteCount,
+ ReadOnly: m.ReadOnly,
+ Version: needle.Version(m.Version),
+ CompactRevision: m.CompactRevision,
+ ModifiedAtSecond: m.ModifiedAtSecond,
+ RemoteStorageName: m.RemoteStorageName,
+ RemoteStorageKey: m.RemoteStorageKey,
}
- rp, e := NewReplicaPlacementFromByte(byte(m.ReplicaPlacement))
+ rp, e := super_block.NewReplicaPlacementFromByte(byte(m.ReplicaPlacement))
if e != nil {
return vi, e
}
@@ -51,7 +56,7 @@ func NewVolumeInfoFromShort(m *master_pb.VolumeShortInformationMessage) (vi Volu
Collection: m.Collection,
Version: needle.Version(m.Version),
}
- rp, e := NewReplicaPlacementFromByte(byte(m.ReplicaPlacement))
+ rp, e := super_block.NewReplicaPlacementFromByte(byte(m.ReplicaPlacement))
if e != nil {
return vi, e
}
@@ -60,6 +65,10 @@ func NewVolumeInfoFromShort(m *master_pb.VolumeShortInformationMessage) (vi Volu
return vi, nil
}
+func (vi VolumeInfo) IsRemote() bool {
+ return vi.RemoteStorageName != ""
+}
+
func (vi VolumeInfo) String() string {
return fmt.Sprintf("Id:%d, Size:%d, ReplicaPlacement:%s, Collection:%s, Version:%v, FileCount:%d, DeleteCount:%d, DeletedByteCount:%d, ReadOnly:%v",
vi.Id, vi.Size, vi.ReplicaPlacement, vi.Collection, vi.Version, vi.FileCount, vi.DeleteCount, vi.DeletedByteCount, vi.ReadOnly)
@@ -67,18 +76,20 @@ func (vi VolumeInfo) String() string {
func (vi VolumeInfo) ToVolumeInformationMessage() *master_pb.VolumeInformationMessage {
return &master_pb.VolumeInformationMessage{
- Id: uint32(vi.Id),
- Size: uint64(vi.Size),
- Collection: vi.Collection,
- FileCount: uint64(vi.FileCount),
- DeleteCount: uint64(vi.DeleteCount),
- DeletedByteCount: vi.DeletedByteCount,
- ReadOnly: vi.ReadOnly,
- ReplicaPlacement: uint32(vi.ReplicaPlacement.Byte()),
- Version: uint32(vi.Version),
- Ttl: vi.Ttl.ToUint32(),
- CompactRevision: vi.CompactRevision,
- ModifiedAtSecond: vi.ModifiedAtSecond,
+ Id: uint32(vi.Id),
+ Size: uint64(vi.Size),
+ Collection: vi.Collection,
+ FileCount: uint64(vi.FileCount),
+ DeleteCount: uint64(vi.DeleteCount),
+ DeletedByteCount: vi.DeletedByteCount,
+ ReadOnly: vi.ReadOnly,
+ ReplicaPlacement: uint32(vi.ReplicaPlacement.Byte()),
+ Version: uint32(vi.Version),
+ Ttl: vi.Ttl.ToUint32(),
+ CompactRevision: vi.CompactRevision,
+ ModifiedAtSecond: vi.ModifiedAtSecond,
+ RemoteStorageName: vi.RemoteStorageName,
+ RemoteStorageKey: vi.RemoteStorageKey,
}
}
diff --git a/weed/storage/volume_loading.go b/weed/storage/volume_loading.go
index 6f1d8fe40..6b42fc452 100644
--- a/weed/storage/volume_loading.go
+++ b/weed/storage/volume_loading.go
@@ -3,146 +3,148 @@ package storage
import (
"fmt"
"os"
- "time"
- "github.com/chrislusf/seaweedfs/weed/stats"
- "github.com/chrislusf/seaweedfs/weed/storage/backend"
- "github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/syndtr/goleveldb/leveldb/opt"
"github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/stats"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ "github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
+ "github.com/chrislusf/seaweedfs/weed/util"
)
-func loadVolumeWithoutIndex(dirname string, collection string, id needle.VolumeId, needleMapKind NeedleMapType) (v *Volume, e error) {
+func loadVolumeWithoutIndex(dirname string, collection string, id needle.VolumeId, needleMapKind NeedleMapType) (v *Volume, err error) {
v = &Volume{dir: dirname, Collection: collection, Id: id}
- v.SuperBlock = SuperBlock{}
+ v.SuperBlock = super_block.SuperBlock{}
v.needleMapKind = needleMapKind
- e = v.load(false, false, needleMapKind, 0)
+ err = v.load(false, false, needleMapKind, 0)
return
}
-func (v *Volume) load(alsoLoadIndex bool, createDatIfMissing bool, needleMapKind NeedleMapType, preallocate int64) error {
- var e error
+func (v *Volume) load(alsoLoadIndex bool, createDatIfMissing bool, needleMapKind NeedleMapType, preallocate int64) (err error) {
fileName := v.FileName()
alreadyHasSuperBlock := false
- // open dat file
- if exists, canRead, canWrite, modifiedTime, fileSize := checkFile(fileName + ".dat"); exists {
+ hasVolumeInfoFile := v.maybeLoadVolumeInfo() && v.volumeInfo.Version != 0
+
+ if v.HasRemoteFile() {
+ v.noWriteCanDelete = true
+ v.noWriteOrDelete = false
+ glog.V(0).Infof("loading volume %d from remote %v", v.Id, v.volumeInfo.Files)
+ v.LoadRemoteFile()
+ alreadyHasSuperBlock = true
+ } else if exists, canRead, canWrite, modifiedTime, fileSize := util.CheckFile(fileName + ".dat"); exists {
+ // open dat file
if !canRead {
return fmt.Errorf("cannot read Volume Data file %s.dat", fileName)
}
var dataFile *os.File
if canWrite {
- dataFile, e = os.OpenFile(fileName+".dat", os.O_RDWR|os.O_CREATE, 0644)
+ dataFile, err = os.OpenFile(fileName+".dat", os.O_RDWR|os.O_CREATE, 0644)
} else {
glog.V(0).Infoln("opening " + fileName + ".dat in READONLY mode")
- dataFile, e = os.Open(fileName + ".dat")
- v.readOnly = true
+ dataFile, err = os.Open(fileName + ".dat")
+ v.noWriteOrDelete = true
}
v.lastModifiedTsSeconds = uint64(modifiedTime.Unix())
- if fileSize >= _SuperBlockSize {
+ if fileSize >= super_block.SuperBlockSize {
alreadyHasSuperBlock = true
}
v.DataBackend = backend.NewDiskFile(dataFile)
} else {
if createDatIfMissing {
- v.DataBackend, e = createVolumeFile(fileName+".dat", preallocate, v.MemoryMapMaxSizeMb)
+ v.DataBackend, err = createVolumeFile(fileName+".dat", preallocate, v.MemoryMapMaxSizeMb)
} else {
return fmt.Errorf("Volume Data file %s.dat does not exist.", fileName)
}
}
- if e != nil {
- if !os.IsPermission(e) {
- return fmt.Errorf("cannot load Volume Data %s.dat: %v", fileName, e)
+ if err != nil {
+ if !os.IsPermission(err) {
+ return fmt.Errorf("cannot load Volume Data %s.dat: %v", fileName, err)
} else {
- return fmt.Errorf("load data file %s.dat: %v", fileName, e)
+ return fmt.Errorf("load data file %s.dat: %v", fileName, err)
}
}
if alreadyHasSuperBlock {
- e = v.readSuperBlock()
+ err = v.readSuperBlock()
} else {
if !v.SuperBlock.Initialized() {
return fmt.Errorf("volume %s.dat not initialized", fileName)
}
- e = v.maybeWriteSuperBlock()
+ err = v.maybeWriteSuperBlock()
}
- if e == nil && alsoLoadIndex {
+ if err == nil && alsoLoadIndex {
var indexFile *os.File
- if v.readOnly {
- glog.V(1).Infoln("open to read file", fileName+".idx")
- if indexFile, e = os.OpenFile(fileName+".idx", os.O_RDONLY, 0644); e != nil {
- return fmt.Errorf("cannot read Volume Index %s.idx: %v", fileName, e)
+ if v.noWriteOrDelete {
+ glog.V(0).Infoln("open to read file", fileName+".idx")
+ if indexFile, err = os.OpenFile(fileName+".idx", os.O_RDONLY, 0644); err != nil {
+ return fmt.Errorf("cannot read Volume Index %s.idx: %v", fileName, err)
}
} else {
glog.V(1).Infoln("open to write file", fileName+".idx")
- if indexFile, e = os.OpenFile(fileName+".idx", os.O_RDWR|os.O_CREATE, 0644); e != nil {
- return fmt.Errorf("cannot write Volume Index %s.idx: %v", fileName, e)
+ if indexFile, err = os.OpenFile(fileName+".idx", os.O_RDWR|os.O_CREATE, 0644); err != nil {
+ return fmt.Errorf("cannot write Volume Index %s.idx: %v", fileName, err)
}
}
- if v.lastAppendAtNs, e = CheckVolumeDataIntegrity(v, indexFile); e != nil {
- v.readOnly = true
- glog.V(0).Infof("volumeDataIntegrityChecking failed %v", e)
+ if v.lastAppendAtNs, err = CheckVolumeDataIntegrity(v, indexFile); err != nil {
+ v.noWriteOrDelete = true
+ glog.V(0).Infof("volumeDataIntegrityChecking failed %v", err)
}
- switch needleMapKind {
- case NeedleMapInMemory:
- glog.V(0).Infoln("loading index", fileName+".idx", "to memory readonly", v.readOnly)
- if v.nm, e = LoadCompactNeedleMap(indexFile); e != nil {
- glog.V(0).Infof("loading index %s to memory error: %v", fileName+".idx", e)
- }
- case NeedleMapLevelDb:
- glog.V(0).Infoln("loading leveldb", fileName+".ldb")
- opts := &opt.Options{
- BlockCacheCapacity: 2 * 1024 * 1024, // default value is 8MiB
- WriteBuffer: 1 * 1024 * 1024, // default value is 4MiB
- CompactionTableSizeMultiplier: 10, // default value is 1
- }
- if v.nm, e = NewLevelDbNeedleMap(fileName+".ldb", indexFile, opts); e != nil {
- glog.V(0).Infof("loading leveldb %s error: %v", fileName+".ldb", e)
- }
- case NeedleMapLevelDbMedium:
- glog.V(0).Infoln("loading leveldb medium", fileName+".ldb")
- opts := &opt.Options{
- BlockCacheCapacity: 4 * 1024 * 1024, // default value is 8MiB
- WriteBuffer: 2 * 1024 * 1024, // default value is 4MiB
- CompactionTableSizeMultiplier: 10, // default value is 1
- }
- if v.nm, e = NewLevelDbNeedleMap(fileName+".ldb", indexFile, opts); e != nil {
- glog.V(0).Infof("loading leveldb %s error: %v", fileName+".ldb", e)
- }
- case NeedleMapLevelDbLarge:
- glog.V(0).Infoln("loading leveldb large", fileName+".ldb")
- opts := &opt.Options{
- BlockCacheCapacity: 8 * 1024 * 1024, // default value is 8MiB
- WriteBuffer: 4 * 1024 * 1024, // default value is 4MiB
- CompactionTableSizeMultiplier: 10, // default value is 1
+
+ if v.noWriteOrDelete || v.noWriteCanDelete {
+ if v.nm, err = NewSortedFileNeedleMap(fileName, indexFile); err != nil {
+ glog.V(0).Infof("loading sorted db %s error: %v", fileName+".sdx", err)
}
- if v.nm, e = NewLevelDbNeedleMap(fileName+".ldb", indexFile, opts); e != nil {
- glog.V(0).Infof("loading leveldb %s error: %v", fileName+".ldb", e)
+ } else {
+ switch needleMapKind {
+ case NeedleMapInMemory:
+ glog.V(0).Infoln("loading index", fileName+".idx", "to memory")
+ if v.nm, err = LoadCompactNeedleMap(indexFile); err != nil {
+ glog.V(0).Infof("loading index %s to memory error: %v", fileName+".idx", err)
+ }
+ case NeedleMapLevelDb:
+ glog.V(0).Infoln("loading leveldb", fileName+".ldb")
+ opts := &opt.Options{
+ BlockCacheCapacity: 2 * 1024 * 1024, // default value is 8MiB
+ WriteBuffer: 1 * 1024 * 1024, // default value is 4MiB
+ CompactionTableSizeMultiplier: 10, // default value is 1
+ }
+ if v.nm, err = NewLevelDbNeedleMap(fileName+".ldb", indexFile, opts); err != nil {
+ glog.V(0).Infof("loading leveldb %s error: %v", fileName+".ldb", err)
+ }
+ case NeedleMapLevelDbMedium:
+ glog.V(0).Infoln("loading leveldb medium", fileName+".ldb")
+ opts := &opt.Options{
+ BlockCacheCapacity: 4 * 1024 * 1024, // default value is 8MiB
+ WriteBuffer: 2 * 1024 * 1024, // default value is 4MiB
+ CompactionTableSizeMultiplier: 10, // default value is 1
+ }
+ if v.nm, err = NewLevelDbNeedleMap(fileName+".ldb", indexFile, opts); err != nil {
+ glog.V(0).Infof("loading leveldb %s error: %v", fileName+".ldb", err)
+ }
+ case NeedleMapLevelDbLarge:
+ glog.V(0).Infoln("loading leveldb large", fileName+".ldb")
+ opts := &opt.Options{
+ BlockCacheCapacity: 8 * 1024 * 1024, // default value is 8MiB
+ WriteBuffer: 4 * 1024 * 1024, // default value is 4MiB
+ CompactionTableSizeMultiplier: 10, // default value is 1
+ }
+ if v.nm, err = NewLevelDbNeedleMap(fileName+".ldb", indexFile, opts); err != nil {
+ glog.V(0).Infof("loading leveldb %s error: %v", fileName+".ldb", err)
+ }
}
}
}
- stats.VolumeServerVolumeCounter.WithLabelValues(v.Collection, "volume").Inc()
+ if !hasVolumeInfoFile {
+ v.volumeInfo.Version = uint32(v.SuperBlock.Version)
+ v.SaveVolumeInfo()
+ }
- return e
-}
+ stats.VolumeServerVolumeCounter.WithLabelValues(v.Collection, "volume").Inc()
-func checkFile(filename string) (exists, canRead, canWrite bool, modTime time.Time, fileSize int64) {
- exists = true
- fi, err := os.Stat(filename)
- if os.IsNotExist(err) {
- exists = false
- return
- }
- if fi.Mode()&0400 != 0 {
- canRead = true
- }
- if fi.Mode()&0200 != 0 {
- canWrite = true
- }
- modTime = fi.ModTime()
- fileSize = fi.Size()
- return
+ return err
}
diff --git a/weed/storage/volume_read_write.go b/weed/storage/volume_read_write.go
index 242325755..ac6154cef 100644
--- a/weed/storage/volume_read_write.go
+++ b/weed/storage/volume_read_write.go
@@ -11,6 +11,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
. "github.com/chrislusf/seaweedfs/weed/storage/types"
)
@@ -45,22 +46,25 @@ func (v *Volume) Destroy() (err error) {
err = fmt.Errorf("volume %d is compacting", v.Id)
return
}
+ storageName, storageKey := v.RemoteStorageNameKey()
+ if v.HasRemoteFile() && storageName != "" && storageKey != "" {
+ if backendStorage, found := backend.BackendStorages[storageName]; found {
+ backendStorage.DeleteFile(storageKey)
+ }
+ }
v.Close()
os.Remove(v.FileName() + ".dat")
os.Remove(v.FileName() + ".idx")
+ os.Remove(v.FileName() + ".vif")
+ os.Remove(v.FileName() + ".sdx")
os.Remove(v.FileName() + ".cpd")
os.Remove(v.FileName() + ".cpx")
os.RemoveAll(v.FileName() + ".ldb")
- os.RemoveAll(v.FileName() + ".bdb")
return
}
func (v *Volume) writeNeedle(n *needle.Needle) (offset uint64, size uint32, isUnchanged bool, err error) {
- glog.V(4).Infof("writing needle %s", needle.NewFileIdFromNeedle(v.Id, n).String())
- if v.readOnly {
- err = fmt.Errorf("%s is read-only", v.DataBackend.String())
- return
- }
+ // glog.V(4).Infof("writing needle %s", needle.NewFileIdFromNeedle(v.Id, n).String())
v.dataFileAccessLock.Lock()
defer v.dataFileAccessLock.Unlock()
if v.isFileUnchanged(n) {
@@ -110,9 +114,6 @@ func (v *Volume) writeNeedle(n *needle.Needle) (offset uint64, size uint32, isUn
func (v *Volume) deleteNeedle(n *needle.Needle) (uint32, error) {
glog.V(4).Infof("delete needle %s", needle.NewFileIdFromNeedle(v.Id, n).String())
- if v.readOnly {
- return 0, fmt.Errorf("%s is read-only", v.DataBackend.String())
- }
v.dataFileAccessLock.Lock()
defer v.dataFileAccessLock.Unlock()
nv, ok := v.nm.Get(n.Id)
@@ -136,8 +137,8 @@ func (v *Volume) deleteNeedle(n *needle.Needle) (uint32, error) {
// read fills in Needle content by looking up n.Id from NeedleMapper
func (v *Volume) readNeedle(n *needle.Needle) (int, error) {
- v.dataFileAccessLock.Lock()
- defer v.dataFileAccessLock.Unlock()
+ v.dataFileAccessLock.RLock()
+ defer v.dataFileAccessLock.RUnlock()
nv, ok := v.nm.Get(n.Id)
if !ok || nv.Offset.IsZero() {
@@ -171,7 +172,7 @@ func (v *Volume) readNeedle(n *needle.Needle) (int, error) {
}
type VolumeFileScanner interface {
- VisitSuperBlock(SuperBlock) error
+ VisitSuperBlock(super_block.SuperBlock) error
ReadNeedleBody() bool
VisitNeedle(n *needle.Needle, offset int64, needleHeader, needleBody []byte) error
}
@@ -183,8 +184,10 @@ func ScanVolumeFile(dirname string, collection string, id needle.VolumeId,
if v, err = loadVolumeWithoutIndex(dirname, collection, id, needleMapKind); err != nil {
return fmt.Errorf("failed to load volume %d: %v", id, err)
}
- if err = volumeFileScanner.VisitSuperBlock(v.SuperBlock); err != nil {
- return fmt.Errorf("failed to process volume %d super block: %v", id, err)
+ if v.volumeInfo.Version == 0 {
+ if err = volumeFileScanner.VisitSuperBlock(v.SuperBlock); err != nil {
+ return fmt.Errorf("failed to process volume %d super block: %v", id, err)
+ }
}
defer v.Close()
@@ -195,13 +198,13 @@ func ScanVolumeFile(dirname string, collection string, id needle.VolumeId,
return ScanVolumeFileFrom(version, v.DataBackend, offset, volumeFileScanner)
}
-func ScanVolumeFileFrom(version needle.Version, datBackend backend.DataStorageBackend, offset int64, volumeFileScanner VolumeFileScanner) (err error) {
+func ScanVolumeFileFrom(version needle.Version, datBackend backend.BackendStorageFile, offset int64, volumeFileScanner VolumeFileScanner) (err error) {
n, nh, rest, e := needle.ReadNeedleHeader(datBackend, version, offset)
if e != nil {
if e == io.EOF {
return nil
}
- return fmt.Errorf("cannot read %s at offset %d: %v", datBackend.String(), offset, e)
+ return fmt.Errorf("cannot read %s at offset %d: %v", datBackend.Name(), offset, e)
}
for n != nil {
var needleBody []byte
diff --git a/weed/storage/volume_super_block.go b/weed/storage/volume_super_block.go
index bce5af465..5e913e062 100644
--- a/weed/storage/volume_super_block.go
+++ b/weed/storage/volume_super_block.go
@@ -5,92 +5,29 @@ import (
"os"
"github.com/chrislusf/seaweedfs/weed/glog"
- "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/storage/backend"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
- "github.com/chrislusf/seaweedfs/weed/util"
- "github.com/golang/protobuf/proto"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
)
-const (
- _SuperBlockSize = 8
-)
-
-/*
-* Super block currently has 8 bytes allocated for each volume.
-* Byte 0: version, 1 or 2
-* Byte 1: Replica Placement strategy, 000, 001, 002, 010, etc
-* Byte 2 and byte 3: Time to live. See TTL for definition
-* Byte 4 and byte 5: The number of times the volume has been compacted.
-* Rest bytes: Reserved
- */
-type SuperBlock struct {
- version needle.Version
- ReplicaPlacement *ReplicaPlacement
- Ttl *needle.TTL
- CompactionRevision uint16
- Extra *master_pb.SuperBlockExtra
- extraSize uint16
-}
-
-func (s *SuperBlock) BlockSize() int {
- switch s.version {
- case needle.Version2, needle.Version3:
- return _SuperBlockSize + int(s.extraSize)
- }
- return _SuperBlockSize
-}
-
-func (s *SuperBlock) Version() needle.Version {
- return s.version
-}
-func (s *SuperBlock) Bytes() []byte {
- header := make([]byte, _SuperBlockSize)
- header[0] = byte(s.version)
- header[1] = s.ReplicaPlacement.Byte()
- s.Ttl.ToBytes(header[2:4])
- util.Uint16toBytes(header[4:6], s.CompactionRevision)
-
- if s.Extra != nil {
- extraData, err := proto.Marshal(s.Extra)
- if err != nil {
- glog.Fatalf("cannot marshal super block extra %+v: %v", s.Extra, err)
- }
- extraSize := len(extraData)
- if extraSize > 256*256-2 {
- // reserve a couple of bits for future extension
- glog.Fatalf("super block extra size is %d bigger than %d", extraSize, 256*256-2)
- }
- s.extraSize = uint16(extraSize)
- util.Uint16toBytes(header[6:8], s.extraSize)
-
- header = append(header, extraData...)
- }
-
- return header
-}
-
-func (s *SuperBlock) Initialized() bool {
- return s.ReplicaPlacement != nil && s.Ttl != nil
-}
-
func (v *Volume) maybeWriteSuperBlock() error {
datSize, _, e := v.DataBackend.GetStat()
if e != nil {
- glog.V(0).Infof("failed to stat datafile %s: %v", v.DataBackend.String(), e)
+ glog.V(0).Infof("failed to stat datafile %s: %v", v.DataBackend.Name(), e)
return e
}
if datSize == 0 {
- v.SuperBlock.version = needle.CurrentVersion
+ v.SuperBlock.Version = needle.CurrentVersion
_, e = v.DataBackend.WriteAt(v.SuperBlock.Bytes(), 0)
if e != nil && os.IsPermission(e) {
//read-only, but zero length - recreate it!
var dataFile *os.File
- if dataFile, e = os.Create(v.DataBackend.String()); e == nil {
+ if dataFile, e = os.Create(v.DataBackend.Name()); e == nil {
v.DataBackend = backend.NewDiskFile(dataFile)
if _, e = v.DataBackend.WriteAt(v.SuperBlock.Bytes(), 0); e == nil {
- v.readOnly = false
+ v.noWriteOrDelete = false
+ v.noWriteCanDelete = false
}
}
}
@@ -99,38 +36,13 @@ func (v *Volume) maybeWriteSuperBlock() error {
}
func (v *Volume) readSuperBlock() (err error) {
- v.SuperBlock, err = ReadSuperBlock(v.DataBackend)
- return err
-}
-
-// ReadSuperBlock reads from data file and load it into volume's super block
-func ReadSuperBlock(datBackend backend.DataStorageBackend) (superBlock SuperBlock, err error) {
-
- header := make([]byte, _SuperBlockSize)
- if _, e := datBackend.ReadAt(header, 0); e != nil {
- err = fmt.Errorf("cannot read volume %s super block: %v", datBackend.String(), e)
- return
- }
-
- superBlock.version = needle.Version(header[0])
- if superBlock.ReplicaPlacement, err = NewReplicaPlacementFromByte(header[1]); err != nil {
- err = fmt.Errorf("cannot read replica type: %s", err.Error())
- return
- }
- superBlock.Ttl = needle.LoadTTLFromBytes(header[2:4])
- superBlock.CompactionRevision = util.BytesToUint16(header[4:6])
- superBlock.extraSize = util.BytesToUint16(header[6:8])
-
- if superBlock.extraSize > 0 {
- // read more
- extraData := make([]byte, int(superBlock.extraSize))
- superBlock.Extra = &master_pb.SuperBlockExtra{}
- err = proto.Unmarshal(extraData, superBlock.Extra)
- if err != nil {
- err = fmt.Errorf("cannot read volume %s super block extra: %v", datBackend.String(), err)
- return
+ v.SuperBlock, err = super_block.ReadSuperBlock(v.DataBackend)
+ if v.volumeInfo != nil && v.volumeInfo.Replication != "" {
+ if replication, err := super_block.NewReplicaPlacementFromString(v.volumeInfo.Replication); err != nil {
+ return fmt.Errorf("Error parse volume %d replication %s : %v", v.Id, v.volumeInfo.Replication, err)
+ } else {
+ v.SuperBlock.ReplicaPlacement = replication
}
}
-
- return
+ return err
}
diff --git a/weed/storage/volume_tier.go b/weed/storage/volume_tier.go
new file mode 100644
index 000000000..fd7b08654
--- /dev/null
+++ b/weed/storage/volume_tier.go
@@ -0,0 +1,50 @@
+package storage
+
+import (
+ "github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb"
+ "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
+ "github.com/chrislusf/seaweedfs/weed/storage/backend"
+ _ "github.com/chrislusf/seaweedfs/weed/storage/backend/s3_backend"
+)
+
+func (v *Volume) GetVolumeInfo() *volume_server_pb.VolumeInfo {
+ return v.volumeInfo
+}
+
+func (v *Volume) maybeLoadVolumeInfo() (found bool) {
+
+ v.volumeInfo, v.hasRemoteFile, _ = pb.MaybeLoadVolumeInfo(v.FileName() + ".vif")
+
+ if v.hasRemoteFile {
+ glog.V(0).Infof("volume %d is tiered to %s as %s and read only", v.Id,
+ v.volumeInfo.Files[0].BackendName(), v.volumeInfo.Files[0].Key)
+ }
+
+ return
+
+}
+
+func (v *Volume) HasRemoteFile() bool {
+ return v.hasRemoteFile
+}
+
+func (v *Volume) LoadRemoteFile() error {
+ tierFile := v.volumeInfo.GetFiles()[0]
+ backendStorage := backend.BackendStorages[tierFile.BackendName()]
+
+ if v.DataBackend != nil {
+ v.DataBackend.Close()
+ }
+
+ v.DataBackend = backendStorage.NewStorageFile(tierFile.Key, v.volumeInfo)
+ return nil
+}
+
+func (v *Volume) SaveVolumeInfo() error {
+
+ tierFileName := v.FileName() + ".vif"
+
+ return pb.SaveVolumeInfo(tierFileName, v.volumeInfo)
+
+}
diff --git a/weed/storage/volume_vacuum.go b/weed/storage/volume_vacuum.go
index e90746b54..5d0d63877 100644
--- a/weed/storage/volume_vacuum.go
+++ b/weed/storage/volume_vacuum.go
@@ -3,6 +3,7 @@ package storage
import (
"fmt"
"os"
+ "runtime"
"time"
"github.com/chrislusf/seaweedfs/weed/glog"
@@ -11,6 +12,7 @@ import (
idx2 "github.com/chrislusf/seaweedfs/weed/storage/idx"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/chrislusf/seaweedfs/weed/storage/needle_map"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
. "github.com/chrislusf/seaweedfs/weed/storage/types"
"github.com/chrislusf/seaweedfs/weed/util"
)
@@ -19,101 +21,124 @@ func (v *Volume) garbageLevel() float64 {
if v.ContentSize() == 0 {
return 0
}
- return float64(v.DeletedSize()) / float64(v.ContentSize())
+ deletedSize := v.DeletedSize()
+ fileSize := v.ContentSize()
+ if v.DeletedCount() > 0 && v.DeletedSize() == 0 {
+ // this happens for .sdx converted back to normal .idx
+ // where deleted entry size is missing
+ datFileSize, _, _ := v.FileStat()
+ deletedSize = datFileSize - fileSize - super_block.SuperBlockSize
+ fileSize = datFileSize
+ }
+ return float64(deletedSize) / float64(fileSize)
}
+// compact a volume based on deletions in .dat files
func (v *Volume) Compact(preallocate int64, compactionBytePerSecond int64) error {
- if v.MemoryMapMaxSizeMb == 0 { //it makes no sense to compact in memory
- glog.V(3).Infof("Compacting volume %d ...", v.Id)
- //no need to lock for copy on write
- //v.accessLock.Lock()
- //defer v.accessLock.Unlock()
- //glog.V(3).Infof("Got Compaction lock...")
- v.isCompacting = true
- defer func() {
- v.isCompacting = false
- }()
-
- filePath := v.FileName()
- v.lastCompactIndexOffset = v.IndexFileSize()
- v.lastCompactRevision = v.SuperBlock.CompactionRevision
- glog.V(3).Infof("creating copies for volume %d ,last offset %d...", v.Id, v.lastCompactIndexOffset)
- return v.copyDataAndGenerateIndexFile(filePath+".cpd", filePath+".cpx", preallocate, compactionBytePerSecond)
- } else {
+ if v.MemoryMapMaxSizeMb != 0 { //it makes no sense to compact in memory
return nil
}
+ glog.V(3).Infof("Compacting volume %d ...", v.Id)
+ //no need to lock for copy on write
+ //v.accessLock.Lock()
+ //defer v.accessLock.Unlock()
+ //glog.V(3).Infof("Got Compaction lock...")
+ v.isCompacting = true
+ defer func() {
+ v.isCompacting = false
+ }()
+
+ filePath := v.FileName()
+ v.lastCompactIndexOffset = v.IndexFileSize()
+ v.lastCompactRevision = v.SuperBlock.CompactionRevision
+ glog.V(3).Infof("creating copies for volume %d ,last offset %d...", v.Id, v.lastCompactIndexOffset)
+ return v.copyDataAndGenerateIndexFile(filePath+".cpd", filePath+".cpx", preallocate, compactionBytePerSecond)
}
-func (v *Volume) Compact2() error {
-
- if v.MemoryMapMaxSizeMb == 0 { //it makes no sense to compact in memory
- glog.V(3).Infof("Compact2 volume %d ...", v.Id)
+// compact a volume based on deletions in .idx files
+func (v *Volume) Compact2(preallocate int64) error {
- v.isCompacting = true
- defer func() {
- v.isCompacting = false
- }()
-
- filePath := v.FileName()
- glog.V(3).Infof("creating copies for volume %d ...", v.Id)
- return v.copyDataBasedOnIndexFile(filePath+".cpd", filePath+".cpx")
- } else {
+ if v.MemoryMapMaxSizeMb != 0 { //it makes no sense to compact in memory
return nil
}
+ glog.V(3).Infof("Compact2 volume %d ...", v.Id)
+
+ v.isCompacting = true
+ defer func() {
+ v.isCompacting = false
+ }()
+
+ filePath := v.FileName()
+ v.lastCompactIndexOffset = v.IndexFileSize()
+ v.lastCompactRevision = v.SuperBlock.CompactionRevision
+ glog.V(3).Infof("creating copies for volume %d ...", v.Id)
+ return copyDataBasedOnIndexFile(filePath+".dat", filePath+".idx", filePath+".cpd", filePath+".cpx", v.SuperBlock, v.Version(), preallocate)
}
func (v *Volume) CommitCompact() error {
- if v.MemoryMapMaxSizeMb == 0 { //it makes no sense to compact in memory
- glog.V(0).Infof("Committing volume %d vacuuming...", v.Id)
+ if v.MemoryMapMaxSizeMb != 0 { //it makes no sense to compact in memory
+ return nil
+ }
+ glog.V(0).Infof("Committing volume %d vacuuming...", v.Id)
- v.isCompacting = true
- defer func() {
- v.isCompacting = false
- }()
+ v.isCompacting = true
+ defer func() {
+ v.isCompacting = false
+ }()
- v.dataFileAccessLock.Lock()
- defer v.dataFileAccessLock.Unlock()
+ v.dataFileAccessLock.Lock()
+ defer v.dataFileAccessLock.Unlock()
- glog.V(3).Infof("Got volume %d committing lock...", v.Id)
- v.nm.Close()
+ glog.V(3).Infof("Got volume %d committing lock...", v.Id)
+ v.nm.Close()
+ if v.DataBackend != nil {
if err := v.DataBackend.Close(); err != nil {
glog.V(0).Infof("fail to close volume %d", v.Id)
}
- v.DataBackend = nil
- stats.VolumeServerVolumeCounter.WithLabelValues(v.Collection, "volume").Dec()
-
- var e error
- if e = v.makeupDiff(v.FileName()+".cpd", v.FileName()+".cpx", v.FileName()+".dat", v.FileName()+".idx"); e != nil {
- glog.V(0).Infof("makeupDiff in CommitCompact volume %d failed %v", v.Id, e)
- e = os.Remove(v.FileName() + ".cpd")
+ }
+ v.DataBackend = nil
+ stats.VolumeServerVolumeCounter.WithLabelValues(v.Collection, "volume").Dec()
+
+ var e error
+ if e = v.makeupDiff(v.FileName()+".cpd", v.FileName()+".cpx", v.FileName()+".dat", v.FileName()+".idx"); e != nil {
+ glog.V(0).Infof("makeupDiff in CommitCompact volume %d failed %v", v.Id, e)
+ e = os.Remove(v.FileName() + ".cpd")
+ if e != nil {
+ return e
+ }
+ e = os.Remove(v.FileName() + ".cpx")
+ if e != nil {
+ return e
+ }
+ } else {
+ if runtime.GOOS == "windows" {
+ e = os.RemoveAll(v.FileName() + ".dat")
if e != nil {
return e
}
- e = os.Remove(v.FileName() + ".cpx")
+ e = os.RemoveAll(v.FileName() + ".idx")
if e != nil {
return e
}
- } else {
- var e error
- if e = os.Rename(v.FileName()+".cpd", v.FileName()+".dat"); e != nil {
- return fmt.Errorf("rename %s: %v", v.FileName()+".cpd", e)
- }
- if e = os.Rename(v.FileName()+".cpx", v.FileName()+".idx"); e != nil {
- return fmt.Errorf("rename %s: %v", v.FileName()+".cpx", e)
- }
}
+ var e error
+ if e = os.Rename(v.FileName()+".cpd", v.FileName()+".dat"); e != nil {
+ return fmt.Errorf("rename %s: %v", v.FileName()+".cpd", e)
+ }
+ if e = os.Rename(v.FileName()+".cpx", v.FileName()+".idx"); e != nil {
+ return fmt.Errorf("rename %s: %v", v.FileName()+".cpx", e)
+ }
+ }
- //glog.V(3).Infof("Pretending to be vacuuming...")
- //time.Sleep(20 * time.Second)
+ //glog.V(3).Infof("Pretending to be vacuuming...")
+ //time.Sleep(20 * time.Second)
- os.RemoveAll(v.FileName() + ".ldb")
- os.RemoveAll(v.FileName() + ".bdb")
+ os.RemoveAll(v.FileName() + ".ldb")
- glog.V(3).Infof("Loading volume %d commit file...", v.Id)
- if e = v.load(true, false, v.needleMapKind, 0); e != nil {
- return e
- }
+ glog.V(3).Infof("Loading volume %d commit file...", v.Id)
+ if e = v.load(true, false, v.needleMapKind, 0); e != nil {
+ return e
}
return nil
}
@@ -132,14 +157,15 @@ func (v *Volume) cleanupCompact() error {
return nil
}
-func fetchCompactRevisionFromDatFile(datBackend backend.DataStorageBackend) (compactRevision uint16, err error) {
- superBlock, err := ReadSuperBlock(datBackend)
+func fetchCompactRevisionFromDatFile(datBackend backend.BackendStorageFile) (compactRevision uint16, err error) {
+ superBlock, err := super_block.ReadSuperBlock(datBackend)
if err != nil {
return 0, err
}
return superBlock.CompactionRevision, nil
}
+// if old .dat and .idx files are updated, this func tries to apply the same changes to new files accordingly
func (v *Volume) makeupDiff(newDatFileName, newIdxFileName, oldDatFileName, oldIdxFileName string) (err error) {
var indexSize int64
@@ -150,6 +176,7 @@ func (v *Volume) makeupDiff(newDatFileName, newIdxFileName, oldDatFileName, oldI
oldDatBackend := backend.NewDiskFile(oldDatFile)
defer oldDatBackend.Close()
+ // skip if the old .idx file has not changed
if indexSize, err = verifyIndexFileIntegrity(oldIdxFile); err != nil {
return fmt.Errorf("verifyIndexFileIntegrity %s failed: %v", oldIdxFileName, err)
}
@@ -157,6 +184,7 @@ func (v *Volume) makeupDiff(newDatFileName, newIdxFileName, oldDatFileName, oldI
return nil
}
+ // fail if the old .dat file has changed to a new revision
oldDatCompactRevision, err := fetchCompactRevisionFromDatFile(oldDatBackend)
if err != nil {
return fmt.Errorf("fetchCompactRevisionFromDatFile src %s failed: %v", oldDatFile.Name(), err)
@@ -270,15 +298,15 @@ func (v *Volume) makeupDiff(newDatFileName, newIdxFileName, oldDatFileName, oldI
type VolumeFileScanner4Vacuum struct {
version needle.Version
v *Volume
- dstBackend backend.DataStorageBackend
- nm *NeedleMap
+ dstBackend backend.BackendStorageFile
+ nm *needle_map.MemDb
newOffset int64
now uint64
writeThrottler *util.WriteThrottler
}
-func (scanner *VolumeFileScanner4Vacuum) VisitSuperBlock(superBlock SuperBlock) error {
- scanner.version = superBlock.Version()
+func (scanner *VolumeFileScanner4Vacuum) VisitSuperBlock(superBlock super_block.SuperBlock) error {
+ scanner.version = superBlock.Version
superBlock.CompactionRevision++
_, err := scanner.dstBackend.WriteAt(superBlock.Bytes(), 0)
scanner.newOffset = int64(superBlock.BlockSize())
@@ -296,7 +324,7 @@ func (scanner *VolumeFileScanner4Vacuum) VisitNeedle(n *needle.Needle, offset in
nv, ok := scanner.v.nm.Get(n.Id)
glog.V(4).Infoln("needle expected offset ", offset, "ok", ok, "nv", nv)
if ok && nv.Offset.ToAcutalOffset() == offset && nv.Size > 0 && nv.Size != TombstoneFileSize {
- if err := scanner.nm.Put(n.Id, ToOffset(scanner.newOffset), n.Size); err != nil {
+ if err := scanner.nm.Set(n.Id, ToOffset(scanner.newOffset), n.Size); err != nil {
return fmt.Errorf("cannot put needle: %s", err)
}
if _, _, _, err := n.Append(scanner.dstBackend, scanner.v.Version()); err != nil {
@@ -312,90 +340,92 @@ func (scanner *VolumeFileScanner4Vacuum) VisitNeedle(n *needle.Needle, offset in
func (v *Volume) copyDataAndGenerateIndexFile(dstName, idxName string, preallocate int64, compactionBytePerSecond int64) (err error) {
var (
- dst backend.DataStorageBackend
- idx *os.File
+ dst backend.BackendStorageFile
)
if dst, err = createVolumeFile(dstName, preallocate, 0); err != nil {
return
}
defer dst.Close()
- if idx, err = os.OpenFile(idxName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644); err != nil {
- return
- }
- defer idx.Close()
+ nm := needle_map.NewMemDb()
+ defer nm.Close()
scanner := &VolumeFileScanner4Vacuum{
v: v,
now: uint64(time.Now().Unix()),
- nm: NewBtreeNeedleMap(idx),
+ nm: nm,
dstBackend: dst,
writeThrottler: util.NewWriteThrottler(compactionBytePerSecond),
}
err = ScanVolumeFile(v.dir, v.Collection, v.Id, v.needleMapKind, scanner)
+ if err != nil {
+ return nil
+ }
+
+ err = nm.SaveToIdx(idxName)
return
}
-func (v *Volume) copyDataBasedOnIndexFile(dstName, idxName string) (err error) {
+func copyDataBasedOnIndexFile(srcDatName, srcIdxName, dstDatName, datIdxName string, sb super_block.SuperBlock, version needle.Version, preallocate int64) (err error) {
var (
- dst, idx, oldIndexFile *os.File
+ srcDatBackend, dstDatBackend backend.BackendStorageFile
+ dataFile *os.File
)
- if dst, err = os.OpenFile(dstName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644); err != nil {
+ if dstDatBackend, err = createVolumeFile(dstDatName, preallocate, 0); err != nil {
return
}
- dstDatBackend := backend.NewDiskFile(dst)
defer dstDatBackend.Close()
- if idx, err = os.OpenFile(idxName, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0644); err != nil {
+ oldNm := needle_map.NewMemDb()
+ defer oldNm.Close()
+ newNm := needle_map.NewMemDb()
+ defer newNm.Close()
+ if err = oldNm.LoadFromIdx(srcIdxName); err != nil {
return
}
- defer idx.Close()
-
- if oldIndexFile, err = os.OpenFile(v.FileName()+".idx", os.O_RDONLY, 0644); err != nil {
- return
+ if dataFile, err = os.Open(srcDatName); err != nil {
+ return err
}
- defer oldIndexFile.Close()
+ srcDatBackend = backend.NewDiskFile(dataFile)
+ defer srcDatBackend.Close()
- nm := NewBtreeNeedleMap(idx)
now := uint64(time.Now().Unix())
- v.SuperBlock.CompactionRevision++
- dst.Write(v.SuperBlock.Bytes())
- newOffset := int64(v.SuperBlock.BlockSize())
+ sb.CompactionRevision++
+ dstDatBackend.WriteAt(sb.Bytes(), 0)
+ newOffset := int64(sb.BlockSize())
- idx2.WalkIndexFile(oldIndexFile, func(key NeedleId, offset Offset, size uint32) error {
- if offset.IsZero() || size == TombstoneFileSize {
- return nil
- }
+ oldNm.AscendingVisit(func(value needle_map.NeedleValue) error {
+
+ offset, size := value.Offset, value.Size
- nv, ok := v.nm.Get(key)
- if !ok {
+ if offset.IsZero() || size == TombstoneFileSize {
return nil
}
n := new(needle.Needle)
- err := n.ReadData(v.DataBackend, offset.ToAcutalOffset(), size, v.Version())
+ err := n.ReadData(srcDatBackend, offset.ToAcutalOffset(), size, version)
if err != nil {
return nil
}
- if n.HasTtl() && now >= n.LastModified+uint64(v.Ttl.Minutes()*60) {
+ if n.HasTtl() && now >= n.LastModified+uint64(sb.Ttl.Minutes()*60) {
return nil
}
- glog.V(4).Infoln("needle expected offset ", offset, "ok", ok, "nv", nv)
- if nv.Offset == offset && nv.Size > 0 {
- if err = nm.Put(n.Id, ToOffset(newOffset), n.Size); err != nil {
- return fmt.Errorf("cannot put needle: %s", err)
- }
- if _, _, _, err = n.Append(dstDatBackend, v.Version()); err != nil {
- return fmt.Errorf("cannot append needle: %s", err)
- }
- newOffset += n.DiskSize(v.Version())
- glog.V(3).Infoln("saving key", n.Id, "volume offset", offset, "=>", newOffset, "data_size", n.Size)
+ if err = newNm.Set(n.Id, ToOffset(newOffset), n.Size); err != nil {
+ return fmt.Errorf("cannot put needle: %s", err)
+ }
+ if _, _, _, err = n.Append(dstDatBackend, sb.Version); err != nil {
+ return fmt.Errorf("cannot append needle: %s", err)
}
+ newOffset += n.DiskSize(version)
+ glog.V(4).Infoln("saving key", n.Id, "volume offset", offset, "=>", newOffset, "data_size", n.Size)
+
return nil
})
+ newNm.SaveToIdx(datIdxName)
+
return
}
diff --git a/weed/storage/volume_vacuum_test.go b/weed/storage/volume_vacuum_test.go
index ba1e59f2c..95f43d6ec 100644
--- a/weed/storage/volume_vacuum_test.go
+++ b/weed/storage/volume_vacuum_test.go
@@ -8,6 +8,7 @@ import (
"time"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"github.com/chrislusf/seaweedfs/weed/storage/types"
)
@@ -46,7 +47,7 @@ func TestMakeDiff(t *testing.T) {
v := new(Volume)
//lastCompactIndexOffset value is the index file size before step 4
v.lastCompactIndexOffset = 96
- v.SuperBlock.version = 0x2
+ v.SuperBlock.Version = 0x2
/*
err := v.makeupDiff(
"/yourpath/1.cpd",
@@ -68,7 +69,7 @@ func TestCompaction(t *testing.T) {
}
defer os.RemoveAll(dir) // clean up
- v, err := NewVolume(dir, "", 1, NeedleMapInMemory, &ReplicaPlacement{}, &needle.TTL{}, 0, 0)
+ v, err := NewVolume(dir, "", 1, NeedleMapInMemory, &super_block.ReplicaPlacement{}, &needle.TTL{}, 0, 0)
if err != nil {
t.Fatalf("volume creation: %v", err)
}
@@ -83,7 +84,7 @@ func TestCompaction(t *testing.T) {
}
startTime := time.Now()
- v.Compact(0, 1024*1024)
+ v.Compact2(0)
speed := float64(v.ContentSize()) / time.Now().Sub(startTime).Seconds()
t.Logf("compaction speed: %.2f bytes/s", speed)
diff --git a/weed/topology/collection.go b/weed/topology/collection.go
index f6b728ec9..7a611d904 100644
--- a/weed/topology/collection.go
+++ b/weed/topology/collection.go
@@ -3,8 +3,8 @@ package topology
import (
"fmt"
- "github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"github.com/chrislusf/seaweedfs/weed/util"
)
@@ -24,7 +24,7 @@ func (c *Collection) String() string {
return fmt.Sprintf("Name:%s, volumeSizeLimit:%d, storageType2VolumeLayout:%v", c.Name, c.volumeSizeLimit, c.storageType2VolumeLayout)
}
-func (c *Collection) GetOrCreateVolumeLayout(rp *storage.ReplicaPlacement, ttl *needle.TTL) *VolumeLayout {
+func (c *Collection) GetOrCreateVolumeLayout(rp *super_block.ReplicaPlacement, ttl *needle.TTL) *VolumeLayout {
keyString := rp.String()
if ttl != nil {
keyString += ttl.String()
diff --git a/weed/topology/data_center.go b/weed/topology/data_center.go
index 640cb1937..dc3accb71 100644
--- a/weed/topology/data_center.go
+++ b/weed/topology/data_center.go
@@ -48,6 +48,7 @@ func (dc *DataCenter) ToDataCenterInfo() *master_pb.DataCenterInfo {
MaxVolumeCount: uint64(dc.GetMaxVolumeCount()),
FreeVolumeCount: uint64(dc.FreeSpace()),
ActiveVolumeCount: uint64(dc.GetActiveVolumeCount()),
+ RemoteVolumeCount: uint64(dc.GetRemoteVolumeCount()),
}
for _, c := range dc.Children() {
rack := c.(*Rack)
diff --git a/weed/topology/data_node.go b/weed/topology/data_node.go
index 3e72ccdbf..617341e54 100644
--- a/weed/topology/data_node.go
+++ b/weed/topology/data_node.go
@@ -2,14 +2,13 @@ package topology
import (
"fmt"
+ "strconv"
"sync"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
- "strconv"
-
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/storage"
)
@@ -44,15 +43,26 @@ func (dn *DataNode) String() string {
func (dn *DataNode) AddOrUpdateVolume(v storage.VolumeInfo) (isNew bool) {
dn.Lock()
defer dn.Unlock()
- if _, ok := dn.volumes[v.Id]; !ok {
+ if oldV, ok := dn.volumes[v.Id]; !ok {
dn.volumes[v.Id] = v
dn.UpAdjustVolumeCountDelta(1)
+ if v.IsRemote() {
+ dn.UpAdjustRemoteVolumeCountDelta(1)
+ }
if !v.ReadOnly {
dn.UpAdjustActiveVolumeCountDelta(1)
}
dn.UpAdjustMaxVolumeId(v.Id)
isNew = true
} else {
+ if oldV.IsRemote() != v.IsRemote() {
+ if v.IsRemote() {
+ dn.UpAdjustRemoteVolumeCountDelta(1)
+ }
+ if oldV.IsRemote() {
+ dn.UpAdjustRemoteVolumeCountDelta(-1)
+ }
+ }
dn.volumes[v.Id] = v
}
return
@@ -70,7 +80,12 @@ func (dn *DataNode) UpdateVolumes(actualVolumes []storage.VolumeInfo) (newVolume
delete(dn.volumes, vid)
deletedVolumes = append(deletedVolumes, v)
dn.UpAdjustVolumeCountDelta(-1)
- dn.UpAdjustActiveVolumeCountDelta(-1)
+ if v.IsRemote() {
+ dn.UpAdjustRemoteVolumeCountDelta(-1)
+ }
+ if !v.ReadOnly {
+ dn.UpAdjustActiveVolumeCountDelta(-1)
+ }
}
}
dn.Unlock()
@@ -88,7 +103,12 @@ func (dn *DataNode) DeltaUpdateVolumes(newlVolumes, deletedVolumes []storage.Vol
for _, v := range deletedVolumes {
delete(dn.volumes, v.Id)
dn.UpAdjustVolumeCountDelta(-1)
- dn.UpAdjustActiveVolumeCountDelta(-1)
+ if v.IsRemote() {
+ dn.UpAdjustRemoteVolumeCountDelta(-1)
+ }
+ if !v.ReadOnly {
+ dn.UpAdjustActiveVolumeCountDelta(-1)
+ }
}
dn.Unlock()
for _, v := range newlVolumes {
@@ -160,6 +180,7 @@ func (dn *DataNode) ToDataNodeInfo() *master_pb.DataNodeInfo {
MaxVolumeCount: uint64(dn.GetMaxVolumeCount()),
FreeVolumeCount: uint64(dn.FreeSpace()),
ActiveVolumeCount: uint64(dn.GetActiveVolumeCount()),
+ RemoteVolumeCount: uint64(dn.GetRemoteVolumeCount()),
}
for _, v := range dn.GetVolumes() {
m.VolumeInfos = append(m.VolumeInfos, v.ToVolumeInformationMessage())
diff --git a/weed/topology/node.go b/weed/topology/node.go
index b2808f589..ceeb96d60 100644
--- a/weed/topology/node.go
+++ b/weed/topology/node.go
@@ -20,6 +20,7 @@ type Node interface {
ReserveOneVolume(r int64) (*DataNode, error)
UpAdjustMaxVolumeCountDelta(maxVolumeCountDelta int64)
UpAdjustVolumeCountDelta(volumeCountDelta int64)
+ UpAdjustRemoteVolumeCountDelta(remoteVolumeCountDelta int64)
UpAdjustEcShardCountDelta(ecShardCountDelta int64)
UpAdjustActiveVolumeCountDelta(activeVolumeCountDelta int64)
UpAdjustMaxVolumeId(vid needle.VolumeId)
@@ -27,6 +28,7 @@ type Node interface {
GetVolumeCount() int64
GetEcShardCount() int64
GetActiveVolumeCount() int64
+ GetRemoteVolumeCount() int64
GetMaxVolumeCount() int64
GetMaxVolumeId() needle.VolumeId
SetParent(Node)
@@ -44,6 +46,7 @@ type Node interface {
}
type NodeImpl struct {
volumeCount int64
+ remoteVolumeCount int64
activeVolumeCount int64
ecShardCount int64
maxVolumeCount int64
@@ -59,56 +62,64 @@ type NodeImpl struct {
}
// the first node must satisfy filterFirstNodeFn(), the rest nodes must have one free slot
-func (n *NodeImpl) RandomlyPickNodes(numberOfNodes int, filterFirstNodeFn func(dn Node) error) (firstNode Node, restNodes []Node, err error) {
- candidates := make([]Node, 0, len(n.children))
+func (n *NodeImpl) PickNodesByWeight(numberOfNodes int, filterFirstNodeFn func(dn Node) error) (firstNode Node, restNodes []Node, err error) {
+ var totalWeights int64
var errs []string
n.RLock()
+ candidates := make([]Node, 0, len(n.children))
+ candidatesWeights := make([]int64, 0, len(n.children))
+ //pick nodes which has enough free volumes as candidates, and use free volumes number as node weight.
for _, node := range n.children {
- if err := filterFirstNodeFn(node); err == nil {
- candidates = append(candidates, node)
- } else {
- errs = append(errs, string(node.Id())+":"+err.Error())
+ if node.FreeSpace() <= 0 {
+ continue
}
+ totalWeights += node.FreeSpace()
+ candidates = append(candidates, node)
+ candidatesWeights = append(candidatesWeights, node.FreeSpace())
}
n.RUnlock()
- if len(candidates) == 0 {
- return nil, nil, errors.New("No matching data node found! \n" + strings.Join(errs, "\n"))
+ if len(candidates) < numberOfNodes {
+ glog.V(2).Infoln(n.Id(), "failed to pick", numberOfNodes, "from ", len(candidates), "node candidates")
+ return nil, nil, errors.New("No enough data node found!")
}
- firstNode = candidates[rand.Intn(len(candidates))]
- glog.V(2).Infoln(n.Id(), "picked main node:", firstNode.Id())
- restNodes = make([]Node, numberOfNodes-1)
- candidates = candidates[:0]
- n.RLock()
- for _, node := range n.children {
- if node.Id() == firstNode.Id() {
- continue
- }
- if node.FreeSpace() <= 0 {
- continue
+ //pick nodes randomly by weights, the node picked earlier has higher final weights
+ sortedCandidates := make([]Node, 0, len(candidates))
+ for i := 0; i < len(candidates); i++ {
+ weightsInterval := rand.Int63n(totalWeights)
+ lastWeights := int64(0)
+ for k, weights := range candidatesWeights {
+ if (weightsInterval >= lastWeights) && (weightsInterval < lastWeights+weights) {
+ sortedCandidates = append(sortedCandidates, candidates[k])
+ candidatesWeights[k] = 0
+ totalWeights -= weights
+ break
+ }
+ lastWeights += weights
}
- glog.V(2).Infoln("select rest node candidate:", node.Id())
- candidates = append(candidates, node)
}
- n.RUnlock()
- glog.V(2).Infoln(n.Id(), "picking", numberOfNodes-1, "from rest", len(candidates), "node candidates")
- ret := len(restNodes) == 0
- for k, node := range candidates {
- if k < len(restNodes) {
- restNodes[k] = node
- if k == len(restNodes)-1 {
- ret = true
+
+ restNodes = make([]Node, 0, numberOfNodes-1)
+ ret := false
+ n.RLock()
+ for k, node := range sortedCandidates {
+ if err := filterFirstNodeFn(node); err == nil {
+ firstNode = node
+ if k >= numberOfNodes-1 {
+ restNodes = sortedCandidates[:numberOfNodes-1]
+ } else {
+ restNodes = append(restNodes, sortedCandidates[:k]...)
+ restNodes = append(restNodes, sortedCandidates[k+1:numberOfNodes]...)
}
+ ret = true
+ break
} else {
- r := rand.Intn(k + 1)
- if r < len(restNodes) {
- restNodes[r] = node
- }
+ errs = append(errs, string(node.Id())+":"+err.Error())
}
}
+ n.RUnlock()
if !ret {
- glog.V(2).Infoln(n.Id(), "failed to pick", numberOfNodes-1, "from rest", len(candidates), "node candidates")
- err = errors.New("No enough data node found!")
+ return nil, nil, errors.New("No matching data node found! \n" + strings.Join(errs, "\n"))
}
return
}
@@ -132,10 +143,11 @@ func (n *NodeImpl) Id() NodeId {
return n.id
}
func (n *NodeImpl) FreeSpace() int64 {
+ freeVolumeSlotCount := n.maxVolumeCount + n.remoteVolumeCount - n.volumeCount
if n.ecShardCount > 0 {
- return n.maxVolumeCount - n.volumeCount - n.ecShardCount/erasure_coding.DataShardsCount - 1
+ freeVolumeSlotCount = freeVolumeSlotCount - n.ecShardCount/erasure_coding.DataShardsCount - 1
}
- return n.maxVolumeCount - n.volumeCount
+ return freeVolumeSlotCount
}
func (n *NodeImpl) SetParent(node Node) {
n.parent = node
@@ -191,6 +203,12 @@ func (n *NodeImpl) UpAdjustVolumeCountDelta(volumeCountDelta int64) { //can be n
n.parent.UpAdjustVolumeCountDelta(volumeCountDelta)
}
}
+func (n *NodeImpl) UpAdjustRemoteVolumeCountDelta(remoteVolumeCountDelta int64) { //can be negative
+ atomic.AddInt64(&n.remoteVolumeCount, remoteVolumeCountDelta)
+ if n.parent != nil {
+ n.parent.UpAdjustRemoteVolumeCountDelta(remoteVolumeCountDelta)
+ }
+}
func (n *NodeImpl) UpAdjustEcShardCountDelta(ecShardCountDelta int64) { //can be negative
atomic.AddInt64(&n.ecShardCount, ecShardCountDelta)
if n.parent != nil {
@@ -220,6 +238,9 @@ func (n *NodeImpl) GetVolumeCount() int64 {
func (n *NodeImpl) GetEcShardCount() int64 {
return n.ecShardCount
}
+func (n *NodeImpl) GetRemoteVolumeCount() int64 {
+ return n.remoteVolumeCount
+}
func (n *NodeImpl) GetActiveVolumeCount() int64 {
return n.activeVolumeCount
}
@@ -235,6 +256,7 @@ func (n *NodeImpl) LinkChildNode(node Node) {
n.UpAdjustMaxVolumeCountDelta(node.GetMaxVolumeCount())
n.UpAdjustMaxVolumeId(node.GetMaxVolumeId())
n.UpAdjustVolumeCountDelta(node.GetVolumeCount())
+ n.UpAdjustRemoteVolumeCountDelta(node.GetRemoteVolumeCount())
n.UpAdjustEcShardCountDelta(node.GetEcShardCount())
n.UpAdjustActiveVolumeCountDelta(node.GetActiveVolumeCount())
node.SetParent(n)
@@ -250,6 +272,7 @@ func (n *NodeImpl) UnlinkChildNode(nodeId NodeId) {
node.SetParent(nil)
delete(n.children, node.Id())
n.UpAdjustVolumeCountDelta(-node.GetVolumeCount())
+ n.UpAdjustRemoteVolumeCountDelta(-node.GetRemoteVolumeCount())
n.UpAdjustEcShardCountDelta(-node.GetEcShardCount())
n.UpAdjustActiveVolumeCountDelta(-node.GetActiveVolumeCount())
n.UpAdjustMaxVolumeCountDelta(-node.GetMaxVolumeCount())
diff --git a/weed/topology/rack.go b/weed/topology/rack.go
index 932c1a804..1921c0c05 100644
--- a/weed/topology/rack.go
+++ b/weed/topology/rack.go
@@ -67,6 +67,7 @@ func (r *Rack) ToRackInfo() *master_pb.RackInfo {
MaxVolumeCount: uint64(r.GetMaxVolumeCount()),
FreeVolumeCount: uint64(r.FreeSpace()),
ActiveVolumeCount: uint64(r.GetActiveVolumeCount()),
+ RemoteVolumeCount: uint64(r.GetRemoteVolumeCount()),
}
for _, c := range r.Children() {
dn := c.(*DataNode)
diff --git a/weed/topology/store_replicate.go b/weed/topology/store_replicate.go
index d21c4d210..8c4996d45 100644
--- a/weed/topology/store_replicate.go
+++ b/weed/topology/store_replicate.go
@@ -1,7 +1,6 @@
package topology
import (
- "bytes"
"encoding/json"
"errors"
"fmt"
@@ -25,58 +24,60 @@ func ReplicatedWrite(masterNode string, s *storage.Store,
//check JWT
jwt := security.GetJwt(r)
+ var remoteLocations []operation.Location
+ if r.FormValue("type") != "replicate" {
+ remoteLocations, err = getWritableRemoteReplications(s, volumeId, masterNode)
+ if err != nil {
+ glog.V(0).Infoln(err)
+ return
+ }
+ }
+
size, isUnchanged, err = s.WriteVolumeNeedle(volumeId, n)
if err != nil {
err = fmt.Errorf("failed to write to local disk: %v", err)
+ glog.V(0).Infoln(err)
return
}
- needToReplicate := !s.HasVolume(volumeId)
- needToReplicate = needToReplicate || s.GetVolume(volumeId).NeedToReplicate()
- if !needToReplicate {
- needToReplicate = s.GetVolume(volumeId).NeedToReplicate()
- }
- if needToReplicate { //send to other replica locations
- if r.FormValue("type") != "replicate" {
-
- if err = distributedOperation(masterNode, s, volumeId, func(location operation.Location) error {
- u := url.URL{
- Scheme: "http",
- Host: location.Url,
- Path: r.URL.Path,
- }
- q := url.Values{
- "type": {"replicate"},
- "ttl": {n.Ttl.String()},
- }
- if n.LastModified > 0 {
- q.Set("ts", strconv.FormatUint(n.LastModified, 10))
- }
- if n.IsChunkedManifest() {
- q.Set("cm", "true")
+ if len(remoteLocations) > 0 { //send to other replica locations
+ if err = distributedOperation(remoteLocations, s, func(location operation.Location) error {
+ u := url.URL{
+ Scheme: "http",
+ Host: location.Url,
+ Path: r.URL.Path,
+ }
+ q := url.Values{
+ "type": {"replicate"},
+ "ttl": {n.Ttl.String()},
+ }
+ if n.LastModified > 0 {
+ q.Set("ts", strconv.FormatUint(n.LastModified, 10))
+ }
+ if n.IsChunkedManifest() {
+ q.Set("cm", "true")
+ }
+ u.RawQuery = q.Encode()
+
+ pairMap := make(map[string]string)
+ if n.HasPairs() {
+ tmpMap := make(map[string]string)
+ err := json.Unmarshal(n.Pairs, &tmpMap)
+ if err != nil {
+ glog.V(0).Infoln("Unmarshal pairs error:", err)
}
- u.RawQuery = q.Encode()
-
- pairMap := make(map[string]string)
- if n.HasPairs() {
- tmpMap := make(map[string]string)
- err := json.Unmarshal(n.Pairs, &tmpMap)
- if err != nil {
- glog.V(0).Infoln("Unmarshal pairs error:", err)
- }
- for k, v := range tmpMap {
- pairMap[needle.PairNamePrefix+k] = v
- }
+ for k, v := range tmpMap {
+ pairMap[needle.PairNamePrefix+k] = v
}
-
- _, err := operation.Upload(u.String(),
- string(n.Name), bytes.NewReader(n.Data), n.IsGzipped(), string(n.Mime),
- pairMap, jwt)
- return err
- }); err != nil {
- size = 0
- err = fmt.Errorf("failed to write to replicas for volume %d: %v", volumeId, err)
}
+
+ // volume server do not know about encryption
+ _, err := operation.UploadData(u.String(), string(n.Name), false, n.Data, n.IsGzipped(), string(n.Mime), pairMap, jwt)
+ return err
+ }); err != nil {
+ size = 0
+ err = fmt.Errorf("failed to write to replicas for volume %d: %v", volumeId, err)
+ glog.V(0).Infoln(err)
}
}
return
@@ -84,31 +85,34 @@ func ReplicatedWrite(masterNode string, s *storage.Store,
func ReplicatedDelete(masterNode string, store *storage.Store,
volumeId needle.VolumeId, n *needle.Needle,
- r *http.Request) (uint32, error) {
+ r *http.Request) (size uint32, err error) {
//check JWT
jwt := security.GetJwt(r)
- ret, err := store.DeleteVolumeNeedle(volumeId, n)
+ var remoteLocations []operation.Location
+ if r.FormValue("type") != "replicate" {
+ remoteLocations, err = getWritableRemoteReplications(store, volumeId, masterNode)
+ if err != nil {
+ glog.V(0).Infoln(err)
+ return
+ }
+ }
+
+ size, err = store.DeleteVolumeNeedle(volumeId, n)
if err != nil {
glog.V(0).Infoln("delete error:", err)
- return ret, err
+ return
}
- needToReplicate := !store.HasVolume(volumeId)
- if !needToReplicate && ret > 0 {
- needToReplicate = store.GetVolume(volumeId).NeedToReplicate()
- }
- if needToReplicate { //send to other replica locations
- if r.FormValue("type") != "replicate" {
- if err = distributedOperation(masterNode, store, volumeId, func(location operation.Location) error {
- return util.Delete("http://"+location.Url+r.URL.Path+"?type=replicate", string(jwt))
- }); err != nil {
- ret = 0
- }
+ if len(remoteLocations) > 0 { //send to other replica locations
+ if err = distributedOperation(remoteLocations, store, func(location operation.Location) error {
+ return util.Delete("http://"+location.Url+r.URL.Path+"?type=replicate", string(jwt))
+ }); err != nil {
+ size = 0
}
}
- return ret, err
+ return
}
type DistributedOperationResult map[string]error
@@ -131,32 +135,44 @@ type RemoteResult struct {
Error error
}
-func distributedOperation(masterNode string, store *storage.Store, volumeId needle.VolumeId, op func(location operation.Location) error) error {
- if lookupResult, lookupErr := operation.Lookup(masterNode, volumeId.String()); lookupErr == nil {
- length := 0
- selfUrl := (store.Ip + ":" + strconv.Itoa(store.Port))
- results := make(chan RemoteResult)
- for _, location := range lookupResult.Locations {
- if location.Url != selfUrl {
- length++
- go func(location operation.Location, results chan RemoteResult) {
- results <- RemoteResult{location.Url, op(location)}
- }(location, results)
+func distributedOperation(locations []operation.Location, store *storage.Store, op func(location operation.Location) error) error {
+ length := len(locations)
+ results := make(chan RemoteResult)
+ for _, location := range locations {
+ go func(location operation.Location, results chan RemoteResult) {
+ results <- RemoteResult{location.Url, op(location)}
+ }(location, results)
+ }
+ ret := DistributedOperationResult(make(map[string]error))
+ for i := 0; i < length; i++ {
+ result := <-results
+ ret[result.Host] = result.Error
+ }
+
+ return ret.Error()
+}
+
+func getWritableRemoteReplications(s *storage.Store, volumeId needle.VolumeId, masterNode string) (
+ remoteLocations []operation.Location, err error) {
+ copyCount := s.GetVolume(volumeId).ReplicaPlacement.GetCopyCount()
+ if copyCount > 1 {
+ if lookupResult, lookupErr := operation.Lookup(masterNode, volumeId.String()); lookupErr == nil {
+ if len(lookupResult.Locations) < copyCount {
+ err = fmt.Errorf("replicating opetations [%d] is less than volume %d replication copy count [%d]",
+ len(lookupResult.Locations), volumeId, copyCount)
+ return
}
- }
- ret := DistributedOperationResult(make(map[string]error))
- for i := 0; i < length; i++ {
- result := <-results
- ret[result.Host] = result.Error
- }
- if volume := store.GetVolume(volumeId); volume != nil {
- if length+1 < volume.ReplicaPlacement.GetCopyCount() {
- return fmt.Errorf("replicating opetations [%d] is less than volume's replication copy count [%d]", length+1, volume.ReplicaPlacement.GetCopyCount())
+ selfUrl := s.Ip + ":" + strconv.Itoa(s.Port)
+ for _, location := range lookupResult.Locations {
+ if location.Url != selfUrl {
+ remoteLocations = append(remoteLocations, location)
+ }
}
+ } else {
+ err = fmt.Errorf("failed to lookup for %d: %v", volumeId, lookupErr)
+ return
}
- return ret.Error()
- } else {
- glog.V(0).Infoln()
- return fmt.Errorf("Failed to lookup for %d: %v", volumeId, lookupErr)
}
+
+ return
}
diff --git a/weed/topology/topology.go b/weed/topology/topology.go
index b7ebe8af5..fbf998707 100644
--- a/weed/topology/topology.go
+++ b/weed/topology/topology.go
@@ -7,11 +7,13 @@ import (
"sync"
"github.com/chrislusf/raft"
+
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/sequence"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"github.com/chrislusf/seaweedfs/weed/util"
)
@@ -58,7 +60,12 @@ func NewTopology(id string, seq sequence.Sequencer, volumeSizeLimit uint64, puls
func (t *Topology) IsLeader() bool {
if t.RaftServer != nil {
- return t.RaftServer.State() == raft.Leader
+ if t.RaftServer.State() == raft.Leader {
+ return true
+ }
+ if t.RaftServer.Leader() == "" {
+ return true
+ }
}
return false
}
@@ -73,7 +80,7 @@ func (t *Topology) Leader() (string, error) {
if l == "" {
// We are a single node cluster, we are the leader
- return t.RaftServer.Name(), errors.New("Raft Server not initialized!")
+ return t.RaftServer.Name(), nil
}
return l, nil
@@ -129,7 +136,7 @@ func (t *Topology) PickForWrite(count uint64, option *VolumeGrowOption) (string,
return needle.NewFileId(*vid, fileId, rand.Uint32()).String(), count, datanodes.Head(), nil
}
-func (t *Topology) GetVolumeLayout(collectionName string, rp *storage.ReplicaPlacement, ttl *needle.TTL) *VolumeLayout {
+func (t *Topology) GetVolumeLayout(collectionName string, rp *super_block.ReplicaPlacement, ttl *needle.TTL) *VolumeLayout {
return t.collectionMap.Get(collectionName, func() interface{} {
return NewCollection(collectionName, t.volumeSizeLimit)
}).(*Collection).GetOrCreateVolumeLayout(rp, ttl)
@@ -150,7 +157,7 @@ func (t *Topology) ListCollections(includeNormalVolumes, includeEcVolumes bool)
t.ecShardMapLock.RUnlock()
}
- for k, _ := range mapOfCollections {
+ for k := range mapOfCollections {
ret = append(ret, k)
}
return ret
diff --git a/weed/topology/topology_event_handling.go b/weed/topology/topology_event_handling.go
index 041351492..068bd401e 100644
--- a/weed/topology/topology_event_handling.go
+++ b/weed/topology/topology_event_handling.go
@@ -59,6 +59,7 @@ func (t *Topology) UnRegisterDataNode(dn *DataNode) {
vl.SetVolumeUnavailable(dn, v.Id)
}
dn.UpAdjustVolumeCountDelta(-dn.GetVolumeCount())
+ dn.UpAdjustRemoteVolumeCountDelta(-dn.GetRemoteVolumeCount())
dn.UpAdjustActiveVolumeCountDelta(-dn.GetActiveVolumeCount())
dn.UpAdjustMaxVolumeCountDelta(-dn.GetMaxVolumeCount())
if dn.Parent() != nil {
diff --git a/weed/topology/topology_map.go b/weed/topology/topology_map.go
index 0ad30f12e..73c55d77d 100644
--- a/weed/topology/topology_map.go
+++ b/weed/topology/topology_map.go
@@ -85,6 +85,7 @@ func (t *Topology) ToTopologyInfo() *master_pb.TopologyInfo {
MaxVolumeCount: uint64(t.GetMaxVolumeCount()),
FreeVolumeCount: uint64(t.FreeSpace()),
ActiveVolumeCount: uint64(t.GetActiveVolumeCount()),
+ RemoteVolumeCount: uint64(t.GetRemoteVolumeCount()),
}
for _, c := range t.Children() {
dc := c.(*DataCenter)
diff --git a/weed/topology/topology_test.go b/weed/topology/topology_test.go
index 8f79ad684..e7676ccf7 100644
--- a/weed/topology/topology_test.go
+++ b/weed/topology/topology_test.go
@@ -5,6 +5,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/sequence"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
"testing"
)
@@ -94,7 +95,7 @@ func TestHandlingVolumeServerHeartbeat(t *testing.T) {
[]*master_pb.VolumeShortInformationMessage{newVolumeShortMessage},
nil,
dn)
- rp, _ := storage.NewReplicaPlacementFromString("000")
+ rp, _ := super_block.NewReplicaPlacementFromString("000")
layout := topo.GetVolumeLayout("", rp, needle.EMPTY_TTL)
assert(t, "writables after repeated add", len(layout.writables), volumeCount)
@@ -154,7 +155,7 @@ func TestAddRemoveVolume(t *testing.T) {
DeletedByteCount: 45,
ReadOnly: false,
Version: needle.CurrentVersion,
- ReplicaPlacement: &storage.ReplicaPlacement{},
+ ReplicaPlacement: &super_block.ReplicaPlacement{},
Ttl: needle.EMPTY_TTL,
}
diff --git a/weed/topology/topology_vacuum.go b/weed/topology/topology_vacuum.go
index ff32f1874..ca626e973 100644
--- a/weed/topology/topology_vacuum.go
+++ b/weed/topology/topology_vacuum.go
@@ -13,8 +13,10 @@ import (
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
)
-func batchVacuumVolumeCheck(grpcDialOption grpc.DialOption, vl *VolumeLayout, vid needle.VolumeId, locationlist *VolumeLocationList, garbageThreshold float64) bool {
- ch := make(chan bool, locationlist.Length())
+func batchVacuumVolumeCheck(grpcDialOption grpc.DialOption, vl *VolumeLayout, vid needle.VolumeId,
+ locationlist *VolumeLocationList, garbageThreshold float64) (*VolumeLocationList, bool) {
+ ch := make(chan int, locationlist.Length())
+ errCount := int32(0)
for index, dn := range locationlist.list {
go func(index int, url string, vid needle.VolumeId) {
err := operation.WithVolumeServerClient(url, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
@@ -22,11 +24,15 @@ func batchVacuumVolumeCheck(grpcDialOption grpc.DialOption, vl *VolumeLayout, vi
VolumeId: uint32(vid),
})
if err != nil {
- ch <- false
+ atomic.AddInt32(&errCount, 1)
+ ch <- -1
return err
}
- isNeeded := resp.GarbageRatio > garbageThreshold
- ch <- isNeeded
+ if resp.GarbageRatio >= garbageThreshold {
+ ch <- index
+ } else {
+ ch <- -1
+ }
return nil
})
if err != nil {
@@ -34,18 +40,21 @@ func batchVacuumVolumeCheck(grpcDialOption grpc.DialOption, vl *VolumeLayout, vi
}
}(index, dn.Url(), vid)
}
- isCheckSuccess := true
+ vacuumLocationList := NewVolumeLocationList()
for range locationlist.list {
select {
- case canVacuum := <-ch:
- isCheckSuccess = isCheckSuccess && canVacuum
+ case index := <-ch:
+ if index != -1 {
+ vacuumLocationList.list = append(vacuumLocationList.list, locationlist.list[index])
+ }
case <-time.After(30 * time.Minute):
- return false
+ return vacuumLocationList, false
}
}
- return isCheckSuccess
+ return vacuumLocationList, errCount == 0 && len(vacuumLocationList.list) > 0
}
-func batchVacuumVolumeCompact(grpcDialOption grpc.DialOption, vl *VolumeLayout, vid needle.VolumeId, locationlist *VolumeLocationList, preallocate int64) bool {
+func batchVacuumVolumeCompact(grpcDialOption grpc.DialOption, vl *VolumeLayout, vid needle.VolumeId,
+ locationlist *VolumeLocationList, preallocate int64) bool {
vl.accessLock.Lock()
vl.removeFromWritable(vid)
vl.accessLock.Unlock()
@@ -163,11 +172,12 @@ func vacuumOneVolumeLayout(grpcDialOption grpc.DialOption, volumeLayout *VolumeL
}
glog.V(2).Infof("check vacuum on collection:%s volume:%d", c.Name, vid)
- if batchVacuumVolumeCheck(grpcDialOption, volumeLayout, vid, locationList, garbageThreshold) {
- if batchVacuumVolumeCompact(grpcDialOption, volumeLayout, vid, locationList, preallocate) {
- batchVacuumVolumeCommit(grpcDialOption, volumeLayout, vid, locationList)
+ if vacuumLocationList, needVacuum := batchVacuumVolumeCheck(
+ grpcDialOption, volumeLayout, vid, locationList, garbageThreshold); needVacuum {
+ if batchVacuumVolumeCompact(grpcDialOption, volumeLayout, vid, vacuumLocationList, preallocate) {
+ batchVacuumVolumeCommit(grpcDialOption, volumeLayout, vid, vacuumLocationList)
} else {
- batchVacuumVolumeCleanup(grpcDialOption, volumeLayout, vid, locationList)
+ batchVacuumVolumeCleanup(grpcDialOption, volumeLayout, vid, vacuumLocationList)
}
}
}
diff --git a/weed/topology/volume_growth.go b/weed/topology/volume_growth.go
index 636eb2260..446c88f60 100644
--- a/weed/topology/volume_growth.go
+++ b/weed/topology/volume_growth.go
@@ -6,6 +6,9 @@ import (
"sync"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
+ "github.com/chrislusf/seaweedfs/weed/util"
+
"google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/glog"
@@ -22,7 +25,7 @@ This package is created to resolve these replica placement issues:
type VolumeGrowOption struct {
Collection string
- ReplicaPlacement *storage.ReplicaPlacement
+ ReplicaPlacement *super_block.ReplicaPlacement
Ttl *needle.TTL
Prealloacte int64
DataCenter string
@@ -46,15 +49,20 @@ func NewDefaultVolumeGrowth() *VolumeGrowth {
// one replication type may need rp.GetCopyCount() actual volumes
// given copyCount, how many logical volumes to create
func (vg *VolumeGrowth) findVolumeCount(copyCount int) (count int) {
+ v := util.GetViper()
+ v.SetDefault("master.volume_growth.copy_1", 7)
+ v.SetDefault("master.volume_growth.copy_2", 6)
+ v.SetDefault("master.volume_growth.copy_3", 3)
+ v.SetDefault("master.volume_growth.copy_other", 1)
switch copyCount {
case 1:
- count = 7
+ count = v.GetInt("master.volume_growth.copy_1")
case 2:
- count = 6
+ count = v.GetInt("master.volume_growth.copy_2")
case 3:
- count = 3
+ count = v.GetInt("master.volume_growth.copy_3")
default:
- count = 1
+ count = v.GetInt("master.volume_growth.copy_other")
}
return
}
@@ -104,7 +112,7 @@ func (vg *VolumeGrowth) findAndGrow(grpcDialOption grpc.DialOption, topo *Topolo
func (vg *VolumeGrowth) findEmptySlotsForOneVolume(topo *Topology, option *VolumeGrowOption) (servers []*DataNode, err error) {
//find main datacenter and other data centers
rp := option.ReplicaPlacement
- mainDataCenter, otherDataCenters, dc_err := topo.RandomlyPickNodes(rp.DiffDataCenterCount+1, func(node Node) error {
+ mainDataCenter, otherDataCenters, dc_err := topo.PickNodesByWeight(rp.DiffDataCenterCount+1, func(node Node) error {
if option.DataCenter != "" && node.IsDataCenter() && node.Id() != NodeId(option.DataCenter) {
return fmt.Errorf("Not matching preferred data center:%s", option.DataCenter)
}
@@ -136,7 +144,7 @@ func (vg *VolumeGrowth) findEmptySlotsForOneVolume(topo *Topology, option *Volum
}
//find main rack and other racks
- mainRack, otherRacks, rackErr := mainDataCenter.(*DataCenter).RandomlyPickNodes(rp.DiffRackCount+1, func(node Node) error {
+ mainRack, otherRacks, rackErr := mainDataCenter.(*DataCenter).PickNodesByWeight(rp.DiffRackCount+1, func(node Node) error {
if option.Rack != "" && node.IsRack() && node.Id() != NodeId(option.Rack) {
return fmt.Errorf("Not matching preferred rack:%s", option.Rack)
}
@@ -163,7 +171,7 @@ func (vg *VolumeGrowth) findEmptySlotsForOneVolume(topo *Topology, option *Volum
}
//find main rack and other racks
- mainServer, otherServers, serverErr := mainRack.(*Rack).RandomlyPickNodes(rp.SameRackCount+1, func(node Node) error {
+ mainServer, otherServers, serverErr := mainRack.(*Rack).PickNodesByWeight(rp.SameRackCount+1, func(node Node) error {
if option.DataNode != "" && node.IsDataNode() && node.Id() != NodeId(option.DataNode) {
return fmt.Errorf("Not matching preferred data node:%s", option.DataNode)
}
diff --git a/weed/topology/volume_growth_test.go b/weed/topology/volume_growth_test.go
index 3573365fd..6ff5be0eb 100644
--- a/weed/topology/volume_growth_test.go
+++ b/weed/topology/volume_growth_test.go
@@ -8,6 +8,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/sequence"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
)
var topologyLayout = `
@@ -113,7 +114,7 @@ func setup(topologyLayout string) *Topology {
func TestFindEmptySlotsForOneVolume(t *testing.T) {
topo := setup(topologyLayout)
vg := NewDefaultVolumeGrowth()
- rp, _ := storage.NewReplicaPlacementFromString("002")
+ rp, _ := super_block.NewReplicaPlacementFromString("002")
volumeGrowOption := &VolumeGrowOption{
Collection: "",
ReplicaPlacement: rp,
@@ -130,3 +131,212 @@ func TestFindEmptySlotsForOneVolume(t *testing.T) {
fmt.Println("assigned node :", server.Id())
}
}
+
+var topologyLayout2 = `
+{
+ "dc1":{
+ "rack1":{
+ "server111":{
+ "volumes":[
+ {"id":1, "size":12312},
+ {"id":2, "size":12312},
+ {"id":3, "size":12312}
+ ],
+ "limit":300
+ },
+ "server112":{
+ "volumes":[
+ {"id":4, "size":12312},
+ {"id":5, "size":12312},
+ {"id":6, "size":12312}
+ ],
+ "limit":300
+ },
+ "server113":{
+ "volumes":[],
+ "limit":300
+ },
+ "server114":{
+ "volumes":[],
+ "limit":300
+ },
+ "server115":{
+ "volumes":[],
+ "limit":300
+ },
+ "server116":{
+ "volumes":[],
+ "limit":300
+ }
+ },
+ "rack2":{
+ "server121":{
+ "volumes":[
+ {"id":4, "size":12312},
+ {"id":5, "size":12312},
+ {"id":6, "size":12312}
+ ],
+ "limit":300
+ },
+ "server122":{
+ "volumes":[],
+ "limit":300
+ },
+ "server123":{
+ "volumes":[
+ {"id":2, "size":12312},
+ {"id":3, "size":12312},
+ {"id":4, "size":12312}
+ ],
+ "limit":300
+ },
+ "server124":{
+ "volumes":[],
+ "limit":300
+ },
+ "server125":{
+ "volumes":[],
+ "limit":300
+ },
+ "server126":{
+ "volumes":[],
+ "limit":300
+ }
+ },
+ "rack3":{
+ "server131":{
+ "volumes":[],
+ "limit":300
+ },
+ "server132":{
+ "volumes":[],
+ "limit":300
+ },
+ "server133":{
+ "volumes":[],
+ "limit":300
+ },
+ "server134":{
+ "volumes":[],
+ "limit":300
+ },
+ "server135":{
+ "volumes":[],
+ "limit":300
+ },
+ "server136":{
+ "volumes":[],
+ "limit":300
+ }
+ }
+ }
+}
+`
+
+func TestReplication011(t *testing.T) {
+ topo := setup(topologyLayout2)
+ vg := NewDefaultVolumeGrowth()
+ rp, _ := super_block.NewReplicaPlacementFromString("011")
+ volumeGrowOption := &VolumeGrowOption{
+ Collection: "MAIL",
+ ReplicaPlacement: rp,
+ DataCenter: "dc1",
+ Rack: "",
+ DataNode: "",
+ }
+ servers, err := vg.findEmptySlotsForOneVolume(topo, volumeGrowOption)
+ if err != nil {
+ fmt.Println("finding empty slots error :", err)
+ t.Fail()
+ }
+ for _, server := range servers {
+ fmt.Println("assigned node :", server.Id())
+ }
+}
+
+var topologyLayout3 = `
+{
+ "dc1":{
+ "rack1":{
+ "server111":{
+ "volumes":[],
+ "limit":2000
+ }
+ }
+ },
+ "dc2":{
+ "rack2":{
+ "server222":{
+ "volumes":[],
+ "limit":2000
+ }
+ }
+ },
+ "dc3":{
+ "rack3":{
+ "server333":{
+ "volumes":[],
+ "limit":1000
+ }
+ }
+ },
+ "dc4":{
+ "rack4":{
+ "server444":{
+ "volumes":[],
+ "limit":1000
+ }
+ }
+ },
+ "dc5":{
+ "rack5":{
+ "server555":{
+ "volumes":[],
+ "limit":500
+ }
+ }
+ },
+ "dc6":{
+ "rack6":{
+ "server666":{
+ "volumes":[],
+ "limit":500
+ }
+ }
+ }
+}
+`
+
+func TestFindEmptySlotsForOneVolumeScheduleByWeight(t *testing.T) {
+ topo := setup(topologyLayout3)
+ vg := NewDefaultVolumeGrowth()
+ rp, _ := super_block.NewReplicaPlacementFromString("100")
+ volumeGrowOption := &VolumeGrowOption{
+ Collection: "Weight",
+ ReplicaPlacement: rp,
+ DataCenter: "",
+ Rack: "",
+ DataNode: "",
+ }
+
+ distribution := map[NodeId]int{}
+ // assign 1000 volumes
+ for i := 0; i < 1000; i++ {
+ servers, err := vg.findEmptySlotsForOneVolume(topo, volumeGrowOption)
+ if err != nil {
+ fmt.Println("finding empty slots error :", err)
+ t.Fail()
+ }
+ for _, server := range servers {
+ // fmt.Println("assigned node :", server.Id())
+ if _, ok := distribution[server.id]; !ok {
+ distribution[server.id] = 0
+ }
+ distribution[server.id] += 1
+ }
+ }
+
+ for k, v := range distribution {
+ fmt.Printf("%s : %d\n", k, v)
+ }
+}
diff --git a/weed/topology/volume_layout.go b/weed/topology/volume_layout.go
index 799cbca62..7633b28be 100644
--- a/weed/topology/volume_layout.go
+++ b/weed/topology/volume_layout.go
@@ -10,11 +10,12 @@ import (
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/storage"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
+ "github.com/chrislusf/seaweedfs/weed/storage/super_block"
)
// mapping from volume to its locations, inverted from server to volume
type VolumeLayout struct {
- rp *storage.ReplicaPlacement
+ rp *super_block.ReplicaPlacement
ttl *needle.TTL
vid2location map[needle.VolumeId]*VolumeLocationList
writables []needle.VolumeId // transient array of writable volume id
@@ -30,7 +31,7 @@ type VolumeLayoutStats struct {
FileCount uint64
}
-func NewVolumeLayout(rp *storage.ReplicaPlacement, ttl *needle.TTL, volumeSizeLimit uint64) *VolumeLayout {
+func NewVolumeLayout(rp *super_block.ReplicaPlacement, ttl *needle.TTL, volumeSizeLimit uint64) *VolumeLayout {
return &VolumeLayout{
rp: rp,
ttl: ttl,
diff --git a/weed/util/bytes.go b/weed/util/bytes.go
index dfa4ae665..9c7e5e2cb 100644
--- a/weed/util/bytes.go
+++ b/weed/util/bytes.go
@@ -1,5 +1,10 @@
package util
+import (
+ "crypto/md5"
+ "io"
+)
+
// big endian
func BytesToUint64(b []byte) (v uint64) {
@@ -43,3 +48,29 @@ func Uint16toBytes(b []byte, v uint16) {
func Uint8toBytes(b []byte, v uint8) {
b[0] = byte(v)
}
+
+// returns a 64 bit big int
+func HashStringToLong(dir string) (v int64) {
+ h := md5.New()
+ io.WriteString(h, dir)
+
+ b := h.Sum(nil)
+
+ v += int64(b[0])
+ v <<= 8
+ v += int64(b[1])
+ v <<= 8
+ v += int64(b[2])
+ v <<= 8
+ v += int64(b[3])
+ v <<= 8
+ v += int64(b[4])
+ v <<= 8
+ v += int64(b[5])
+ v <<= 8
+ v += int64(b[6])
+ v <<= 8
+ v += int64(b[7])
+
+ return
+}
diff --git a/weed/util/cipher.go b/weed/util/cipher.go
new file mode 100644
index 000000000..7bcb6559a
--- /dev/null
+++ b/weed/util/cipher.go
@@ -0,0 +1,81 @@
+package util
+
+import (
+ "bytes"
+ "crypto/aes"
+ "crypto/cipher"
+ "crypto/rand"
+ "errors"
+ "fmt"
+ "io"
+ "io/ioutil"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
+)
+
+type CipherKey []byte
+
+func GenCipherKey() CipherKey {
+ key := make([]byte, 32)
+ if _, err := io.ReadFull(rand.Reader, key); err != nil {
+ glog.Fatalf("random key gen: %v", err)
+ }
+ return CipherKey(key)
+}
+
+func Encrypt(plaintext []byte, key CipherKey) ([]byte, error) {
+ c, err := aes.NewCipher(key)
+ if err != nil {
+ return nil, err
+ }
+
+ gcm, err := cipher.NewGCM(c)
+ if err != nil {
+ return nil, err
+ }
+
+ nonce := make([]byte, gcm.NonceSize())
+ if _, err = io.ReadFull(rand.Reader, nonce); err != nil {
+ return nil, err
+ }
+
+ return gcm.Seal(nonce, nonce, plaintext, nil), nil
+}
+
+func Decrypt(ciphertext []byte, key CipherKey) ([]byte, error) {
+ c, err := aes.NewCipher(key)
+ if err != nil {
+ return nil, err
+ }
+
+ gcm, err := cipher.NewGCM(c)
+ if err != nil {
+ return nil, err
+ }
+
+ nonceSize := gcm.NonceSize()
+ if len(ciphertext) < nonceSize {
+ return nil, errors.New("ciphertext too short")
+ }
+
+ nonce, ciphertext := ciphertext[:nonceSize], ciphertext[nonceSize:]
+ return gcm.Open(nil, nonce, ciphertext, nil)
+}
+
+func EncryptReader(clearReader io.Reader) (cipherKey CipherKey, encryptedReader io.ReadCloser, clearDataLen, encryptedDataLen int, err error) {
+ clearData, err := ioutil.ReadAll(clearReader)
+ if err != nil {
+ err = fmt.Errorf("read raw input: %v", err)
+ return
+ }
+ clearDataLen = len(clearData)
+ cipherKey = GenCipherKey()
+ encryptedData, err := Encrypt(clearData, cipherKey)
+ if err != nil {
+ err = fmt.Errorf("encrypt input: %v", err)
+ return
+ }
+ encryptedDataLen = len(encryptedData)
+ encryptedReader = ioutil.NopCloser(bytes.NewReader(encryptedData))
+ return
+}
diff --git a/weed/util/compression.go b/weed/util/compression.go
index c6c9423e2..6072df632 100644
--- a/weed/util/compression.go
+++ b/weed/util/compression.go
@@ -60,7 +60,7 @@ func UnGzipData(input []byte) ([]byte, error) {
// images
switch ext {
- case ".svg", ".bmp":
+ case ".svg", ".bmp", ".wav":
return true, true
}
if strings.HasPrefix(mtype, "image/") {
@@ -87,6 +87,14 @@ func UnGzipData(input []byte) ([]byte, error) {
if strings.HasSuffix(mtype, "script") {
return true, true
}
+
+ }
+
+ if strings.HasPrefix(mtype, "audio/") {
+ switch strings.TrimPrefix(mtype, "audio/") {
+ case "wave", "wav", "x-wav", "x-pn-wav":
+ return true, true
+ }
}
return false, false
diff --git a/weed/util/config.go b/weed/util/config.go
index 84f146bc8..dfbfdbd82 100644
--- a/weed/util/config.go
+++ b/weed/util/config.go
@@ -1,17 +1,19 @@
package util
import (
- "github.com/chrislusf/seaweedfs/weed/glog"
+ "strings"
+
"github.com/spf13/viper"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
)
type Configuration interface {
GetString(key string) string
GetBool(key string) bool
GetInt(key string) int
- GetInt64(key string) int64
- GetFloat64(key string) float64
GetStringSlice(key string) []string
+ SetDefault(key string, value interface{})
}
func LoadConfiguration(configFileName string, required bool) (loaded bool) {
@@ -28,10 +30,7 @@ func LoadConfiguration(configFileName string, required bool) (loaded bool) {
glog.V(0).Infof("Reading %s: %v", viper.ConfigFileUsed(), err)
if required {
glog.Fatalf("Failed to load %s.toml file from current directory, or $HOME/.seaweedfs/, or /etc/seaweedfs/"+
- "\n\nPlease follow this example and add a filer.toml file to "+
- "current directory, or $HOME/.seaweedfs/, or /etc/seaweedfs/:\n"+
- " https://github.com/chrislusf/seaweedfs/blob/master/weed/%s.toml\n"+
- "\nOr use this command to generate the default toml file\n"+
+ "\n\nPlease use this command to generate the default %s.toml file\n"+
" weed scaffold -config=%s -output=.\n\n\n",
configFileName, configFileName, configFileName)
} else {
@@ -41,3 +40,11 @@ func LoadConfiguration(configFileName string, required bool) (loaded bool) {
return true
}
+
+func GetViper() *viper.Viper {
+ v := viper.GetViper()
+ v.AutomaticEnv()
+ v.SetEnvPrefix("weed")
+ v.SetEnvKeyReplacer(strings.NewReplacer(".", "_"))
+ return v
+}
diff --git a/weed/util/constants.go b/weed/util/constants.go
index f0df5fd59..c23bc11f6 100644
--- a/weed/util/constants.go
+++ b/weed/util/constants.go
@@ -5,5 +5,5 @@ import (
)
var (
- VERSION = fmt.Sprintf("%s %d.%d", sizeLimit, 1, 45)
+ VERSION = fmt.Sprintf("%s %d.%d", sizeLimit, 1, 61)
)
diff --git a/weed/util/file_util.go b/weed/util/file_util.go
index 78add6724..bef9f7cd6 100644
--- a/weed/util/file_util.go
+++ b/weed/util/file_util.go
@@ -3,6 +3,7 @@ package util
import (
"errors"
"os"
+ "time"
"github.com/chrislusf/seaweedfs/weed/glog"
)
@@ -40,3 +41,21 @@ func FileExists(filename string) bool {
return true
}
+
+func CheckFile(filename string) (exists, canRead, canWrite bool, modTime time.Time, fileSize int64) {
+ exists = true
+ fi, err := os.Stat(filename)
+ if os.IsNotExist(err) {
+ exists = false
+ return
+ }
+ if fi.Mode()&0400 != 0 {
+ canRead = true
+ }
+ if fi.Mode()&0200 != 0 {
+ canWrite = true
+ }
+ modTime = fi.ModTime()
+ fileSize = fi.Size()
+ return
+}
diff --git a/weed/util/http_util.go b/weed/util/http_util.go
index 79a442a56..750516b92 100644
--- a/weed/util/http_util.go
+++ b/weed/util/http_util.go
@@ -11,6 +11,8 @@ import (
"net/http"
"net/url"
"strings"
+
+ "github.com/chrislusf/seaweedfs/weed/glog"
)
var (
@@ -28,18 +30,18 @@ func init() {
}
func PostBytes(url string, body []byte) ([]byte, error) {
- r, err := client.Post(url, "application/octet-stream", bytes.NewReader(body))
+ r, err := client.Post(url, "", bytes.NewReader(body))
if err != nil {
return nil, fmt.Errorf("Post to %s: %v", url, err)
}
defer r.Body.Close()
- if r.StatusCode >= 400 {
- return nil, fmt.Errorf("%s: %s", url, r.Status)
- }
b, err := ioutil.ReadAll(r.Body)
if err != nil {
return nil, fmt.Errorf("Read response body: %v", err)
}
+ if r.StatusCode >= 400 {
+ return nil, fmt.Errorf("%s: %s", url, r.Status)
+ }
return b, nil
}
@@ -86,7 +88,7 @@ func Head(url string) (http.Header, error) {
if err != nil {
return nil, err
}
- defer r.Body.Close()
+ defer CloseResponse(r)
if r.StatusCode >= 400 {
return nil, fmt.Errorf("%s: %s", url, r.Status)
}
@@ -128,7 +130,7 @@ func GetBufferStream(url string, values url.Values, allocatedBytes []byte, eachB
if err != nil {
return err
}
- defer r.Body.Close()
+ defer CloseResponse(r)
if r.StatusCode != 200 {
return fmt.Errorf("%s: %s", url, r.Status)
}
@@ -151,7 +153,7 @@ func GetUrlStream(url string, values url.Values, readFn func(io.Reader) error) e
if err != nil {
return err
}
- defer r.Body.Close()
+ defer CloseResponse(r)
if r.StatusCode != 200 {
return fmt.Errorf("%s: %s", url, r.Status)
}
@@ -187,11 +189,22 @@ func NormalizeUrl(url string) string {
return "http://" + url
}
-func ReadUrl(fileUrl string, offset int64, size int, buf []byte, isReadRange bool) (n int64, e error) {
+func ReadUrl(fileUrl string, cipherKey []byte, isGzipped bool, isFullChunk bool, offset int64, size int, buf []byte) (int64, error) {
+
+ if cipherKey != nil {
+ var n int
+ err := readEncryptedUrl(fileUrl, cipherKey, isGzipped, offset, size, func(data []byte) {
+ n = copy(buf, data)
+ })
+ return int64(n), err
+ }
- req, _ := http.NewRequest("GET", fileUrl, nil)
- if isReadRange {
- req.Header.Add("Range", fmt.Sprintf("bytes=%d-%d", offset, offset+int64(size)))
+ req, err := http.NewRequest("GET", fileUrl, nil)
+ if err != nil {
+ return 0, err
+ }
+ if !isFullChunk {
+ req.Header.Add("Range", fmt.Sprintf("bytes=%d-%d", offset, offset+int64(size)-1))
} else {
req.Header.Set("Accept-Encoding", "gzip")
}
@@ -207,7 +220,8 @@ func ReadUrl(fileUrl string, offset int64, size int, buf []byte, isReadRange boo
}
var reader io.ReadCloser
- switch r.Header.Get("Content-Encoding") {
+ contentEncoding := r.Header.Get("Content-Encoding")
+ switch contentEncoding {
case "gzip":
reader, err = gzip.NewReader(r.Body)
defer reader.Close()
@@ -215,55 +229,121 @@ func ReadUrl(fileUrl string, offset int64, size int, buf []byte, isReadRange boo
reader = r.Body
}
- var i, m int
+ var (
+ i, m int
+ n int64
+ )
+ // refers to https://github.com/golang/go/blob/master/src/bytes/buffer.go#L199
+ // commit id c170b14c2c1cfb2fd853a37add92a82fd6eb4318
for {
m, err = reader.Read(buf[i:])
- if m == 0 {
- return
- }
i += m
n += int64(m)
if err == io.EOF {
return n, nil
}
- if e != nil {
- return n, e
+ if err != nil {
+ return n, err
+ }
+ if n == int64(len(buf)) {
+ break
}
}
-
+ // drains the response body to avoid memory leak
+ data, _ := ioutil.ReadAll(reader)
+ if len(data) != 0 {
+ glog.V(1).Infof("%s reader has remaining %d bytes", contentEncoding, len(data))
+ }
+ return n, err
}
-func ReadUrlAsStream(fileUrl string, offset int64, size int, fn func(data []byte)) (n int64, e error) {
+func ReadUrlAsStream(fileUrl string, cipherKey []byte, isContentGzipped bool, isFullChunk bool, offset int64, size int, fn func(data []byte)) error {
+
+ if cipherKey != nil {
+ return readEncryptedUrl(fileUrl, cipherKey, isContentGzipped, offset, size, fn)
+ }
+
+ req, err := http.NewRequest("GET", fileUrl, nil)
+ if err != nil {
+ return err
+ }
- req, _ := http.NewRequest("GET", fileUrl, nil)
- req.Header.Add("Range", fmt.Sprintf("bytes=%d-%d", offset, offset+int64(size)))
+ if !isFullChunk {
+ req.Header.Add("Range", fmt.Sprintf("bytes=%d-%d", offset, offset+int64(size)-1))
+ }
r, err := client.Do(req)
if err != nil {
- return 0, err
+ return err
}
- defer r.Body.Close()
+ defer CloseResponse(r)
if r.StatusCode >= 400 {
- return 0, fmt.Errorf("%s: %s", fileUrl, r.Status)
+ return fmt.Errorf("%s: %s", fileUrl, r.Status)
}
- var m int
+ var (
+ m int
+ )
buf := make([]byte, 64*1024)
for {
m, err = r.Body.Read(buf)
- if m == 0 {
- return
- }
fn(buf[:m])
- n += int64(m)
if err == io.EOF {
- return n, nil
+ return nil
+ }
+ if err != nil {
+ return err
}
- if e != nil {
- return n, e
+ }
+
+}
+
+func readEncryptedUrl(fileUrl string, cipherKey []byte, isContentGzipped bool, offset int64, size int, fn func(data []byte)) error {
+ encryptedData, err := Get(fileUrl)
+ if err != nil {
+ return fmt.Errorf("fetch %s: %v", fileUrl, err)
+ }
+ decryptedData, err := Decrypt(encryptedData, CipherKey(cipherKey))
+ if err != nil {
+ return fmt.Errorf("decrypt %s: %v", fileUrl, err)
+ }
+ if isContentGzipped {
+ decryptedData, err = UnGzipData(decryptedData)
+ if err != nil {
+ return fmt.Errorf("unzip decrypt %s: %v", fileUrl, err)
}
}
+ if len(decryptedData) < int(offset)+size {
+ return fmt.Errorf("read decrypted %s size %d [%d, %d)", fileUrl, len(decryptedData), offset, int(offset)+size)
+ }
+ fn(decryptedData[int(offset) : int(offset)+size])
+ return nil
+}
+
+func ReadUrlAsReaderCloser(fileUrl string, rangeHeader string) (io.ReadCloser, error) {
+
+ req, err := http.NewRequest("GET", fileUrl, nil)
+ if err != nil {
+ return nil, err
+ }
+ if rangeHeader != "" {
+ req.Header.Add("Range", rangeHeader)
+ }
+
+ r, err := client.Do(req)
+ if err != nil {
+ return nil, err
+ }
+ if r.StatusCode >= 400 {
+ return nil, fmt.Errorf("%s: %s", fileUrl, r.Status)
+ }
+
+ return r.Body, nil
+}
+func CloseResponse(resp *http.Response) {
+ io.Copy(ioutil.Discard, resp.Body)
+ resp.Body.Close()
}
diff --git a/weed/util/httpdown/http_down.go b/weed/util/httpdown/http_down.go
new file mode 100644
index 000000000..5cbd9611c
--- /dev/null
+++ b/weed/util/httpdown/http_down.go
@@ -0,0 +1,395 @@
+// Package httpdown provides http.ConnState enabled graceful termination of
+// http.Server.
+// based on github.com/facebookarchive/httpdown, who's licence is MIT-licence,
+// we add a feature of supporting for http TLS
+package httpdown
+
+import (
+ "crypto/tls"
+ "fmt"
+ "net"
+ "net/http"
+ "os"
+ "os/signal"
+ "sync"
+ "syscall"
+ "time"
+
+ "github.com/facebookgo/clock"
+ "github.com/facebookgo/stats"
+)
+
+const (
+ defaultStopTimeout = time.Minute
+ defaultKillTimeout = time.Minute
+)
+
+// A Server allows encapsulates the process of accepting new connections and
+// serving them, and gracefully shutting down the listener without dropping
+// active connections.
+type Server interface {
+ // Wait waits for the serving loop to finish. This will happen when Stop is
+ // called, at which point it returns no error, or if there is an error in the
+ // serving loop. You must call Wait after calling Serve or ListenAndServe.
+ Wait() error
+
+ // Stop stops the listener. It will block until all connections have been
+ // closed.
+ Stop() error
+}
+
+// HTTP defines the configuration for serving a http.Server. Multiple calls to
+// Serve or ListenAndServe can be made on the same HTTP instance. The default
+// timeouts of 1 minute each result in a maximum of 2 minutes before a Stop()
+// returns.
+type HTTP struct {
+ // StopTimeout is the duration before we begin force closing connections.
+ // Defaults to 1 minute.
+ StopTimeout time.Duration
+
+ // KillTimeout is the duration before which we completely give up and abort
+ // even though we still have connected clients. This is useful when a large
+ // number of client connections exist and closing them can take a long time.
+ // Note, this is in addition to the StopTimeout. Defaults to 1 minute.
+ KillTimeout time.Duration
+
+ // Stats is optional. If provided, it will be used to record various metrics.
+ Stats stats.Client
+
+ // Clock allows for testing timing related functionality. Do not specify this
+ // in production code.
+ Clock clock.Clock
+
+ // when set CertFile and KeyFile, the httpDown will start a http with TLS.
+ // Files containing a certificate and matching private key for the
+ // server must be provided if neither the Server's
+ // TLSConfig.Certificates nor TLSConfig.GetCertificate are populated.
+ // If the certificate is signed by a certificate authority, the
+ // certFile should be the concatenation of the server's certificate,
+ // any intermediates, and the CA's certificate.
+ CertFile, KeyFile string
+}
+
+// Serve provides the low-level API which is useful if you're creating your own
+// net.Listener.
+func (h HTTP) Serve(s *http.Server, l net.Listener) Server {
+ stopTimeout := h.StopTimeout
+ if stopTimeout == 0 {
+ stopTimeout = defaultStopTimeout
+ }
+ killTimeout := h.KillTimeout
+ if killTimeout == 0 {
+ killTimeout = defaultKillTimeout
+ }
+ klock := h.Clock
+ if klock == nil {
+ klock = clock.New()
+ }
+
+ ss := &server{
+ stopTimeout: stopTimeout,
+ killTimeout: killTimeout,
+ stats: h.Stats,
+ clock: klock,
+ oldConnState: s.ConnState,
+ listener: l,
+ server: s,
+ serveDone: make(chan struct{}),
+ serveErr: make(chan error, 1),
+ new: make(chan net.Conn),
+ active: make(chan net.Conn),
+ idle: make(chan net.Conn),
+ closed: make(chan net.Conn),
+ stop: make(chan chan struct{}),
+ kill: make(chan chan struct{}),
+ certFile: h.CertFile,
+ keyFile: h.KeyFile,
+ }
+ s.ConnState = ss.connState
+ go ss.manage()
+ go ss.serve()
+ return ss
+}
+
+// ListenAndServe returns a Server for the given http.Server. It is equivalent
+// to ListenAndServe from the standard library, but returns immediately.
+// Requests will be accepted in a background goroutine. If the http.Server has
+// a non-nil TLSConfig, a TLS enabled listener will be setup.
+func (h HTTP) ListenAndServe(s *http.Server) (Server, error) {
+ addr := s.Addr
+ if addr == "" {
+ if s.TLSConfig == nil {
+ addr = ":http"
+ } else {
+ addr = ":https"
+ }
+ }
+ l, err := net.Listen("tcp", addr)
+ if err != nil {
+ stats.BumpSum(h.Stats, "listen.error", 1)
+ return nil, err
+ }
+ if s.TLSConfig != nil {
+ l = tls.NewListener(l, s.TLSConfig)
+ }
+ return h.Serve(s, l), nil
+}
+
+// server manages the serving process and allows for gracefully stopping it.
+type server struct {
+ stopTimeout time.Duration
+ killTimeout time.Duration
+ stats stats.Client
+ clock clock.Clock
+
+ oldConnState func(net.Conn, http.ConnState)
+ server *http.Server
+ serveDone chan struct{}
+ serveErr chan error
+ listener net.Listener
+
+ new chan net.Conn
+ active chan net.Conn
+ idle chan net.Conn
+ closed chan net.Conn
+ stop chan chan struct{}
+ kill chan chan struct{}
+
+ stopOnce sync.Once
+ stopErr error
+
+ certFile, keyFile string
+}
+
+func (s *server) connState(c net.Conn, cs http.ConnState) {
+ if s.oldConnState != nil {
+ s.oldConnState(c, cs)
+ }
+
+ switch cs {
+ case http.StateNew:
+ s.new <- c
+ case http.StateActive:
+ s.active <- c
+ case http.StateIdle:
+ s.idle <- c
+ case http.StateHijacked, http.StateClosed:
+ s.closed <- c
+ }
+}
+
+func (s *server) manage() {
+ defer func() {
+ close(s.new)
+ close(s.active)
+ close(s.idle)
+ close(s.closed)
+ close(s.stop)
+ close(s.kill)
+ }()
+
+ var stopDone chan struct{}
+
+ conns := map[net.Conn]http.ConnState{}
+ var countNew, countActive, countIdle float64
+
+ // decConn decrements the count associated with the current state of the
+ // given connection.
+ decConn := func(c net.Conn) {
+ switch conns[c] {
+ default:
+ panic(fmt.Errorf("unknown existing connection: %s", c))
+ case http.StateNew:
+ countNew--
+ case http.StateActive:
+ countActive--
+ case http.StateIdle:
+ countIdle--
+ }
+ }
+
+ // setup a ticker to report various values every minute. if we don't have a
+ // Stats implementation provided, we Stop it so it never ticks.
+ statsTicker := s.clock.Ticker(time.Minute)
+ if s.stats == nil {
+ statsTicker.Stop()
+ }
+
+ for {
+ select {
+ case <-statsTicker.C:
+ // we'll only get here when s.stats is not nil
+ s.stats.BumpAvg("http-state.new", countNew)
+ s.stats.BumpAvg("http-state.active", countActive)
+ s.stats.BumpAvg("http-state.idle", countIdle)
+ s.stats.BumpAvg("http-state.total", countNew+countActive+countIdle)
+ case c := <-s.new:
+ conns[c] = http.StateNew
+ countNew++
+ case c := <-s.active:
+ decConn(c)
+ countActive++
+
+ conns[c] = http.StateActive
+ case c := <-s.idle:
+ decConn(c)
+ countIdle++
+
+ conns[c] = http.StateIdle
+
+ // if we're already stopping, close it
+ if stopDone != nil {
+ c.Close()
+ }
+ case c := <-s.closed:
+ stats.BumpSum(s.stats, "conn.closed", 1)
+ decConn(c)
+ delete(conns, c)
+
+ // if we're waiting to stop and are all empty, we just closed the last
+ // connection and we're done.
+ if stopDone != nil && len(conns) == 0 {
+ close(stopDone)
+ return
+ }
+ case stopDone = <-s.stop:
+ // if we're already all empty, we're already done
+ if len(conns) == 0 {
+ close(stopDone)
+ return
+ }
+
+ // close current idle connections right away
+ for c, cs := range conns {
+ if cs == http.StateIdle {
+ c.Close()
+ }
+ }
+
+ // continue the loop and wait for all the ConnState updates which will
+ // eventually close(stopDone) and return from this goroutine.
+
+ case killDone := <-s.kill:
+ // force close all connections
+ stats.BumpSum(s.stats, "kill.conn.count", float64(len(conns)))
+ for c := range conns {
+ c.Close()
+ }
+
+ // don't block the kill.
+ close(killDone)
+
+ // continue the loop and we wait for all the ConnState updates and will
+ // return from this goroutine when we're all done. otherwise we'll try to
+ // send those ConnState updates on closed channels.
+
+ }
+ }
+}
+
+func (s *server) serve() {
+ stats.BumpSum(s.stats, "serve", 1)
+ if s.certFile == "" && s.keyFile == "" {
+ s.serveErr <- s.server.Serve(s.listener)
+ } else {
+ s.serveErr <- s.server.ServeTLS(s.listener, s.certFile, s.keyFile)
+ }
+ close(s.serveDone)
+ close(s.serveErr)
+}
+
+func (s *server) Wait() error {
+ if err := <-s.serveErr; !isUseOfClosedError(err) {
+ return err
+ }
+ return nil
+}
+
+func (s *server) Stop() error {
+ s.stopOnce.Do(func() {
+ defer stats.BumpTime(s.stats, "stop.time").End()
+ stats.BumpSum(s.stats, "stop", 1)
+
+ // first disable keep-alive for new connections
+ s.server.SetKeepAlivesEnabled(false)
+
+ // then close the listener so new connections can't connect come thru
+ closeErr := s.listener.Close()
+ <-s.serveDone
+
+ // then trigger the background goroutine to stop and wait for it
+ stopDone := make(chan struct{})
+ s.stop <- stopDone
+
+ // wait for stop
+ select {
+ case <-stopDone:
+ case <-s.clock.After(s.stopTimeout):
+ defer stats.BumpTime(s.stats, "kill.time").End()
+ stats.BumpSum(s.stats, "kill", 1)
+
+ // stop timed out, wait for kill
+ killDone := make(chan struct{})
+ s.kill <- killDone
+ select {
+ case <-killDone:
+ case <-s.clock.After(s.killTimeout):
+ // kill timed out, give up
+ stats.BumpSum(s.stats, "kill.timeout", 1)
+ }
+ }
+
+ if closeErr != nil && !isUseOfClosedError(closeErr) {
+ stats.BumpSum(s.stats, "listener.close.error", 1)
+ s.stopErr = closeErr
+ }
+ })
+ return s.stopErr
+}
+
+func isUseOfClosedError(err error) bool {
+ if err == nil {
+ return false
+ }
+ if opErr, ok := err.(*net.OpError); ok {
+ err = opErr.Err
+ }
+ return err.Error() == "use of closed network connection"
+}
+
+// ListenAndServe is a convenience function to serve and wait for a SIGTERM
+// or SIGINT before shutting down.
+func ListenAndServe(s *http.Server, hd *HTTP) error {
+ if hd == nil {
+ hd = &HTTP{}
+ }
+ hs, err := hd.ListenAndServe(s)
+ if err != nil {
+ return err
+ }
+
+ waiterr := make(chan error, 1)
+ go func() {
+ defer close(waiterr)
+ waiterr <- hs.Wait()
+ }()
+
+ signals := make(chan os.Signal, 10)
+ signal.Notify(signals, syscall.SIGTERM, syscall.SIGINT)
+
+ select {
+ case err := <-waiterr:
+ if err != nil {
+ return err
+ }
+ case <-signals:
+ signal.Stop(signals)
+ if err := hs.Stop(); err != nil {
+ return err
+ }
+ if err := <-waiterr; err != nil {
+ return err
+ }
+ }
+ return nil
+}
diff --git a/weed/util/net_timeout.go b/weed/util/net_timeout.go
index b8068e67f..8acd50d42 100644
--- a/weed/util/net_timeout.go
+++ b/weed/util/net_timeout.go
@@ -66,11 +66,8 @@ func (c *Conn) Write(b []byte) (count int, e error) {
}
func (c *Conn) Close() error {
- err := c.Conn.Close()
- if err == nil {
- stats.ConnectionClose()
- }
- return err
+ stats.ConnectionClose()
+ return c.Conn.Close()
}
func NewListener(addr string, timeout time.Duration) (net.Listener, error) {
diff --git a/weed/util/queue.go b/weed/util/queue.go
new file mode 100644
index 000000000..1e6211e0d
--- /dev/null
+++ b/weed/util/queue.go
@@ -0,0 +1,61 @@
+package util
+
+import "sync"
+
+type node struct {
+ data interface{}
+ next *node
+}
+
+type Queue struct {
+ head *node
+ tail *node
+ count int
+ sync.RWMutex
+}
+
+func NewQueue() *Queue {
+ q := &Queue{}
+ return q
+}
+
+func (q *Queue) Len() int {
+ q.RLock()
+ defer q.RUnlock()
+ return q.count
+}
+
+func (q *Queue) Enqueue(item interface{}) {
+ q.Lock()
+ defer q.Unlock()
+
+ n := &node{data: item}
+
+ if q.tail == nil {
+ q.tail = n
+ q.head = n
+ } else {
+ q.tail.next = n
+ q.tail = n
+ }
+ q.count++
+}
+
+func (q *Queue) Dequeue() interface{} {
+ q.Lock()
+ defer q.Unlock()
+
+ if q.head == nil {
+ return nil
+ }
+
+ n := q.head
+ q.head = n.next
+
+ if q.head == nil {
+ q.tail = nil
+ }
+ q.count--
+
+ return n.data
+}
diff --git a/weed/util/queue_unbounded.go b/weed/util/queue_unbounded.go
new file mode 100644
index 000000000..664cd965e
--- /dev/null
+++ b/weed/util/queue_unbounded.go
@@ -0,0 +1,45 @@
+package util
+
+import "sync"
+
+type UnboundedQueue struct {
+ outbound []string
+ outboundLock sync.RWMutex
+ inbound []string
+ inboundLock sync.RWMutex
+}
+
+func NewUnboundedQueue() *UnboundedQueue {
+ q := &UnboundedQueue{}
+ return q
+}
+
+func (q *UnboundedQueue) EnQueue(items ...string) {
+ q.inboundLock.Lock()
+ defer q.inboundLock.Unlock()
+
+ q.outbound = append(q.outbound, items...)
+
+}
+
+func (q *UnboundedQueue) Consume(fn func([]string)) {
+ q.outboundLock.Lock()
+ defer q.outboundLock.Unlock()
+
+ if len(q.outbound) == 0 {
+ q.inboundLock.Lock()
+ inbountLen := len(q.inbound)
+ if inbountLen > 0 {
+ t := q.outbound
+ q.outbound = q.inbound
+ q.inbound = t
+ }
+ q.inboundLock.Unlock()
+ }
+
+ if len(q.outbound) > 0 {
+ fn(q.outbound)
+ q.outbound = q.outbound[:0]
+ }
+
+}
diff --git a/weed/util/queue_unbounded_test.go b/weed/util/queue_unbounded_test.go
new file mode 100644
index 000000000..2d02032cb
--- /dev/null
+++ b/weed/util/queue_unbounded_test.go
@@ -0,0 +1,25 @@
+package util
+
+import "testing"
+
+func TestEnqueueAndConsume(t *testing.T) {
+
+ q := NewUnboundedQueue()
+
+ q.EnQueue("1", "2", "3")
+
+ f := func(items []string) {
+ for _, t := range items {
+ println(t)
+ }
+ println("-----------------------")
+ }
+ q.Consume(f)
+
+ q.Consume(f)
+
+ q.EnQueue("4", "5")
+ q.EnQueue("6", "7")
+ q.Consume(f)
+
+}
diff --git a/weed/wdclient/masterclient.go b/weed/wdclient/masterclient.go
index 6ba668ade..301f20615 100644
--- a/weed/wdclient/masterclient.go
+++ b/weed/wdclient/masterclient.go
@@ -2,19 +2,19 @@ package wdclient
import (
"context"
- "fmt"
"math/rand"
"time"
+ "google.golang.org/grpc"
+
"github.com/chrislusf/seaweedfs/weed/glog"
+ "github.com/chrislusf/seaweedfs/weed/pb"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
- "github.com/chrislusf/seaweedfs/weed/util"
- "google.golang.org/grpc"
)
type MasterClient struct {
- ctx context.Context
name string
+ grpcPort uint32
currentMaster string
masters []string
grpcDialOption grpc.DialOption
@@ -22,10 +22,10 @@ type MasterClient struct {
vidMap
}
-func NewMasterClient(ctx context.Context, grpcDialOption grpc.DialOption, clientName string, masters []string) *MasterClient {
+func NewMasterClient(grpcDialOption grpc.DialOption, clientName string, clientGrpcPort uint32, masters []string) *MasterClient {
return &MasterClient{
- ctx: ctx,
name: clientName,
+ grpcPort: clientGrpcPort,
masters: masters,
grpcDialOption: grpcDialOption,
vidMap: newVidMap(),
@@ -66,15 +66,15 @@ func (mc *MasterClient) tryAllMasters() {
func (mc *MasterClient) tryConnectToMaster(master string) (nextHintedLeader string) {
glog.V(1).Infof("%s Connecting to master %v", mc.name, master)
- gprcErr := withMasterClient(context.Background(), master, mc.grpcDialOption, func(ctx context.Context, client master_pb.SeaweedClient) error {
+ gprcErr := pb.WithMasterClient(master, mc.grpcDialOption, func(client master_pb.SeaweedClient) error {
- stream, err := client.KeepConnected(ctx)
+ stream, err := client.KeepConnected(context.Background())
if err != nil {
glog.V(0).Infof("%s failed to keep connected to %s: %v", mc.name, master, err)
return err
}
- if err = stream.Send(&master_pb.KeepConnectedRequest{Name: mc.name}); err != nil {
+ if err = stream.Send(&master_pb.KeepConnectedRequest{Name: mc.name, GrpcPort: mc.grpcPort}); err != nil {
glog.V(0).Infof("%s failed to send to %s: %v", mc.name, master, err)
return err
}
@@ -91,7 +91,7 @@ func (mc *MasterClient) tryConnectToMaster(master string) (nextHintedLeader stri
// maybe the leader is changed
if volumeLocation.Leader != "" {
- glog.V(1).Infof("redirected to leader %v", volumeLocation.Leader)
+ glog.V(0).Infof("redirected to leader %v", volumeLocation.Leader)
nextHintedLeader = volumeLocation.Leader
return nil
}
@@ -118,22 +118,8 @@ func (mc *MasterClient) tryConnectToMaster(master string) (nextHintedLeader stri
return
}
-func withMasterClient(ctx context.Context, master string, grpcDialOption grpc.DialOption, fn func(ctx context.Context, client master_pb.SeaweedClient) error) error {
-
- masterGrpcAddress, parseErr := util.ParseServerToGrpcAddress(master)
- if parseErr != nil {
- return fmt.Errorf("failed to parse master grpc %v: %v", master, parseErr)
- }
-
- return util.WithCachedGrpcClient(ctx, func(grpcConnection *grpc.ClientConn) error {
- client := master_pb.NewSeaweedClient(grpcConnection)
- return fn(ctx, client)
- }, masterGrpcAddress, grpcDialOption)
-
-}
-
-func (mc *MasterClient) WithClient(ctx context.Context, fn func(client master_pb.SeaweedClient) error) error {
- return withMasterClient(ctx, mc.currentMaster, mc.grpcDialOption, func(ctx context.Context, client master_pb.SeaweedClient) error {
+func (mc *MasterClient) WithClient(fn func(client master_pb.SeaweedClient) error) error {
+ return pb.WithMasterClient(mc.currentMaster, mc.grpcDialOption, func(client master_pb.SeaweedClient) error {
return fn(client)
})
}