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.

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