You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

483 lines
14 KiB

7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
5 years ago
5 years ago
5 years ago
4 years ago
5 years ago
5 years ago
5 years ago
4 years ago
5 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
4 years ago
6 years ago
5 years ago
4 years ago
5 years ago
  1. package weed_server
  2. import (
  3. "context"
  4. "fmt"
  5. "github.com/chrislusf/seaweedfs/weed/pb"
  6. "os"
  7. "path/filepath"
  8. "strconv"
  9. "time"
  10. "github.com/chrislusf/seaweedfs/weed/filer"
  11. "github.com/chrislusf/seaweedfs/weed/glog"
  12. "github.com/chrislusf/seaweedfs/weed/operation"
  13. "github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
  14. "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
  15. "github.com/chrislusf/seaweedfs/weed/storage/needle"
  16. "github.com/chrislusf/seaweedfs/weed/util"
  17. )
  18. func (fs *FilerServer) LookupDirectoryEntry(ctx context.Context, req *filer_pb.LookupDirectoryEntryRequest) (*filer_pb.LookupDirectoryEntryResponse, error) {
  19. glog.V(4).Infof("LookupDirectoryEntry %s", filepath.Join(req.Directory, req.Name))
  20. entry, err := fs.filer.FindEntry(ctx, util.JoinPath(req.Directory, req.Name))
  21. if err == filer_pb.ErrNotFound {
  22. return &filer_pb.LookupDirectoryEntryResponse{}, err
  23. }
  24. if err != nil {
  25. glog.V(3).Infof("LookupDirectoryEntry %s: %+v, ", filepath.Join(req.Directory, req.Name), err)
  26. return nil, err
  27. }
  28. return &filer_pb.LookupDirectoryEntryResponse{
  29. Entry: entry.ToProtoEntry(),
  30. }, nil
  31. }
  32. func (fs *FilerServer) ListEntries(req *filer_pb.ListEntriesRequest, stream filer_pb.SeaweedFiler_ListEntriesServer) (err error) {
  33. glog.V(4).Infof("ListEntries %v", req)
  34. limit := int(req.Limit)
  35. if limit == 0 {
  36. limit = fs.option.DirListingLimit
  37. }
  38. paginationLimit := filer.PaginationSize
  39. if limit < paginationLimit {
  40. paginationLimit = limit
  41. }
  42. lastFileName := req.StartFromFileName
  43. includeLastFile := req.InclusiveStartFrom
  44. var listErr error
  45. for limit > 0 {
  46. var hasEntries bool
  47. lastFileName, listErr = fs.filer.StreamListDirectoryEntries(stream.Context(), util.FullPath(req.Directory), lastFileName, includeLastFile, int64(paginationLimit), req.Prefix, "", "", func(entry *filer.Entry) bool {
  48. hasEntries = true
  49. if err = stream.Send(&filer_pb.ListEntriesResponse{
  50. Entry: entry.ToProtoEntry(),
  51. }); err != nil {
  52. return false
  53. }
  54. limit--
  55. if limit == 0 {
  56. return false
  57. }
  58. return true
  59. })
  60. if listErr != nil {
  61. return listErr
  62. }
  63. if err != nil {
  64. return err
  65. }
  66. if !hasEntries {
  67. return nil
  68. }
  69. includeLastFile = false
  70. }
  71. return nil
  72. }
  73. func (fs *FilerServer) LookupVolume(ctx context.Context, req *filer_pb.LookupVolumeRequest) (*filer_pb.LookupVolumeResponse, error) {
  74. resp := &filer_pb.LookupVolumeResponse{
  75. LocationsMap: make(map[string]*filer_pb.Locations),
  76. }
  77. for _, vidString := range req.VolumeIds {
  78. vid, err := strconv.Atoi(vidString)
  79. if err != nil {
  80. glog.V(1).Infof("Unknown volume id %d", vid)
  81. return nil, err
  82. }
  83. var locs []*filer_pb.Location
  84. locations, found := fs.filer.MasterClient.GetLocations(uint32(vid))
  85. if !found {
  86. continue
  87. }
  88. for _, loc := range locations {
  89. locs = append(locs, &filer_pb.Location{
  90. Url: loc.Url,
  91. PublicUrl: loc.PublicUrl,
  92. GrpcPort: uint32(loc.GrpcPort),
  93. })
  94. }
  95. resp.LocationsMap[vidString] = &filer_pb.Locations{
  96. Locations: locs,
  97. }
  98. }
  99. return resp, nil
  100. }
  101. func (fs *FilerServer) lookupFileId(fileId string) (targetUrls []string, err error) {
  102. fid, err := needle.ParseFileIdFromString(fileId)
  103. if err != nil {
  104. return nil, err
  105. }
  106. locations, found := fs.filer.MasterClient.GetLocations(uint32(fid.VolumeId))
  107. if !found || len(locations) == 0 {
  108. return nil, fmt.Errorf("not found volume %d in %s", fid.VolumeId, fileId)
  109. }
  110. for _, loc := range locations {
  111. targetUrls = append(targetUrls, fmt.Sprintf("http://%s/%s", loc.Url, fileId))
  112. }
  113. return
  114. }
  115. func (fs *FilerServer) CreateEntry(ctx context.Context, req *filer_pb.CreateEntryRequest) (resp *filer_pb.CreateEntryResponse, err error) {
  116. glog.V(4).Infof("CreateEntry %v/%v", req.Directory, req.Entry.Name)
  117. resp = &filer_pb.CreateEntryResponse{}
  118. chunks, garbage, err2 := fs.cleanupChunks(util.Join(req.Directory, req.Entry.Name), nil, req.Entry)
  119. if err2 != nil {
  120. return &filer_pb.CreateEntryResponse{}, fmt.Errorf("CreateEntry cleanupChunks %s %s: %v", req.Directory, req.Entry.Name, err2)
  121. }
  122. newEntry := filer.FromPbEntry(req.Directory, req.Entry)
  123. newEntry.Chunks = chunks
  124. createErr := fs.filer.CreateEntry(ctx, newEntry, req.OExcl, req.IsFromOtherCluster, req.Signatures)
  125. if createErr == nil {
  126. fs.filer.DeleteChunks(garbage)
  127. } else {
  128. glog.V(3).Infof("CreateEntry %s: %v", filepath.Join(req.Directory, req.Entry.Name), createErr)
  129. resp.Error = createErr.Error()
  130. }
  131. return
  132. }
  133. func (fs *FilerServer) UpdateEntry(ctx context.Context, req *filer_pb.UpdateEntryRequest) (*filer_pb.UpdateEntryResponse, error) {
  134. glog.V(4).Infof("UpdateEntry %v", req)
  135. fullpath := util.Join(req.Directory, req.Entry.Name)
  136. entry, err := fs.filer.FindEntry(ctx, util.FullPath(fullpath))
  137. if err != nil {
  138. return &filer_pb.UpdateEntryResponse{}, fmt.Errorf("not found %s: %v", fullpath, err)
  139. }
  140. chunks, garbage, err2 := fs.cleanupChunks(fullpath, entry, req.Entry)
  141. if err2 != nil {
  142. return &filer_pb.UpdateEntryResponse{}, fmt.Errorf("UpdateEntry cleanupChunks %s: %v", fullpath, err2)
  143. }
  144. newEntry := filer.FromPbEntry(req.Directory, req.Entry)
  145. newEntry.Chunks = chunks
  146. if filer.EqualEntry(entry, newEntry) {
  147. return &filer_pb.UpdateEntryResponse{}, err
  148. }
  149. if err = fs.filer.UpdateEntry(ctx, entry, newEntry); err == nil {
  150. fs.filer.DeleteChunks(garbage)
  151. fs.filer.NotifyUpdateEvent(ctx, entry, newEntry, true, req.IsFromOtherCluster, req.Signatures)
  152. } else {
  153. glog.V(3).Infof("UpdateEntry %s: %v", filepath.Join(req.Directory, req.Entry.Name), err)
  154. }
  155. return &filer_pb.UpdateEntryResponse{}, err
  156. }
  157. func (fs *FilerServer) cleanupChunks(fullpath string, existingEntry *filer.Entry, newEntry *filer_pb.Entry) (chunks, garbage []*filer_pb.FileChunk, err error) {
  158. // remove old chunks if not included in the new ones
  159. if existingEntry != nil {
  160. garbage, err = filer.MinusChunks(fs.lookupFileId, existingEntry.Chunks, newEntry.Chunks)
  161. if err != nil {
  162. return newEntry.Chunks, nil, fmt.Errorf("MinusChunks: %v", err)
  163. }
  164. }
  165. // files with manifest chunks are usually large and append only, skip calculating covered chunks
  166. manifestChunks, nonManifestChunks := filer.SeparateManifestChunks(newEntry.Chunks)
  167. chunks, coveredChunks := filer.CompactFileChunks(fs.lookupFileId, nonManifestChunks)
  168. garbage = append(garbage, coveredChunks...)
  169. if newEntry.Attributes != nil {
  170. so, _ := fs.detectStorageOption(fullpath,
  171. newEntry.Attributes.Collection,
  172. newEntry.Attributes.Replication,
  173. newEntry.Attributes.TtlSec,
  174. newEntry.Attributes.DiskType,
  175. "",
  176. "",
  177. ) // ignore readonly error for capacity needed to manifestize
  178. chunks, err = filer.MaybeManifestize(fs.saveAsChunk(so), chunks)
  179. if err != nil {
  180. // not good, but should be ok
  181. glog.V(0).Infof("MaybeManifestize: %v", err)
  182. }
  183. }
  184. chunks = append(chunks, manifestChunks...)
  185. return
  186. }
  187. func (fs *FilerServer) AppendToEntry(ctx context.Context, req *filer_pb.AppendToEntryRequest) (*filer_pb.AppendToEntryResponse, error) {
  188. glog.V(4).Infof("AppendToEntry %v", req)
  189. fullpath := util.NewFullPath(req.Directory, req.EntryName)
  190. var offset int64 = 0
  191. entry, err := fs.filer.FindEntry(ctx, fullpath)
  192. if err == filer_pb.ErrNotFound {
  193. entry = &filer.Entry{
  194. FullPath: fullpath,
  195. Attr: filer.Attr{
  196. Crtime: time.Now(),
  197. Mtime: time.Now(),
  198. Mode: os.FileMode(0644),
  199. Uid: OS_UID,
  200. Gid: OS_GID,
  201. },
  202. }
  203. } else {
  204. offset = int64(filer.TotalSize(entry.Chunks))
  205. }
  206. for _, chunk := range req.Chunks {
  207. chunk.Offset = offset
  208. offset += int64(chunk.Size)
  209. }
  210. entry.Chunks = append(entry.Chunks, req.Chunks...)
  211. so, err := fs.detectStorageOption(string(fullpath), entry.Collection, entry.Replication, entry.TtlSec, entry.DiskType, "", "")
  212. if err != nil {
  213. glog.Warningf("detectStorageOption: %v", err)
  214. return &filer_pb.AppendToEntryResponse{}, err
  215. }
  216. entry.Chunks, err = filer.MaybeManifestize(fs.saveAsChunk(so), entry.Chunks)
  217. if err != nil {
  218. // not good, but should be ok
  219. glog.V(0).Infof("MaybeManifestize: %v", err)
  220. }
  221. err = fs.filer.CreateEntry(context.Background(), entry, false, false, nil)
  222. return &filer_pb.AppendToEntryResponse{}, err
  223. }
  224. func (fs *FilerServer) DeleteEntry(ctx context.Context, req *filer_pb.DeleteEntryRequest) (resp *filer_pb.DeleteEntryResponse, err error) {
  225. glog.V(4).Infof("DeleteEntry %v", req)
  226. err = fs.filer.DeleteEntryMetaAndData(ctx, util.JoinPath(req.Directory, req.Name), req.IsRecursive, req.IgnoreRecursiveError, req.IsDeleteData, req.IsFromOtherCluster, req.Signatures)
  227. resp = &filer_pb.DeleteEntryResponse{}
  228. if err != nil && err != filer_pb.ErrNotFound {
  229. resp.Error = err.Error()
  230. }
  231. return resp, nil
  232. }
  233. func (fs *FilerServer) AssignVolume(ctx context.Context, req *filer_pb.AssignVolumeRequest) (resp *filer_pb.AssignVolumeResponse, err error) {
  234. so, err := fs.detectStorageOption(req.Path, req.Collection, req.Replication, req.TtlSec, req.DiskType, req.DataCenter, req.Rack)
  235. if err != nil {
  236. glog.V(3).Infof("AssignVolume: %v", err)
  237. return &filer_pb.AssignVolumeResponse{Error: fmt.Sprintf("assign volume: %v", err)}, nil
  238. }
  239. assignRequest, altRequest := so.ToAssignRequests(int(req.Count))
  240. assignResult, err := operation.Assign(fs.filer.GetMaster, fs.grpcDialOption, assignRequest, altRequest)
  241. if err != nil {
  242. glog.V(3).Infof("AssignVolume: %v", err)
  243. return &filer_pb.AssignVolumeResponse{Error: fmt.Sprintf("assign volume: %v", err)}, nil
  244. }
  245. if assignResult.Error != "" {
  246. glog.V(3).Infof("AssignVolume error: %v", assignResult.Error)
  247. return &filer_pb.AssignVolumeResponse{Error: fmt.Sprintf("assign volume result: %v", assignResult.Error)}, nil
  248. }
  249. return &filer_pb.AssignVolumeResponse{
  250. FileId: assignResult.Fid,
  251. Count: int32(assignResult.Count),
  252. Location: &filer_pb.Location{
  253. Url: assignResult.Url,
  254. PublicUrl: assignResult.PublicUrl,
  255. GrpcPort: uint32(assignResult.GrpcPort),
  256. },
  257. Auth: string(assignResult.Auth),
  258. Collection: so.Collection,
  259. Replication: so.Replication,
  260. }, nil
  261. }
  262. func (fs *FilerServer) CollectionList(ctx context.Context, req *filer_pb.CollectionListRequest) (resp *filer_pb.CollectionListResponse, err error) {
  263. glog.V(4).Infof("CollectionList %v", req)
  264. resp = &filer_pb.CollectionListResponse{}
  265. err = fs.filer.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
  266. masterResp, err := client.CollectionList(context.Background(), &master_pb.CollectionListRequest{
  267. IncludeNormalVolumes: req.IncludeNormalVolumes,
  268. IncludeEcVolumes: req.IncludeEcVolumes,
  269. })
  270. if err != nil {
  271. return err
  272. }
  273. for _, c := range masterResp.Collections {
  274. resp.Collections = append(resp.Collections, &filer_pb.Collection{Name: c.Name})
  275. }
  276. return nil
  277. })
  278. return
  279. }
  280. func (fs *FilerServer) DeleteCollection(ctx context.Context, req *filer_pb.DeleteCollectionRequest) (resp *filer_pb.DeleteCollectionResponse, err error) {
  281. glog.V(4).Infof("DeleteCollection %v", req)
  282. err = fs.filer.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
  283. _, err := client.CollectionDelete(context.Background(), &master_pb.CollectionDeleteRequest{
  284. Name: req.GetCollection(),
  285. })
  286. return err
  287. })
  288. return &filer_pb.DeleteCollectionResponse{}, err
  289. }
  290. func (fs *FilerServer) Statistics(ctx context.Context, req *filer_pb.StatisticsRequest) (resp *filer_pb.StatisticsResponse, err error) {
  291. var output *master_pb.StatisticsResponse
  292. err = fs.filer.MasterClient.WithClient(func(masterClient master_pb.SeaweedClient) error {
  293. grpcResponse, grpcErr := masterClient.Statistics(context.Background(), &master_pb.StatisticsRequest{
  294. Replication: req.Replication,
  295. Collection: req.Collection,
  296. Ttl: req.Ttl,
  297. DiskType: req.DiskType,
  298. })
  299. if grpcErr != nil {
  300. return grpcErr
  301. }
  302. output = grpcResponse
  303. return nil
  304. })
  305. if err != nil {
  306. return nil, err
  307. }
  308. return &filer_pb.StatisticsResponse{
  309. TotalSize: output.TotalSize,
  310. UsedSize: output.UsedSize,
  311. FileCount: output.FileCount,
  312. }, nil
  313. }
  314. func (fs *FilerServer) GetFilerConfiguration(ctx context.Context, req *filer_pb.GetFilerConfigurationRequest) (resp *filer_pb.GetFilerConfigurationResponse, err error) {
  315. clusterId, _ := fs.filer.Store.KvGet(context.Background(), []byte("clusterId"))
  316. t := &filer_pb.GetFilerConfigurationResponse{
  317. Masters: pb.ToAddressStrings(fs.option.Masters),
  318. Collection: fs.option.Collection,
  319. Replication: fs.option.DefaultReplication,
  320. MaxMb: uint32(fs.option.MaxMB),
  321. DirBuckets: fs.filer.DirBucketsPath,
  322. Cipher: fs.filer.Cipher,
  323. Signature: fs.filer.Signature,
  324. MetricsAddress: fs.metricsAddress,
  325. MetricsIntervalSec: int32(fs.metricsIntervalSec),
  326. Version: util.Version(),
  327. ClusterId: string(clusterId),
  328. }
  329. glog.V(4).Infof("GetFilerConfiguration: %v", t)
  330. return t, nil
  331. }
  332. func (fs *FilerServer) KeepConnected(stream filer_pb.SeaweedFiler_KeepConnectedServer) error {
  333. req, err := stream.Recv()
  334. if err != nil {
  335. return err
  336. }
  337. clientName := util.JoinHostPort(req.Name, int(req.GrpcPort))
  338. m := make(map[string]bool)
  339. for _, tp := range req.Resources {
  340. m[tp] = true
  341. }
  342. fs.brokersLock.Lock()
  343. fs.brokers[clientName] = m
  344. glog.V(0).Infof("+ broker %v", clientName)
  345. fs.brokersLock.Unlock()
  346. defer func() {
  347. fs.brokersLock.Lock()
  348. delete(fs.brokers, clientName)
  349. glog.V(0).Infof("- broker %v: %v", clientName, err)
  350. fs.brokersLock.Unlock()
  351. }()
  352. for {
  353. if err := stream.Send(&filer_pb.KeepConnectedResponse{}); err != nil {
  354. glog.V(0).Infof("send broker %v: %+v", clientName, err)
  355. return err
  356. }
  357. // println("replied")
  358. if _, err := stream.Recv(); err != nil {
  359. glog.V(0).Infof("recv broker %v: %v", clientName, err)
  360. return err
  361. }
  362. // println("received")
  363. }
  364. }
  365. func (fs *FilerServer) LocateBroker(ctx context.Context, req *filer_pb.LocateBrokerRequest) (resp *filer_pb.LocateBrokerResponse, err error) {
  366. resp = &filer_pb.LocateBrokerResponse{}
  367. fs.brokersLock.Lock()
  368. defer fs.brokersLock.Unlock()
  369. var localBrokers []*filer_pb.LocateBrokerResponse_Resource
  370. for b, m := range fs.brokers {
  371. if _, found := m[req.Resource]; found {
  372. resp.Found = true
  373. resp.Resources = []*filer_pb.LocateBrokerResponse_Resource{
  374. {
  375. GrpcAddresses: b,
  376. ResourceCount: int32(len(m)),
  377. },
  378. }
  379. return
  380. }
  381. localBrokers = append(localBrokers, &filer_pb.LocateBrokerResponse_Resource{
  382. GrpcAddresses: b,
  383. ResourceCount: int32(len(m)),
  384. })
  385. }
  386. resp.Resources = localBrokers
  387. return resp, nil
  388. }