mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2025-10-21 21:09:24 +08:00
Merge remote-tracking branch 'origin/master'
This commit is contained in:
@@ -100,7 +100,7 @@ func debug(params ...interface{}) {
|
||||
glog.V(4).Infoln(params...)
|
||||
}
|
||||
|
||||
func submitForClientHandler(w http.ResponseWriter, r *http.Request, masterUrl string, grpcDialOption grpc.DialOption) {
|
||||
func submitForClientHandler(w http.ResponseWriter, r *http.Request, masterFn operation.GetMasterFn, grpcDialOption grpc.DialOption) {
|
||||
m := make(map[string]interface{})
|
||||
if r.Method != "POST" {
|
||||
writeJsonError(w, r, http.StatusMethodNotAllowed, errors.New("Only submit via POST!"))
|
||||
@@ -131,8 +131,9 @@ func submitForClientHandler(w http.ResponseWriter, r *http.Request, masterUrl st
|
||||
Replication: r.FormValue("replication"),
|
||||
Collection: r.FormValue("collection"),
|
||||
Ttl: r.FormValue("ttl"),
|
||||
DiskType: r.FormValue("disk"),
|
||||
}
|
||||
assignResult, ae := operation.Assign(masterUrl, grpcDialOption, ar)
|
||||
assignResult, ae := operation.Assign(masterFn, grpcDialOption, ar)
|
||||
if ae != nil {
|
||||
writeJsonError(w, r, http.StatusInternalServerError, ae)
|
||||
return
|
||||
|
@@ -263,6 +263,7 @@ func (fs *FilerServer) cleanupChunks(fullpath string, existingEntry *filer.Entry
|
||||
newEntry.Attributes.Collection,
|
||||
newEntry.Attributes.Replication,
|
||||
newEntry.Attributes.TtlSec,
|
||||
newEntry.Attributes.DiskType,
|
||||
"",
|
||||
"",
|
||||
)
|
||||
@@ -306,7 +307,7 @@ func (fs *FilerServer) AppendToEntry(ctx context.Context, req *filer_pb.AppendTo
|
||||
}
|
||||
|
||||
entry.Chunks = append(entry.Chunks, req.Chunks...)
|
||||
so := fs.detectStorageOption(string(fullpath), entry.Collection, entry.Replication, entry.TtlSec, "", "")
|
||||
so := fs.detectStorageOption(string(fullpath), entry.Collection, entry.Replication, entry.TtlSec, entry.DiskType, "", "")
|
||||
entry.Chunks, err = filer.MaybeManifestize(fs.saveAsChunk(so), entry.Chunks)
|
||||
if err != nil {
|
||||
// not good, but should be ok
|
||||
@@ -332,11 +333,11 @@ func (fs *FilerServer) DeleteEntry(ctx context.Context, req *filer_pb.DeleteEntr
|
||||
|
||||
func (fs *FilerServer) AssignVolume(ctx context.Context, req *filer_pb.AssignVolumeRequest) (resp *filer_pb.AssignVolumeResponse, err error) {
|
||||
|
||||
so := fs.detectStorageOption(req.Path, req.Collection, req.Replication, req.TtlSec, req.DataCenter, req.Rack)
|
||||
so := fs.detectStorageOption(req.Path, req.Collection, req.Replication, req.TtlSec, req.DiskType, req.DataCenter, req.Rack)
|
||||
|
||||
assignRequest, altRequest := so.ToAssignRequests(int(req.Count))
|
||||
|
||||
assignResult, err := operation.Assign(fs.filer.GetMaster(), fs.grpcDialOption, assignRequest, altRequest)
|
||||
assignResult, err := operation.Assign(fs.filer.GetMaster, fs.grpcDialOption, assignRequest, altRequest)
|
||||
if err != nil {
|
||||
glog.V(3).Infof("AssignVolume: %v", err)
|
||||
return &filer_pb.AssignVolumeResponse{Error: fmt.Sprintf("assign volume: %v", err)}, nil
|
||||
@@ -402,6 +403,7 @@ func (fs *FilerServer) Statistics(ctx context.Context, req *filer_pb.StatisticsR
|
||||
Replication: req.Replication,
|
||||
Collection: req.Collection,
|
||||
Ttl: req.Ttl,
|
||||
DiskType: req.DiskType,
|
||||
})
|
||||
if grpcErr != nil {
|
||||
return grpcErr
|
||||
|
@@ -14,6 +14,7 @@ var (
|
||||
|
||||
func init() {
|
||||
client = &http.Client{Transport: &http.Transport{
|
||||
MaxIdleConns: 1024,
|
||||
MaxIdleConnsPerHost: 1024,
|
||||
}}
|
||||
}
|
||||
|
@@ -37,7 +37,7 @@ func (fs *FilerServer) assignNewFileInfo(so *operation.StorageOption) (fileId, u
|
||||
|
||||
ar, altRequest := so.ToAssignRequests(1)
|
||||
|
||||
assignResult, ae := operation.Assign(fs.filer.GetMaster(), fs.grpcDialOption, ar, altRequest)
|
||||
assignResult, ae := operation.Assign(fs.filer.GetMaster, fs.grpcDialOption, ar, altRequest)
|
||||
if ae != nil {
|
||||
glog.Errorf("failing to assign a file id: %v", ae)
|
||||
err = ae
|
||||
@@ -61,6 +61,7 @@ func (fs *FilerServer) PostHandler(w http.ResponseWriter, r *http.Request) {
|
||||
query.Get("collection"),
|
||||
query.Get("replication"),
|
||||
query.Get("ttl"),
|
||||
query.Get("disk"),
|
||||
query.Get("dataCenter"),
|
||||
query.Get("rack"),
|
||||
)
|
||||
@@ -104,7 +105,7 @@ func (fs *FilerServer) DeleteHandler(w http.ResponseWriter, r *http.Request) {
|
||||
w.WriteHeader(http.StatusNoContent)
|
||||
}
|
||||
|
||||
func (fs *FilerServer) detectStorageOption(requestURI, qCollection, qReplication string, ttlSeconds int32, dataCenter, rack string) *operation.StorageOption {
|
||||
func (fs *FilerServer) detectStorageOption(requestURI, qCollection, qReplication string, ttlSeconds int32, diskType string, dataCenter, rack string) *operation.StorageOption {
|
||||
collection := util.Nvl(qCollection, fs.option.Collection)
|
||||
replication := util.Nvl(qReplication, fs.option.DefaultReplication)
|
||||
|
||||
@@ -134,17 +135,18 @@ func (fs *FilerServer) detectStorageOption(requestURI, qCollection, qReplication
|
||||
DataCenter: util.Nvl(dataCenter, fs.option.DataCenter),
|
||||
Rack: util.Nvl(rack, fs.option.Rack),
|
||||
TtlSeconds: ttlSeconds,
|
||||
DiskType: util.Nvl(diskType, rule.DiskType),
|
||||
Fsync: fsync || rule.Fsync,
|
||||
VolumeGrowthCount: rule.VolumeGrowthCount,
|
||||
}
|
||||
}
|
||||
|
||||
func (fs *FilerServer) detectStorageOption0(requestURI, qCollection, qReplication string, qTtl string, dataCenter, rack string) *operation.StorageOption {
|
||||
func (fs *FilerServer) detectStorageOption0(requestURI, qCollection, qReplication string, qTtl string, diskType string, dataCenter, rack string) *operation.StorageOption {
|
||||
|
||||
ttl, err := needle.ReadTTL(qTtl)
|
||||
if err != nil {
|
||||
glog.Errorf("fail to parse ttl %s: %v", qTtl, err)
|
||||
}
|
||||
|
||||
return fs.detectStorageOption(requestURI, qCollection, qReplication, int32(ttl.Minutes())*60, dataCenter, rack)
|
||||
return fs.detectStorageOption(requestURI, qCollection, qReplication, int32(ttl.Minutes())*60, diskType, dataCenter, rack)
|
||||
}
|
||||
|
@@ -104,7 +104,7 @@ func (fs *FilerServer) doPostAutoChunk(ctx context.Context, w http.ResponseWrite
|
||||
|
||||
func (fs *FilerServer) doPutAutoChunk(ctx context.Context, w http.ResponseWriter, r *http.Request, chunkSize int32, so *operation.StorageOption) (filerResult *FilerPostResult, md5bytes []byte, replyerr error) {
|
||||
|
||||
fileName := ""
|
||||
fileName := path.Base(r.URL.Path)
|
||||
contentType := r.Header.Get("Content-Type")
|
||||
if contentType == "application/octet-stream" {
|
||||
contentType = ""
|
||||
@@ -186,6 +186,7 @@ func (fs *FilerServer) saveMetaData(ctx context.Context, r *http.Request, fileNa
|
||||
Replication: so.Replication,
|
||||
Collection: so.Collection,
|
||||
TtlSec: so.TtlSeconds,
|
||||
DiskType: so.DiskType,
|
||||
Mime: contentType,
|
||||
Md5: md5bytes,
|
||||
FileSize: uint64(chunkOffset),
|
||||
|
@@ -68,6 +68,7 @@ func (fs *FilerServer) encrypt(ctx context.Context, w http.ResponseWriter, r *ht
|
||||
Replication: so.Replication,
|
||||
Collection: so.Collection,
|
||||
TtlSec: so.TtlSeconds,
|
||||
DiskType: so.DiskType,
|
||||
Mime: pu.MimeType,
|
||||
Md5: util.Base64Md5ToBytes(pu.ContentMd5),
|
||||
},
|
||||
|
@@ -67,9 +67,7 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
|
||||
dcName, rackName := ms.Topo.Configuration.Locate(heartbeat.Ip, heartbeat.DataCenter, heartbeat.Rack)
|
||||
dc := ms.Topo.GetOrCreateDataCenter(dcName)
|
||||
rack := dc.GetOrCreateRack(rackName)
|
||||
dn = rack.GetOrCreateDataNode(heartbeat.Ip,
|
||||
int(heartbeat.Port), heartbeat.PublicUrl,
|
||||
int64(heartbeat.MaxVolumeCount))
|
||||
dn = rack.GetOrCreateDataNode(heartbeat.Ip, int(heartbeat.Port), heartbeat.PublicUrl, heartbeat.MaxVolumeCounts)
|
||||
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,
|
||||
@@ -79,10 +77,7 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
|
||||
}
|
||||
}
|
||||
|
||||
if heartbeat.MaxVolumeCount != 0 && dn.GetMaxVolumeCount() != int64(heartbeat.MaxVolumeCount) {
|
||||
delta := int64(heartbeat.MaxVolumeCount) - dn.GetMaxVolumeCount()
|
||||
dn.UpAdjustMaxVolumeCountDelta(delta)
|
||||
}
|
||||
dn.AdjustMaxVolumeCounts(heartbeat.MaxVolumeCounts)
|
||||
|
||||
glog.V(4).Infof("master received heartbeat %s", heartbeat.String())
|
||||
message := &master_pb.VolumeLocation{
|
||||
|
@@ -4,6 +4,7 @@ import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/chrislusf/raft"
|
||||
"github.com/chrislusf/seaweedfs/weed/storage/types"
|
||||
|
||||
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
|
||||
"github.com/chrislusf/seaweedfs/weed/security"
|
||||
@@ -60,11 +61,13 @@ func (ms *MasterServer) Assign(ctx context.Context, req *master_pb.AssignRequest
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
diskType := types.ToDiskType(req.DiskType)
|
||||
|
||||
option := &topology.VolumeGrowOption{
|
||||
Collection: req.Collection,
|
||||
ReplicaPlacement: replicaPlacement,
|
||||
Ttl: ttl,
|
||||
DiskType: diskType,
|
||||
Prealloacte: ms.preallocateSize,
|
||||
DataCenter: req.DataCenter,
|
||||
Rack: req.Rack,
|
||||
@@ -73,7 +76,7 @@ func (ms *MasterServer) Assign(ctx context.Context, req *master_pb.AssignRequest
|
||||
}
|
||||
|
||||
if !ms.Topo.HasWritableVolume(option) {
|
||||
if ms.Topo.FreeSpace() <= 0 {
|
||||
if ms.Topo.AvailableSpaceFor(option) <= 0 {
|
||||
return nil, fmt.Errorf("No free volumes left!")
|
||||
}
|
||||
ms.vgLock.Lock()
|
||||
@@ -117,10 +120,10 @@ func (ms *MasterServer) Statistics(ctx context.Context, req *master_pb.Statistic
|
||||
return nil, err
|
||||
}
|
||||
|
||||
volumeLayout := ms.Topo.GetVolumeLayout(req.Collection, replicaPlacement, ttl)
|
||||
volumeLayout := ms.Topo.GetVolumeLayout(req.Collection, replicaPlacement, ttl, types.ToDiskType(req.DiskType))
|
||||
stats := volumeLayout.Stats()
|
||||
|
||||
totalSize := ms.Topo.GetMaxVolumeCount() * int64(ms.option.VolumeSizeLimitMB) * 1024 * 1024
|
||||
totalSize := ms.Topo.GetDiskUsages().GetMaxVolumeCount() * int64(ms.option.VolumeSizeLimitMB) * 1024 * 1024
|
||||
|
||||
resp := &master_pb.StatisticsResponse{
|
||||
TotalSize: uint64(totalSize),
|
||||
|
@@ -112,7 +112,7 @@ func (ms *MasterServer) dirAssignHandler(w http.ResponseWriter, r *http.Request)
|
||||
}
|
||||
|
||||
if !ms.Topo.HasWritableVolume(option) {
|
||||
if ms.Topo.FreeSpace() <= 0 {
|
||||
if ms.Topo.AvailableSpaceFor(option) <= 0 {
|
||||
writeJsonQuiet(w, r, http.StatusNotFound, operation.AssignResult{Error: "No free volumes left!"})
|
||||
return
|
||||
}
|
||||
@@ -136,6 +136,9 @@ func (ms *MasterServer) dirAssignHandler(w http.ResponseWriter, r *http.Request)
|
||||
}
|
||||
|
||||
func (ms *MasterServer) maybeAddJwtAuthorization(w http.ResponseWriter, fileId string, isWrite bool) {
|
||||
if fileId == "" {
|
||||
return
|
||||
}
|
||||
var encodedJwt security.EncodedJwt
|
||||
if isWrite {
|
||||
encodedJwt = security.GenJwt(ms.guard.SigningKey, ms.guard.ExpiresAfterSec, fileId)
|
||||
|
@@ -3,6 +3,7 @@ package weed_server
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/storage/types"
|
||||
"math/rand"
|
||||
"net/http"
|
||||
"strconv"
|
||||
@@ -75,8 +76,8 @@ func (ms *MasterServer) volumeGrowHandler(w http.ResponseWriter, r *http.Request
|
||||
}
|
||||
|
||||
if count, err = strconv.Atoi(r.FormValue("count")); err == nil {
|
||||
if ms.Topo.FreeSpace() < int64(count*option.ReplicaPlacement.GetCopyCount()) {
|
||||
err = fmt.Errorf("only %d volumes left, not enough for %d", ms.Topo.FreeSpace(), count*option.ReplicaPlacement.GetCopyCount())
|
||||
if ms.Topo.AvailableSpaceFor(option) < int64(count*option.ReplicaPlacement.GetCopyCount()) {
|
||||
err = fmt.Errorf("only %d volumes left, not enough for %d", ms.Topo.AvailableSpaceFor(option), count*option.ReplicaPlacement.GetCopyCount())
|
||||
} else {
|
||||
count, err = ms.vg.GrowByCountAndType(ms.grpcDialOption, count, option, ms.Topo)
|
||||
}
|
||||
@@ -124,19 +125,19 @@ func (ms *MasterServer) selfUrl(r *http.Request) string {
|
||||
}
|
||||
func (ms *MasterServer) submitFromMasterServerHandler(w http.ResponseWriter, r *http.Request) {
|
||||
if ms.Topo.IsLeader() {
|
||||
submitForClientHandler(w, r, ms.selfUrl(r), ms.grpcDialOption)
|
||||
submitForClientHandler(w, r, func() string { return ms.selfUrl(r) }, ms.grpcDialOption)
|
||||
} else {
|
||||
masterUrl, err := ms.Topo.Leader()
|
||||
if err != nil {
|
||||
writeJsonError(w, r, http.StatusInternalServerError, err)
|
||||
} else {
|
||||
submitForClientHandler(w, r, masterUrl, ms.grpcDialOption)
|
||||
submitForClientHandler(w, r, func() string { return masterUrl }, ms.grpcDialOption)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *MasterServer) HasWritableVolume(option *topology.VolumeGrowOption) bool {
|
||||
vl := ms.Topo.GetVolumeLayout(option.Collection, option.ReplicaPlacement, option.Ttl)
|
||||
vl := ms.Topo.GetVolumeLayout(option.Collection, option.ReplicaPlacement, option.Ttl, option.DiskType)
|
||||
return vl.GetActiveVolumeCount(option) > 0
|
||||
}
|
||||
|
||||
@@ -157,6 +158,7 @@ func (ms *MasterServer) getVolumeGrowOption(r *http.Request) (*topology.VolumeGr
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
diskType := types.ToDiskType(r.FormValue("disk"))
|
||||
|
||||
preallocate := ms.preallocateSize
|
||||
if r.FormValue("preallocate") != "" {
|
||||
@@ -169,6 +171,7 @@ func (ms *MasterServer) getVolumeGrowOption(r *http.Request) (*topology.VolumeGr
|
||||
Collection: r.FormValue("collection"),
|
||||
ReplicaPlacement: replicaPlacement,
|
||||
Ttl: ttl,
|
||||
DiskType: diskType,
|
||||
Prealloacte: preallocate,
|
||||
DataCenter: r.FormValue("dataCenter"),
|
||||
Rack: r.FormValue("rack"),
|
||||
|
@@ -41,6 +41,7 @@ func (vs *VolumeServer) AllocateVolume(ctx context.Context, req *volume_server_p
|
||||
req.Ttl,
|
||||
req.Preallocate,
|
||||
req.MemoryMapMaxSizeMb,
|
||||
types.ToDiskType(req.DiskType),
|
||||
)
|
||||
|
||||
if err != nil {
|
||||
|
@@ -219,15 +219,14 @@ func (vs *VolumeServer) doHeartbeat(masterNode, masterGrpcAddress string, grpcDi
|
||||
case <-vs.stopChan:
|
||||
var volumeMessages []*master_pb.VolumeInformationMessage
|
||||
emptyBeat := &master_pb.Heartbeat{
|
||||
Ip: vs.store.Ip,
|
||||
Port: uint32(vs.store.Port),
|
||||
PublicUrl: vs.store.PublicUrl,
|
||||
MaxVolumeCount: uint32(0),
|
||||
MaxFileKey: uint64(0),
|
||||
DataCenter: vs.store.GetDataCenter(),
|
||||
Rack: vs.store.GetRack(),
|
||||
Volumes: volumeMessages,
|
||||
HasNoVolumes: len(volumeMessages) == 0,
|
||||
Ip: vs.store.Ip,
|
||||
Port: uint32(vs.store.Port),
|
||||
PublicUrl: vs.store.PublicUrl,
|
||||
MaxFileKey: uint64(0),
|
||||
DataCenter: vs.store.GetDataCenter(),
|
||||
Rack: vs.store.GetRack(),
|
||||
Volumes: volumeMessages,
|
||||
HasNoVolumes: len(volumeMessages) == 0,
|
||||
}
|
||||
glog.V(1).Infof("volume server %s:%d stops and deletes all volumes", vs.store.Ip, vs.store.Port)
|
||||
if err = stream.Send(emptyBeat); err != nil {
|
||||
|
@@ -3,6 +3,7 @@ package weed_server
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/storage/types"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"math"
|
||||
@@ -36,11 +37,6 @@ func (vs *VolumeServer) VolumeCopy(ctx context.Context, req *volume_server_pb.Vo
|
||||
glog.V(0).Infof("deleted existing volume %d before copying.", req.VolumeId)
|
||||
}
|
||||
|
||||
location := vs.store.FindFreeLocation()
|
||||
if location == nil {
|
||||
return nil, fmt.Errorf("no space left")
|
||||
}
|
||||
|
||||
// the master will not start compaction for read-only volumes, so it is safe to just copy files directly
|
||||
// copy .dat and .idx files
|
||||
// read .idx .dat file size and timestamp
|
||||
@@ -59,6 +55,15 @@ func (vs *VolumeServer) VolumeCopy(ctx context.Context, req *volume_server_pb.Vo
|
||||
return fmt.Errorf("read volume file status failed, %v", err)
|
||||
}
|
||||
|
||||
diskType := volFileInfoResp.DiskType
|
||||
if req.DiskType != "" {
|
||||
diskType = req.DiskType
|
||||
}
|
||||
location := vs.store.FindFreeLocation(types.ToDiskType(diskType))
|
||||
if location == nil {
|
||||
return fmt.Errorf("no space left")
|
||||
}
|
||||
|
||||
dataBaseFileName = storage.VolumeFileName(location.Directory, volFileInfoResp.Collection, int(req.VolumeId))
|
||||
indexBaseFileName = storage.VolumeFileName(location.IdxDirectory, volFileInfoResp.Collection, int(req.VolumeId))
|
||||
|
||||
@@ -206,6 +211,7 @@ func (vs *VolumeServer) ReadVolumeFileStatus(ctx context.Context, req *volume_se
|
||||
resp.FileCount = v.FileCount()
|
||||
resp.CompactionRevision = uint32(v.CompactionRevision)
|
||||
resp.Collection = v.Collection
|
||||
resp.DiskType = string(v.DiskType())
|
||||
return resp, nil
|
||||
}
|
||||
|
||||
|
@@ -105,7 +105,7 @@ func (vs *VolumeServer) VolumeEcShardsCopy(ctx context.Context, req *volume_serv
|
||||
|
||||
glog.V(0).Infof("VolumeEcShardsCopy: %v", req)
|
||||
|
||||
location := vs.store.FindFreeLocation()
|
||||
location := vs.store.FindFreeLocation(types.HardDriveType)
|
||||
if location == nil {
|
||||
return nil, fmt.Errorf("no space left")
|
||||
}
|
||||
|
@@ -2,6 +2,7 @@ package weed_server
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/chrislusf/seaweedfs/weed/storage/types"
|
||||
"net/http"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
@@ -37,7 +38,7 @@ type VolumeServer struct {
|
||||
|
||||
func NewVolumeServer(adminMux, publicMux *http.ServeMux, ip string,
|
||||
port int, publicUrl string,
|
||||
folders []string, maxCounts []int, minFreeSpacePercents []float32,
|
||||
folders []string, maxCounts []int, minFreeSpacePercents []float32, diskTypes []types.DiskType,
|
||||
idxFolder string,
|
||||
needleMapKind storage.NeedleMapKind,
|
||||
masterNodes []string, pulseSeconds int,
|
||||
@@ -76,7 +77,7 @@ func NewVolumeServer(adminMux, publicMux *http.ServeMux, ip string,
|
||||
|
||||
vs.checkWithMaster()
|
||||
|
||||
vs.store = storage.NewStore(vs.grpcDialOption, port, ip, publicUrl, folders, maxCounts, minFreeSpacePercents, idxFolder, vs.needleMapKind)
|
||||
vs.store = storage.NewStore(vs.grpcDialOption, port, ip, publicUrl, folders, maxCounts, minFreeSpacePercents, idxFolder, vs.needleMapKind, diskTypes)
|
||||
vs.guard = security.NewGuard(whiteList, signingKey, expiresAfterSec, readSigningKey, readExpiresAfterSec)
|
||||
|
||||
handleStaticResources(adminMux)
|
||||
|
@@ -16,7 +16,9 @@ func (vs *VolumeServer) statusHandler(w http.ResponseWriter, r *http.Request) {
|
||||
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))
|
||||
newDiskStatus := stats.NewDiskStatus(dir)
|
||||
newDiskStatus.DiskType = loc.DiskType.String()
|
||||
ds = append(ds, newDiskStatus)
|
||||
}
|
||||
}
|
||||
m["DiskStatuses"] = ds
|
||||
@@ -31,7 +33,9 @@ func (vs *VolumeServer) statsDiskHandler(w http.ResponseWriter, r *http.Request)
|
||||
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))
|
||||
newDiskStatus := stats.NewDiskStatus(dir)
|
||||
newDiskStatus.DiskType = loc.DiskType.String()
|
||||
ds = append(ds, newDiskStatus)
|
||||
}
|
||||
}
|
||||
m["DiskStatuses"] = ds
|
||||
|
@@ -63,7 +63,7 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
|
||||
w.WriteHeader(http.StatusNotFound)
|
||||
return
|
||||
}
|
||||
lookupResult, err := operation.Lookup(vs.GetMaster(), volumeId.String())
|
||||
lookupResult, err := operation.Lookup(vs.GetMaster, volumeId.String())
|
||||
glog.V(2).Infoln("volume", volumeId, "found on", lookupResult, "error", err)
|
||||
if err == nil && len(lookupResult.Locations) > 0 {
|
||||
u, _ := url.Parse(util.NormalizeUrl(lookupResult.Locations[0].PublicUrl))
|
||||
|
@@ -19,7 +19,9 @@ func (vs *VolumeServer) uiStatusHandler(w http.ResponseWriter, r *http.Request)
|
||||
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))
|
||||
newDiskStatus := stats.NewDiskStatus(dir)
|
||||
newDiskStatus.DiskType = loc.DiskType.String()
|
||||
ds = append(ds, newDiskStatus)
|
||||
}
|
||||
}
|
||||
volumeInfos := vs.store.VolumeInfos()
|
||||
|
@@ -50,7 +50,7 @@ func (vs *VolumeServer) PostHandler(w http.ResponseWriter, r *http.Request) {
|
||||
}
|
||||
|
||||
ret := operation.UploadResult{}
|
||||
isUnchanged, writeError := topology.ReplicatedWrite(vs.GetMaster(), vs.store, volumeId, reqNeedle, r)
|
||||
isUnchanged, writeError := topology.ReplicatedWrite(vs.GetMaster, vs.store, volumeId, reqNeedle, r)
|
||||
|
||||
// http 204 status code does not allow body
|
||||
if writeError == nil && isUnchanged {
|
||||
@@ -128,7 +128,7 @@ func (vs *VolumeServer) DeleteHandler(w http.ResponseWriter, r *http.Request) {
|
||||
return
|
||||
}
|
||||
// make sure all chunks had deleted before delete manifest
|
||||
if e := chunkManifest.DeleteChunks(vs.GetMaster(), false, vs.grpcDialOption); e != nil {
|
||||
if e := chunkManifest.DeleteChunks(vs.GetMaster, false, vs.grpcDialOption); e != nil {
|
||||
writeJsonError(w, r, http.StatusInternalServerError, fmt.Errorf("Delete chunks error: %v", e))
|
||||
return
|
||||
}
|
||||
@@ -143,7 +143,7 @@ func (vs *VolumeServer) DeleteHandler(w http.ResponseWriter, r *http.Request) {
|
||||
}
|
||||
}
|
||||
|
||||
_, err := topology.ReplicatedDelete(vs.GetMaster(), vs.store, volumeId, n, r)
|
||||
_, err := topology.ReplicatedDelete(vs.GetMaster, vs.store, volumeId, n, r)
|
||||
|
||||
writeDeleteResult(err, count, w, r)
|
||||
|
||||
|
@@ -69,6 +69,7 @@ var StatusTpl = template.Must(template.New("status").Funcs(funcMap).Parse(`<!DOC
|
||||
<thead>
|
||||
<tr>
|
||||
<th>Path</th>
|
||||
<th>Disk</th>
|
||||
<th>Total</th>
|
||||
<th>Free</th>
|
||||
<th>Usage</th>
|
||||
@@ -78,6 +79,7 @@ var StatusTpl = template.Must(template.New("status").Funcs(funcMap).Parse(`<!DOC
|
||||
{{ range .DiskStatuses }}
|
||||
<tr>
|
||||
<td>{{ .Dir }}</td>
|
||||
<td>{{ .DiskType }}</td>
|
||||
<td>{{ bytesToHumanReadable .All }}</td>
|
||||
<td>{{ bytesToHumanReadable .Free }}</td>
|
||||
<td>{{ percentFrom .All .Used}}%</td>
|
||||
@@ -127,6 +129,7 @@ var StatusTpl = template.Must(template.New("status").Funcs(funcMap).Parse(`<!DOC
|
||||
<tr>
|
||||
<th>Id</th>
|
||||
<th>Collection</th>
|
||||
<th>Disk</th>
|
||||
<th>Data Size</th>
|
||||
<th>Files</th>
|
||||
<th>Trash</th>
|
||||
@@ -139,6 +142,7 @@ var StatusTpl = template.Must(template.New("status").Funcs(funcMap).Parse(`<!DOC
|
||||
<tr>
|
||||
<td><code>{{ .Id }}</code></td>
|
||||
<td>{{ .Collection }}</td>
|
||||
<td>{{ .DiskType }}</td>
|
||||
<td>{{ bytesToHumanReadable .Size }}</td>
|
||||
<td>{{ .FileCount }}</td>
|
||||
<td>{{ .DeleteCount }} / {{bytesToHumanReadable .DeletedByteCount}}</td>
|
||||
|
@@ -33,6 +33,7 @@ type WebDavOption struct {
|
||||
BucketsPath string
|
||||
GrpcDialOption grpc.DialOption
|
||||
Collection string
|
||||
DiskType string
|
||||
Uid uint32
|
||||
Gid uint32
|
||||
Cipher bool
|
||||
@@ -382,6 +383,7 @@ func (f *WebDavFile) saveDataAsChunk(reader io.Reader, name string, offset int64
|
||||
Count: 1,
|
||||
Replication: "",
|
||||
Collection: f.fs.option.Collection,
|
||||
DiskType: f.fs.option.DiskType,
|
||||
Path: name,
|
||||
}
|
||||
|
||||
|
Reference in New Issue
Block a user