filer_sync.go 19 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495
  1. package command
  2. import (
  3. "context"
  4. "errors"
  5. "fmt"
  6. "github.com/seaweedfs/seaweedfs/weed/glog"
  7. "github.com/seaweedfs/seaweedfs/weed/pb"
  8. "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
  9. "github.com/seaweedfs/seaweedfs/weed/replication"
  10. "github.com/seaweedfs/seaweedfs/weed/replication/sink"
  11. "github.com/seaweedfs/seaweedfs/weed/replication/sink/filersink"
  12. "github.com/seaweedfs/seaweedfs/weed/replication/source"
  13. "github.com/seaweedfs/seaweedfs/weed/security"
  14. statsCollect "github.com/seaweedfs/seaweedfs/weed/stats"
  15. "github.com/seaweedfs/seaweedfs/weed/util"
  16. "github.com/seaweedfs/seaweedfs/weed/util/grace"
  17. "google.golang.org/grpc"
  18. "os"
  19. "strings"
  20. "time"
  21. )
  22. type SyncOptions struct {
  23. isActivePassive *bool
  24. filerA *string
  25. filerB *string
  26. aPath *string
  27. aExcludePaths *string
  28. bPath *string
  29. bExcludePaths *string
  30. aReplication *string
  31. bReplication *string
  32. aCollection *string
  33. bCollection *string
  34. aTtlSec *int
  35. bTtlSec *int
  36. aDiskType *string
  37. bDiskType *string
  38. aDebug *bool
  39. bDebug *bool
  40. aFromTsMs *int64
  41. bFromTsMs *int64
  42. aProxyByFiler *bool
  43. bProxyByFiler *bool
  44. metricsHttpIp *string
  45. metricsHttpPort *int
  46. concurrency *int
  47. clientId int32
  48. clientEpoch int32
  49. }
  50. const (
  51. SyncKeyPrefix = "sync."
  52. DefaultConcurrencyLimit = 32
  53. )
  54. var (
  55. syncOptions SyncOptions
  56. syncCpuProfile *string
  57. syncMemProfile *string
  58. )
  59. func init() {
  60. cmdFilerSynchronize.Run = runFilerSynchronize // break init cycle
  61. syncOptions.isActivePassive = cmdFilerSynchronize.Flag.Bool("isActivePassive", false, "one directional follow from A to B if true")
  62. syncOptions.filerA = cmdFilerSynchronize.Flag.String("a", "", "filer A in one SeaweedFS cluster")
  63. syncOptions.filerB = cmdFilerSynchronize.Flag.String("b", "", "filer B in the other SeaweedFS cluster")
  64. syncOptions.aPath = cmdFilerSynchronize.Flag.String("a.path", "/", "directory to sync on filer A")
  65. syncOptions.aExcludePaths = cmdFilerSynchronize.Flag.String("a.excludePaths", "", "exclude directories to sync on filer A")
  66. syncOptions.bPath = cmdFilerSynchronize.Flag.String("b.path", "/", "directory to sync on filer B")
  67. syncOptions.bExcludePaths = cmdFilerSynchronize.Flag.String("b.excludePaths", "", "exclude directories to sync on filer B")
  68. syncOptions.aReplication = cmdFilerSynchronize.Flag.String("a.replication", "", "replication on filer A")
  69. syncOptions.bReplication = cmdFilerSynchronize.Flag.String("b.replication", "", "replication on filer B")
  70. syncOptions.aCollection = cmdFilerSynchronize.Flag.String("a.collection", "", "collection on filer A")
  71. syncOptions.bCollection = cmdFilerSynchronize.Flag.String("b.collection", "", "collection on filer B")
  72. syncOptions.aTtlSec = cmdFilerSynchronize.Flag.Int("a.ttlSec", 0, "ttl in seconds on filer A")
  73. syncOptions.bTtlSec = cmdFilerSynchronize.Flag.Int("b.ttlSec", 0, "ttl in seconds on filer B")
  74. syncOptions.aDiskType = cmdFilerSynchronize.Flag.String("a.disk", "", "[hdd|ssd|<tag>] hard drive or solid state drive or any tag on filer A")
  75. syncOptions.bDiskType = cmdFilerSynchronize.Flag.String("b.disk", "", "[hdd|ssd|<tag>] hard drive or solid state drive or any tag on filer B")
  76. syncOptions.aProxyByFiler = cmdFilerSynchronize.Flag.Bool("a.filerProxy", false, "read and write file chunks by filer A instead of volume servers")
  77. syncOptions.bProxyByFiler = cmdFilerSynchronize.Flag.Bool("b.filerProxy", false, "read and write file chunks by filer B instead of volume servers")
  78. syncOptions.aDebug = cmdFilerSynchronize.Flag.Bool("a.debug", false, "debug mode to print out filer A received files")
  79. syncOptions.bDebug = cmdFilerSynchronize.Flag.Bool("b.debug", false, "debug mode to print out filer B received files")
  80. syncOptions.aFromTsMs = cmdFilerSynchronize.Flag.Int64("a.fromTsMs", 0, "synchronization from timestamp on filer A. The unit is millisecond")
  81. syncOptions.bFromTsMs = cmdFilerSynchronize.Flag.Int64("b.fromTsMs", 0, "synchronization from timestamp on filer B. The unit is millisecond")
  82. syncOptions.concurrency = cmdFilerSynchronize.Flag.Int("concurrency", DefaultConcurrencyLimit, "The maximum number of files that will be synced concurrently.")
  83. syncCpuProfile = cmdFilerSynchronize.Flag.String("cpuprofile", "", "cpu profile output file")
  84. syncMemProfile = cmdFilerSynchronize.Flag.String("memprofile", "", "memory profile output file")
  85. syncOptions.metricsHttpIp = cmdFilerSynchronize.Flag.String("metricsIp", "", "metrics listen ip")
  86. syncOptions.metricsHttpPort = cmdFilerSynchronize.Flag.Int("metricsPort", 0, "metrics listen port")
  87. syncOptions.clientId = util.RandomInt32()
  88. }
  89. var cmdFilerSynchronize = &Command{
  90. UsageLine: "filer.sync -a=<oneFilerHost>:<oneFilerPort> -b=<otherFilerHost>:<otherFilerPort>",
  91. Short: "resumable continuous synchronization between two active-active or active-passive SeaweedFS clusters",
  92. Long: `resumable continuous synchronization for file changes between two active-active or active-passive filers
  93. filer.sync listens on filer notifications. If any file is updated, it will fetch the updated content,
  94. and write to the other destination. Different from filer.replicate:
  95. * filer.sync only works between two filers.
  96. * filer.sync does not need any special message queue setup.
  97. * filer.sync supports both active-active and active-passive modes.
  98. If restarted, the synchronization will resume from the previous checkpoints, persisted every minute.
  99. A fresh sync will start from the earliest metadata logs.
  100. `,
  101. }
  102. func runFilerSynchronize(cmd *Command, args []string) bool {
  103. util.LoadConfiguration("security", false)
  104. grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.client")
  105. grace.SetupProfiling(*syncCpuProfile, *syncMemProfile)
  106. filerA := pb.ServerAddress(*syncOptions.filerA)
  107. filerB := pb.ServerAddress(*syncOptions.filerB)
  108. // start filer.sync metrics server
  109. go statsCollect.StartMetricsServer(*syncOptions.metricsHttpIp, *syncOptions.metricsHttpPort)
  110. // read a filer signature
  111. aFilerSignature, aFilerErr := replication.ReadFilerSignature(grpcDialOption, filerA)
  112. if aFilerErr != nil {
  113. glog.Errorf("get filer 'a' signature %d error from %s to %s: %v", aFilerSignature, *syncOptions.filerA, *syncOptions.filerB, aFilerErr)
  114. return true
  115. }
  116. // read b filer signature
  117. bFilerSignature, bFilerErr := replication.ReadFilerSignature(grpcDialOption, filerB)
  118. if bFilerErr != nil {
  119. glog.Errorf("get filer 'b' signature %d error from %s to %s: %v", bFilerSignature, *syncOptions.filerA, *syncOptions.filerB, bFilerErr)
  120. return true
  121. }
  122. go func() {
  123. // a->b
  124. // set synchronization start timestamp to offset
  125. initOffsetError := initOffsetFromTsMs(grpcDialOption, filerB, aFilerSignature, *syncOptions.bFromTsMs, getSignaturePrefixByPath(*syncOptions.aPath))
  126. if initOffsetError != nil {
  127. glog.Errorf("init offset from timestamp %d error from %s to %s: %v", *syncOptions.bFromTsMs, *syncOptions.filerA, *syncOptions.filerB, initOffsetError)
  128. os.Exit(2)
  129. }
  130. for {
  131. syncOptions.clientEpoch++
  132. err := doSubscribeFilerMetaChanges(
  133. syncOptions.clientId,
  134. syncOptions.clientEpoch,
  135. grpcDialOption,
  136. filerA,
  137. *syncOptions.aPath,
  138. util.StringSplit(*syncOptions.aExcludePaths, ","),
  139. *syncOptions.aProxyByFiler,
  140. filerB,
  141. *syncOptions.bPath,
  142. *syncOptions.bReplication,
  143. *syncOptions.bCollection,
  144. *syncOptions.bTtlSec,
  145. *syncOptions.bProxyByFiler,
  146. *syncOptions.bDiskType,
  147. *syncOptions.bDebug,
  148. *syncOptions.concurrency,
  149. aFilerSignature,
  150. bFilerSignature)
  151. if err != nil {
  152. glog.Errorf("sync from %s to %s: %v", *syncOptions.filerA, *syncOptions.filerB, err)
  153. time.Sleep(1747 * time.Millisecond)
  154. }
  155. }
  156. }()
  157. if !*syncOptions.isActivePassive {
  158. // b->a
  159. // set synchronization start timestamp to offset
  160. initOffsetError := initOffsetFromTsMs(grpcDialOption, filerA, bFilerSignature, *syncOptions.aFromTsMs, getSignaturePrefixByPath(*syncOptions.bPath))
  161. if initOffsetError != nil {
  162. glog.Errorf("init offset from timestamp %d error from %s to %s: %v", *syncOptions.aFromTsMs, *syncOptions.filerB, *syncOptions.filerA, initOffsetError)
  163. os.Exit(2)
  164. }
  165. go func() {
  166. for {
  167. syncOptions.clientEpoch++
  168. err := doSubscribeFilerMetaChanges(
  169. syncOptions.clientId,
  170. syncOptions.clientEpoch,
  171. grpcDialOption,
  172. filerB,
  173. *syncOptions.bPath,
  174. util.StringSplit(*syncOptions.bExcludePaths, ","),
  175. *syncOptions.bProxyByFiler,
  176. filerA,
  177. *syncOptions.aPath,
  178. *syncOptions.aReplication,
  179. *syncOptions.aCollection,
  180. *syncOptions.aTtlSec,
  181. *syncOptions.aProxyByFiler,
  182. *syncOptions.aDiskType,
  183. *syncOptions.aDebug,
  184. *syncOptions.concurrency,
  185. bFilerSignature,
  186. aFilerSignature)
  187. if err != nil {
  188. glog.Errorf("sync from %s to %s: %v", *syncOptions.filerB, *syncOptions.filerA, err)
  189. time.Sleep(2147 * time.Millisecond)
  190. }
  191. }
  192. }()
  193. }
  194. select {}
  195. return true
  196. }
  197. // initOffsetFromTsMs Initialize offset
  198. func initOffsetFromTsMs(grpcDialOption grpc.DialOption, targetFiler pb.ServerAddress, sourceFilerSignature int32, fromTsMs int64, signaturePrefix string) error {
  199. if fromTsMs <= 0 {
  200. return nil
  201. }
  202. // convert to nanosecond
  203. fromTsNs := fromTsMs * 1000_000
  204. // If not successful, exit the program.
  205. setOffsetErr := setOffset(grpcDialOption, targetFiler, signaturePrefix, sourceFilerSignature, fromTsNs)
  206. if setOffsetErr != nil {
  207. return setOffsetErr
  208. }
  209. glog.Infof("setOffset from timestamp ms success! start offset: %d from %s to %s", fromTsNs, *syncOptions.filerA, *syncOptions.filerB)
  210. return nil
  211. }
  212. func doSubscribeFilerMetaChanges(clientId int32, clientEpoch int32, grpcDialOption grpc.DialOption, sourceFiler pb.ServerAddress, sourcePath string, sourceExcludePaths []string, sourceReadChunkFromFiler bool, targetFiler pb.ServerAddress, targetPath string,
  213. replicationStr, collection string, ttlSec int, sinkWriteChunkByFiler bool, diskType string, debug bool, concurrency int, sourceFilerSignature int32, targetFilerSignature int32) error {
  214. // if first time, start from now
  215. // if has previously synced, resume from that point of time
  216. sourceFilerOffsetTsNs, err := getOffset(grpcDialOption, targetFiler, getSignaturePrefixByPath(sourcePath), sourceFilerSignature)
  217. if err != nil {
  218. return err
  219. }
  220. glog.V(0).Infof("start sync %s(%d) => %s(%d) from %v(%d)", sourceFiler, sourceFilerSignature, targetFiler, targetFilerSignature, time.Unix(0, sourceFilerOffsetTsNs), sourceFilerOffsetTsNs)
  221. // create filer sink
  222. filerSource := &source.FilerSource{}
  223. filerSource.DoInitialize(sourceFiler.ToHttpAddress(), sourceFiler.ToGrpcAddress(), sourcePath, sourceReadChunkFromFiler)
  224. filerSink := &filersink.FilerSink{}
  225. filerSink.DoInitialize(targetFiler.ToHttpAddress(), targetFiler.ToGrpcAddress(), targetPath, replicationStr, collection, ttlSec, diskType, grpcDialOption, sinkWriteChunkByFiler)
  226. filerSink.SetSourceFiler(filerSource)
  227. persistEventFn := genProcessFunction(sourcePath, targetPath, sourceExcludePaths, filerSink, debug)
  228. processEventFn := func(resp *filer_pb.SubscribeMetadataResponse) error {
  229. message := resp.EventNotification
  230. for _, sig := range message.Signatures {
  231. if sig == targetFilerSignature && targetFilerSignature != 0 {
  232. fmt.Printf("%s skipping %s change to %v\n", targetFiler, sourceFiler, message)
  233. return nil
  234. }
  235. }
  236. return persistEventFn(resp)
  237. }
  238. if concurrency < 0 || concurrency > 1024 {
  239. glog.Warningf("invalid concurrency value, using default: %d", DefaultConcurrencyLimit)
  240. concurrency = DefaultConcurrencyLimit
  241. }
  242. processor := NewMetadataProcessor(processEventFn, concurrency)
  243. var lastLogTsNs = time.Now().UnixNano()
  244. var clientName = fmt.Sprintf("syncFrom_%s_To_%s", string(sourceFiler), string(targetFiler))
  245. processEventFnWithOffset := pb.AddOffsetFunc(func(resp *filer_pb.SubscribeMetadataResponse) error {
  246. processor.AddSyncJob(resp)
  247. return nil
  248. }, 3*time.Second, func(counter int64, lastTsNs int64) error {
  249. if processor.processedTsWatermark == 0 {
  250. return nil
  251. }
  252. // use processor.processedTsWatermark instead of the lastTsNs from the most recent job
  253. now := time.Now().UnixNano()
  254. glog.V(0).Infof("sync %s to %s progressed to %v %0.2f/sec", sourceFiler, targetFiler, time.Unix(0, processor.processedTsWatermark), float64(counter)/(float64(now-lastLogTsNs)/1e9))
  255. lastLogTsNs = now
  256. // collect synchronous offset
  257. statsCollect.FilerSyncOffsetGauge.WithLabelValues(sourceFiler.String(), targetFiler.String(), clientName, sourcePath).Set(float64(processor.processedTsWatermark))
  258. return setOffset(grpcDialOption, targetFiler, getSignaturePrefixByPath(sourcePath), sourceFilerSignature, processor.processedTsWatermark)
  259. })
  260. metadataFollowOption := &pb.MetadataFollowOption{
  261. ClientName: clientName,
  262. ClientId: clientId,
  263. ClientEpoch: clientEpoch,
  264. SelfSignature: targetFilerSignature,
  265. PathPrefix: sourcePath,
  266. AdditionalPathPrefixes: nil,
  267. DirectoriesToWatch: nil,
  268. StartTsNs: sourceFilerOffsetTsNs,
  269. StopTsNs: 0,
  270. EventErrorType: pb.RetryForeverOnError,
  271. }
  272. return pb.FollowMetadata(sourceFiler, grpcDialOption, metadataFollowOption, processEventFnWithOffset)
  273. }
  274. // When each business is distinguished according to path, and offsets need to be maintained separately.
  275. func getSignaturePrefixByPath(path string) string {
  276. // compatible historical version
  277. if path == "/" {
  278. return SyncKeyPrefix
  279. } else {
  280. return SyncKeyPrefix + path
  281. }
  282. }
  283. func getOffset(grpcDialOption grpc.DialOption, filer pb.ServerAddress, signaturePrefix string, signature int32) (lastOffsetTsNs int64, readErr error) {
  284. readErr = pb.WithFilerClient(false, signature, filer, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
  285. syncKey := []byte(signaturePrefix + "____")
  286. util.Uint32toBytes(syncKey[len(signaturePrefix):len(signaturePrefix)+4], uint32(signature))
  287. resp, err := client.KvGet(context.Background(), &filer_pb.KvGetRequest{Key: syncKey})
  288. if err != nil {
  289. return err
  290. }
  291. if len(resp.Error) != 0 {
  292. return errors.New(resp.Error)
  293. }
  294. if len(resp.Value) < 8 {
  295. return nil
  296. }
  297. lastOffsetTsNs = int64(util.BytesToUint64(resp.Value))
  298. return nil
  299. })
  300. return
  301. }
  302. func setOffset(grpcDialOption grpc.DialOption, filer pb.ServerAddress, signaturePrefix string, signature int32, offsetTsNs int64) error {
  303. return pb.WithFilerClient(false, signature, filer, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
  304. syncKey := []byte(signaturePrefix + "____")
  305. util.Uint32toBytes(syncKey[len(signaturePrefix):len(signaturePrefix)+4], uint32(signature))
  306. valueBuf := make([]byte, 8)
  307. util.Uint64toBytes(valueBuf, uint64(offsetTsNs))
  308. resp, err := client.KvPut(context.Background(), &filer_pb.KvPutRequest{
  309. Key: syncKey,
  310. Value: valueBuf,
  311. })
  312. if err != nil {
  313. return err
  314. }
  315. if len(resp.Error) != 0 {
  316. return errors.New(resp.Error)
  317. }
  318. return nil
  319. })
  320. }
  321. func genProcessFunction(sourcePath string, targetPath string, excludePaths []string, dataSink sink.ReplicationSink, debug bool) func(resp *filer_pb.SubscribeMetadataResponse) error {
  322. // process function
  323. processEventFn := func(resp *filer_pb.SubscribeMetadataResponse) error {
  324. message := resp.EventNotification
  325. var sourceOldKey, sourceNewKey util.FullPath
  326. if message.OldEntry != nil {
  327. sourceOldKey = util.FullPath(resp.Directory).Child(message.OldEntry.Name)
  328. }
  329. if message.NewEntry != nil {
  330. sourceNewKey = util.FullPath(message.NewParentPath).Child(message.NewEntry.Name)
  331. }
  332. if debug {
  333. glog.V(0).Infof("received %v", resp)
  334. }
  335. if !strings.HasPrefix(resp.Directory, sourcePath) {
  336. return nil
  337. }
  338. for _, excludePath := range excludePaths {
  339. if strings.HasPrefix(resp.Directory, excludePath) {
  340. return nil
  341. }
  342. }
  343. // handle deletions
  344. if filer_pb.IsDelete(resp) {
  345. if !strings.HasPrefix(string(sourceOldKey), sourcePath) {
  346. return nil
  347. }
  348. key := buildKey(dataSink, message, targetPath, sourceOldKey, sourcePath)
  349. if !dataSink.IsIncremental() {
  350. return dataSink.DeleteEntry(key, message.OldEntry.IsDirectory, message.DeleteChunks, message.Signatures)
  351. }
  352. return nil
  353. }
  354. // handle new entries
  355. if filer_pb.IsCreate(resp) {
  356. if !strings.HasPrefix(string(sourceNewKey), sourcePath) {
  357. return nil
  358. }
  359. key := buildKey(dataSink, message, targetPath, sourceNewKey, sourcePath)
  360. if err := dataSink.CreateEntry(key, message.NewEntry, message.Signatures); err != nil {
  361. return fmt.Errorf("create entry1 : %v", err)
  362. } else {
  363. return nil
  364. }
  365. }
  366. // this is something special?
  367. if filer_pb.IsEmpty(resp) {
  368. return nil
  369. }
  370. // handle updates
  371. if strings.HasPrefix(string(sourceOldKey), sourcePath) {
  372. // old key is in the watched directory
  373. if strings.HasPrefix(string(sourceNewKey), sourcePath) {
  374. // new key is also in the watched directory
  375. if !dataSink.IsIncremental() {
  376. oldKey := util.Join(targetPath, string(sourceOldKey)[len(sourcePath):])
  377. message.NewParentPath = util.Join(targetPath, message.NewParentPath[len(sourcePath):])
  378. foundExisting, err := dataSink.UpdateEntry(string(oldKey), message.OldEntry, message.NewParentPath, message.NewEntry, message.DeleteChunks, message.Signatures)
  379. if foundExisting {
  380. return err
  381. }
  382. // not able to find old entry
  383. if err = dataSink.DeleteEntry(string(oldKey), message.OldEntry.IsDirectory, false, message.Signatures); err != nil {
  384. return fmt.Errorf("delete old entry %v: %v", oldKey, err)
  385. }
  386. }
  387. // create the new entry
  388. newKey := buildKey(dataSink, message, targetPath, sourceNewKey, sourcePath)
  389. if err := dataSink.CreateEntry(newKey, message.NewEntry, message.Signatures); err != nil {
  390. return fmt.Errorf("create entry2 : %v", err)
  391. } else {
  392. return nil
  393. }
  394. } else {
  395. // new key is outside of the watched directory
  396. if !dataSink.IsIncremental() {
  397. key := buildKey(dataSink, message, targetPath, sourceOldKey, sourcePath)
  398. return dataSink.DeleteEntry(key, message.OldEntry.IsDirectory, message.DeleteChunks, message.Signatures)
  399. }
  400. }
  401. } else {
  402. // old key is outside of the watched directory
  403. if strings.HasPrefix(string(sourceNewKey), sourcePath) {
  404. // new key is in the watched directory
  405. key := buildKey(dataSink, message, targetPath, sourceNewKey, sourcePath)
  406. if err := dataSink.CreateEntry(key, message.NewEntry, message.Signatures); err != nil {
  407. return fmt.Errorf("create entry3 : %v", err)
  408. } else {
  409. return nil
  410. }
  411. } else {
  412. // new key is also outside of the watched directory
  413. // skip
  414. }
  415. }
  416. return nil
  417. }
  418. return processEventFn
  419. }
  420. func buildKey(dataSink sink.ReplicationSink, message *filer_pb.EventNotification, targetPath string, sourceKey util.FullPath, sourcePath string) (key string) {
  421. if !dataSink.IsIncremental() {
  422. key = util.Join(targetPath, string(sourceKey)[len(sourcePath):])
  423. } else {
  424. var mTime int64
  425. if message.NewEntry != nil {
  426. mTime = message.NewEntry.Attributes.Mtime
  427. } else if message.OldEntry != nil {
  428. mTime = message.OldEntry.Attributes.Mtime
  429. }
  430. dateKey := time.Unix(mTime, 0).Format("2006-01-02")
  431. key = util.Join(targetPath, dateKey, string(sourceKey)[len(sourcePath):])
  432. }
  433. return escapeKey(key)
  434. }