Merge remote-tracking branch 'origin/master'

This commit is contained in:
bingoohuang
2021-02-18 14:05:51 +08:00
147 changed files with 4359 additions and 2928 deletions

View File

@@ -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

View File

@@ -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

View File

@@ -14,6 +14,7 @@ var (
func init() {
client = &http.Client{Transport: &http.Transport{
MaxIdleConns: 1024,
MaxIdleConnsPerHost: 1024,
}}
}

View File

@@ -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)
}

View File

@@ -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),

View File

@@ -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),
},

View File

@@ -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{

View File

@@ -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),

View File

@@ -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)

View File

@@ -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"),

View File

@@ -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 {

View File

@@ -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 {

View File

@@ -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
}

View File

@@ -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")
}

View File

@@ -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)

View File

@@ -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

View File

@@ -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))

View File

@@ -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()

View File

@@ -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)

View File

@@ -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>

View File

@@ -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,
}