command_fs_verify.go 10 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313
  1. package shell
  2. import (
  3. "bytes"
  4. "context"
  5. "flag"
  6. "fmt"
  7. "github.com/seaweedfs/seaweedfs/weed/filer"
  8. "github.com/seaweedfs/seaweedfs/weed/operation"
  9. "github.com/seaweedfs/seaweedfs/weed/pb"
  10. "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
  11. "github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
  12. "github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb"
  13. "github.com/seaweedfs/seaweedfs/weed/storage"
  14. "github.com/seaweedfs/seaweedfs/weed/util"
  15. "go.uber.org/atomic"
  16. "golang.org/x/exp/slices"
  17. "io"
  18. "math"
  19. "strings"
  20. "sync"
  21. "time"
  22. )
  23. func init() {
  24. Commands = append(Commands, &commandFsVerify{})
  25. }
  26. type commandFsVerify struct {
  27. env *CommandEnv
  28. volumeServers []pb.ServerAddress
  29. volumeIds map[uint32][]pb.ServerAddress
  30. verbose *bool
  31. metadataFromLog *bool
  32. concurrency *int
  33. modifyTimeAgoAtSec int64
  34. writer io.Writer
  35. waitChan map[string]chan struct{}
  36. waitChanLock sync.RWMutex
  37. }
  38. func (c *commandFsVerify) Name() string {
  39. return "fs.verify"
  40. }
  41. func (c *commandFsVerify) Help() string {
  42. return `recursively verify all files under a directory
  43. fs.verify [-v] [-modifyTimeAgo 1h] /buckets/dir
  44. `
  45. }
  46. func (c *commandFsVerify) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
  47. c.env = commandEnv
  48. c.writer = writer
  49. fsVerifyCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
  50. c.verbose = fsVerifyCommand.Bool("v", false, "print out each processed files")
  51. modifyTimeAgo := fsVerifyCommand.Duration("modifyTimeAgo", 0, "only include files after this modify time to verify")
  52. c.concurrency = fsVerifyCommand.Int("concurrency", 0, "number of parallel verification per volume server")
  53. c.metadataFromLog = fsVerifyCommand.Bool("metadataFromLog", false, "Using filer log to get metadata")
  54. if err = fsVerifyCommand.Parse(args); err != nil {
  55. return err
  56. }
  57. path, parseErr := commandEnv.parseUrl(findInputDirectory(fsVerifyCommand.Args()))
  58. if parseErr != nil {
  59. return parseErr
  60. }
  61. c.modifyTimeAgoAtSec = int64(modifyTimeAgo.Seconds())
  62. c.volumeIds = make(map[uint32][]pb.ServerAddress)
  63. c.waitChan = make(map[string]chan struct{})
  64. c.volumeServers = []pb.ServerAddress{}
  65. defer func() {
  66. c.modifyTimeAgoAtSec = 0
  67. c.volumeIds = nil
  68. c.waitChan = nil
  69. c.volumeServers = nil
  70. }()
  71. if err := c.collectVolumeIds(); err != nil {
  72. return parseErr
  73. }
  74. if *c.concurrency > 0 {
  75. for _, volumeServer := range c.volumeServers {
  76. volumeServerStr := string(volumeServer)
  77. c.waitChan[volumeServerStr] = make(chan struct{}, *c.concurrency)
  78. defer close(c.waitChan[volumeServerStr])
  79. }
  80. }
  81. var fCount, eCount uint64
  82. if *c.metadataFromLog {
  83. var wg sync.WaitGroup
  84. fCount, eCount, err = c.verifyProcessMetadata(path, &wg)
  85. wg.Wait()
  86. if err != nil {
  87. return err
  88. }
  89. } else {
  90. fCount, eCount, err = c.verifyTraverseBfs(path)
  91. }
  92. fmt.Fprintf(writer, "verified %d files, error %d files \n", fCount, eCount)
  93. return err
  94. }
  95. func (c *commandFsVerify) collectVolumeIds() error {
  96. topologyInfo, _, err := collectTopologyInfo(c.env, 0)
  97. if err != nil {
  98. return err
  99. }
  100. eachDataNode(topologyInfo, func(dc string, rack RackId, nodeInfo *master_pb.DataNodeInfo) {
  101. for _, diskInfo := range nodeInfo.DiskInfos {
  102. for _, vi := range diskInfo.VolumeInfos {
  103. volumeServer := pb.NewServerAddressFromDataNode(nodeInfo)
  104. c.volumeIds[vi.Id] = append(c.volumeIds[vi.Id], volumeServer)
  105. if !slices.Contains(c.volumeServers, volumeServer) {
  106. c.volumeServers = append(c.volumeServers, volumeServer)
  107. }
  108. }
  109. for _, vi := range diskInfo.EcShardInfos {
  110. volumeServer := pb.NewServerAddressFromDataNode(nodeInfo)
  111. c.volumeIds[vi.Id] = append(c.volumeIds[vi.Id], volumeServer)
  112. if !slices.Contains(c.volumeServers, volumeServer) {
  113. c.volumeServers = append(c.volumeServers, volumeServer)
  114. }
  115. }
  116. }
  117. })
  118. return nil
  119. }
  120. func (c *commandFsVerify) verifyChunk(volumeServer pb.ServerAddress, fileId *filer_pb.FileId) error {
  121. err := operation.WithVolumeServerClient(false, volumeServer, c.env.option.GrpcDialOption,
  122. func(client volume_server_pb.VolumeServerClient) error {
  123. _, err := client.VolumeNeedleStatus(context.Background(),
  124. &volume_server_pb.VolumeNeedleStatusRequest{
  125. VolumeId: fileId.VolumeId,
  126. NeedleId: fileId.FileKey})
  127. return err
  128. },
  129. )
  130. if err != nil && !strings.Contains(err.Error(), storage.ErrorDeleted.Error()) {
  131. return err
  132. }
  133. return nil
  134. }
  135. type ItemEntry struct {
  136. chunks []*filer_pb.FileChunk
  137. path util.FullPath
  138. }
  139. func (c *commandFsVerify) verifyProcessMetadata(path string, wg *sync.WaitGroup) (fileCount uint64, errCount uint64, err error) {
  140. processEventFn := func(resp *filer_pb.SubscribeMetadataResponse) error {
  141. message := resp.EventNotification
  142. if resp.EventNotification.NewEntry == nil {
  143. return nil
  144. }
  145. chunkCount := len(message.NewEntry.Chunks)
  146. if chunkCount == 0 {
  147. return nil
  148. }
  149. entryPath := fmt.Sprintf("%s/%s", message.NewParentPath, message.NewEntry.Name)
  150. errorChunksCount := atomic.NewUint64(0)
  151. if !c.verifyEntry(entryPath, message.NewEntry.Chunks, errorChunksCount, wg) {
  152. if err = c.env.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
  153. entryResp, errReq := client.LookupDirectoryEntry(context.Background(), &filer_pb.LookupDirectoryEntryRequest{
  154. Directory: message.NewParentPath,
  155. Name: message.NewEntry.Name,
  156. })
  157. if errReq != nil {
  158. if strings.HasSuffix(errReq.Error(), "no entry is found in filer store") {
  159. return nil
  160. }
  161. return errReq
  162. }
  163. if entryResp.Entry.Attributes.Mtime == message.NewEntry.Attributes.Mtime &&
  164. bytes.Equal(entryResp.Entry.Attributes.Md5, message.NewEntry.Attributes.Md5) {
  165. fmt.Fprintf(c.writer, "file: %s needles:%d failed:%d\n", entryPath, chunkCount, errorChunksCount.Load())
  166. errCount++
  167. }
  168. return nil
  169. }); err != nil {
  170. return err
  171. }
  172. return nil
  173. }
  174. if *c.verbose {
  175. fmt.Fprintf(c.writer, "file: %s needles:%d verifed\n", entryPath, chunkCount)
  176. }
  177. fileCount++
  178. return nil
  179. }
  180. metadataFollowOption := &pb.MetadataFollowOption{
  181. ClientName: "shell_verify",
  182. ClientId: util.RandomInt32(),
  183. ClientEpoch: 0,
  184. SelfSignature: 0,
  185. PathPrefix: path,
  186. AdditionalPathPrefixes: nil,
  187. DirectoriesToWatch: nil,
  188. StartTsNs: time.Now().Add(-1 * time.Second * time.Duration(c.modifyTimeAgoAtSec)).UnixNano(),
  189. StopTsNs: time.Now().UnixNano(),
  190. EventErrorType: pb.DontLogError,
  191. }
  192. return fileCount, errCount, pb.FollowMetadata(c.env.option.FilerAddress, c.env.option.GrpcDialOption, metadataFollowOption, processEventFn)
  193. }
  194. func (c *commandFsVerify) verifyEntry(path string, chunks []*filer_pb.FileChunk, errorCount *atomic.Uint64, wg *sync.WaitGroup) bool {
  195. fileMsg := fmt.Sprintf("file:%s", path)
  196. itemIsVerifed := atomic.NewBool(true)
  197. for _, chunk := range chunks {
  198. if volumeIds, ok := c.volumeIds[chunk.Fid.VolumeId]; ok {
  199. for _, volumeServer := range volumeIds {
  200. if *c.concurrency == 0 {
  201. if err := c.verifyChunk(volumeServer, chunk.Fid); err != nil {
  202. if !(*c.metadataFromLog && strings.HasSuffix(err.Error(), "not found")) {
  203. fmt.Fprintf(c.writer, "%s failed verify fileId %s: %+v, at volume server %v\n",
  204. fileMsg, chunk.GetFileIdString(), err, volumeServer)
  205. }
  206. if itemIsVerifed.Load() {
  207. itemIsVerifed.Store(false)
  208. errorCount.Add(1)
  209. }
  210. }
  211. continue
  212. }
  213. c.waitChanLock.RLock()
  214. waitChan, ok := c.waitChan[string(volumeServer)]
  215. c.waitChanLock.RUnlock()
  216. if !ok {
  217. fmt.Fprintf(c.writer, "%s failed to get channel for %s fileId: %s\n",
  218. string(volumeServer), fileMsg, chunk.GetFileIdString())
  219. if itemIsVerifed.Load() {
  220. itemIsVerifed.Store(false)
  221. errorCount.Add(1)
  222. }
  223. continue
  224. }
  225. wg.Add(1)
  226. waitChan <- struct{}{}
  227. go func(fChunk *filer_pb.FileChunk, path string, volumeServer pb.ServerAddress, msg string) {
  228. defer wg.Done()
  229. if err := c.verifyChunk(volumeServer, fChunk.Fid); err != nil {
  230. if !(*c.metadataFromLog && strings.HasSuffix(err.Error(), "not found")) {
  231. fmt.Fprintf(c.writer, "%s failed verify fileId %s: %+v, at volume server %v\n",
  232. msg, fChunk.GetFileIdString(), err, volumeServer)
  233. }
  234. if itemIsVerifed.Load() {
  235. itemIsVerifed.Store(false)
  236. errorCount.Add(1)
  237. }
  238. }
  239. <-waitChan
  240. }(chunk, path, volumeServer, fileMsg)
  241. }
  242. } else {
  243. if !*c.metadataFromLog {
  244. err := fmt.Errorf("volumeId %d not found", chunk.Fid.VolumeId)
  245. fmt.Fprintf(c.writer, "%s failed verify fileId %s: %+v\n",
  246. fileMsg, chunk.GetFileIdString(), err)
  247. }
  248. if itemIsVerifed.Load() {
  249. itemIsVerifed.Store(false)
  250. errorCount.Add(1)
  251. }
  252. break
  253. }
  254. }
  255. return itemIsVerifed.Load()
  256. }
  257. func (c *commandFsVerify) verifyTraverseBfs(path string) (fileCount uint64, errCount uint64, err error) {
  258. timeNowAtSec := time.Now().Unix()
  259. return fileCount, errCount, doTraverseBfsAndSaving(c.env, c.writer, path, false,
  260. func(entry *filer_pb.FullEntry, outputChan chan interface{}) (err error) {
  261. if c.modifyTimeAgoAtSec > 0 {
  262. if entry.Entry.Attributes != nil && c.modifyTimeAgoAtSec < timeNowAtSec-entry.Entry.Attributes.Mtime {
  263. return nil
  264. }
  265. }
  266. dataChunks, manifestChunks, resolveErr := filer.ResolveChunkManifest(filer.LookupFn(c.env), entry.Entry.GetChunks(), 0, math.MaxInt64)
  267. if resolveErr != nil {
  268. return fmt.Errorf("failed to ResolveChunkManifest: %+v", resolveErr)
  269. }
  270. dataChunks = append(dataChunks, manifestChunks...)
  271. if len(dataChunks) > 0 {
  272. outputChan <- &ItemEntry{
  273. chunks: dataChunks,
  274. path: util.NewFullPath(entry.Dir, entry.Entry.Name),
  275. }
  276. }
  277. return nil
  278. },
  279. func(outputChan chan interface{}) {
  280. var wg sync.WaitGroup
  281. itemErrCount := atomic.NewUint64(0)
  282. for itemEntry := range outputChan {
  283. i := itemEntry.(*ItemEntry)
  284. itemPath := string(i.path)
  285. if c.verifyEntry(itemPath, i.chunks, itemErrCount, &wg) {
  286. if *c.verbose {
  287. fmt.Fprintf(c.writer, "file: %s needles:%d verifed\n", itemPath, len(i.chunks))
  288. }
  289. fileCount++
  290. }
  291. }
  292. wg.Wait()
  293. errCount = itemErrCount.Load()
  294. })
  295. }