Skip to content

Commit c001e9e

Browse files
committed
tarfs: remount EROFS for existing tarfs instances on startup
On startup, we need to recover information for all tarfs related snapshots, and remount EROFS filesystems. Signed-off-by: Jiang Liu <[email protected]>
1 parent 64fcc6b commit c001e9e

File tree

5 files changed

+172
-13
lines changed

5 files changed

+172
-13
lines changed

pkg/filesystem/fs.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -351,7 +351,7 @@ func (fs *Filesystem) Mount(ctx context.Context, snapshotID string, labels map[s
351351
err = errors.Wrapf(err, "mount file system by daemon %s, snapshot %s", d.ID(), snapshotID)
352352
}
353353
case config.FsDriverBlockdev:
354-
err = fs.tarfsMgr.MountTarErofs(snapshotID, s, labels, rafs)
354+
err = fs.tarfsMgr.MountErofs(snapshotID, s, labels, rafs)
355355
if err != nil {
356356
err = errors.Wrapf(err, "mount tarfs for snapshot %s", snapshotID)
357357
}

pkg/label/label.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,8 @@ const (
3636
NydusRefLayer = "containerd.io/snapshot/nydus-ref"
3737
// The blobID of associated layer, also marking the layer as a nydus tarfs, set by the snapshotter
3838
NydusTarfsLayer = "containerd.io/snapshot/nydus-tarfs"
39+
// List of parent snapshot IDs, saved in `Rafs.Annotation`.
40+
NydusTarfsParents = "containerd.io/snapshot/nydus-tarfs-parent-snapshot-list"
3941
// Dm-verity information for image block device
4042
NydusImageBlockInfo = "containerd.io/snapshot/nydus-image-block"
4143
// Dm-verity information for layer block device

pkg/tarfs/tarfs.go

Lines changed: 127 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -57,6 +57,7 @@ const (
5757
)
5858

5959
type Manager struct {
60+
RemountMap map[string]*rafs.Rafs // Scratch space to store rafs instances needing remount on startup
6061
snapshotMap map[string]*snapshotStatus // tarfs snapshots status, indexed by snapshot ID
6162
mutex sync.Mutex
6263
mutexLoopDev sync.Mutex
@@ -87,6 +88,7 @@ type snapshotStatus struct {
8788
func NewManager(insecure, checkTarfsHint bool, cacheDirPath, nydusImagePath string, maxConcurrentProcess int64) *Manager {
8889
return &Manager{
8990
snapshotMap: map[string]*snapshotStatus{},
91+
RemountMap: map[string]*rafs.Rafs{},
9092
cacheDirPath: cacheDirPath,
9193
nydusImagePath: nydusImagePath,
9294
insecure: insecure,
@@ -363,10 +365,9 @@ func (t *Manager) blobProcess(ctx context.Context, snapshotID, ref string,
363365
err = t.generateBootstrap(ds, snapshotID, layerBlobID, upperDirPath)
364366
if err != nil && !errdefs.IsAlreadyExists(err) {
365367
return epilog(err, "generate tarfs data from image layer blob")
366-
} else {
367-
msg := fmt.Sprintf("Nydus tarfs for snapshot %s is ready", snapshotID)
368-
return epilog(nil, msg)
369368
}
369+
msg := fmt.Sprintf("Nydus tarfs for snapshot %s is ready", snapshotID)
370+
return epilog(nil, msg)
370371
}
371372
}
372373

@@ -389,7 +390,9 @@ func (t *Manager) blobProcess(ctx context.Context, snapshotID, ref string,
389390
} else {
390391
// Download and convert layer content in background.
391392
// Will retry when the content is actually needed if the background process failed.
392-
go process(rc, remote)
393+
go func() {
394+
_ = process(rc, remote)
395+
}()
393396
}
394397

395398
return err
@@ -418,7 +421,7 @@ func (t *Manager) retryPrepareLayer(snapshotID, upperDirPath string, labels map[
418421
case TarfsStatusPrepare:
419422
log.L.Infof("Another thread is retrying snapshot %s, wait for the result", snapshotID)
420423
st.mutex.Unlock()
421-
st, err = t.waitLayerReady(snapshotID, false)
424+
_, err = t.waitLayerReady(snapshotID, false)
422425
return err
423426
case TarfsStatusReady:
424427
log.L.Infof("Another thread has retried snapshot %s and succeed", snapshotID)
@@ -622,7 +625,7 @@ func (t *Manager) ExportBlockData(s storage.Snapshot, perLayer bool, labels map[
622625
return updateFields, nil
623626
}
624627

625-
func (t *Manager) MountTarErofs(snapshotID string, s *storage.Snapshot, labels map[string]string, rafs *rafs.Rafs) error {
628+
func (t *Manager) MountErofs(snapshotID string, s *storage.Snapshot, labels map[string]string, rafs *rafs.Rafs) error {
626629
if s == nil {
627630
return errors.New("snapshot object for MountTarErofs() is nil")
628631
}
@@ -643,6 +646,7 @@ func (t *Manager) MountTarErofs(snapshotID string, s *storage.Snapshot, labels m
643646
}
644647

645648
var devices []string
649+
var parents []string
646650
// When merging bootstrap, we need to arrange layer bootstrap in order from low to high
647651
for idx := len(s.ParentIDs) - 1; idx >= 0; idx-- {
648652
snapshotID := s.ParentIDs[idx]
@@ -663,10 +667,89 @@ func (t *Manager) MountTarErofs(snapshotID string, s *storage.Snapshot, labels m
663667
st.dataLoopdev = loopdev
664668
}
665669
devices = append(devices, "device="+st.dataLoopdev.Name())
670+
parents = append(parents, snapshotID)
671+
}
672+
673+
st.mutex.Unlock()
674+
}
675+
parentList := strings.Join(parents, ",")
676+
devices = append(devices, "ro")
677+
mountOpts := strings.Join(devices, ",")
678+
679+
st, err := t.getSnapshotStatus(snapshotID, true)
680+
if err != nil {
681+
return err
682+
}
683+
defer st.mutex.Unlock()
684+
685+
mountPoint := path.Join(rafs.GetSnapshotDir(), "mnt")
686+
if len(st.erofsMountPoint) > 0 {
687+
if st.erofsMountPoint == mountPoint {
688+
log.L.Debugf("tarfs for snapshot %s has already been mounted at %s", snapshotID, mountPoint)
689+
return nil
690+
}
691+
return errors.Errorf("tarfs for snapshot %s has already been mounted at %s", snapshotID, st.erofsMountPoint)
692+
}
693+
694+
if st.metaLoopdev == nil {
695+
loopdev, err := t.attachLoopdev(mergedBootstrap)
696+
if err != nil {
697+
return errors.Wrapf(err, "attach merged bootstrap %s to loopdev", mergedBootstrap)
698+
}
699+
st.metaLoopdev = loopdev
700+
}
701+
devName := st.metaLoopdev.Name()
702+
703+
if err = os.MkdirAll(mountPoint, 0750); err != nil {
704+
return errors.Wrapf(err, "create tarfs mount dir %s", mountPoint)
705+
}
706+
707+
err = unix.Mount(devName, mountPoint, "erofs", 0, mountOpts)
708+
if err != nil {
709+
return errors.Wrapf(err, "mount erofs at %s with opts %s", mountPoint, mountOpts)
710+
}
711+
st.erofsMountPoint = mountPoint
712+
rafs.SetMountpoint(mountPoint)
713+
rafs.AddAnnotation(label.NydusTarfsParents, parentList)
714+
return nil
715+
}
716+
717+
func (t *Manager) RemountErofs(snapshotID string, rafs *rafs.Rafs) error {
718+
upperDirPath := path.Join(rafs.GetSnapshotDir(), "fs")
719+
720+
log.L.Infof("remount EROFS for tarfs snapshot %s at %s", snapshotID, upperDirPath)
721+
var parents []string
722+
if parentList, ok := rafs.Annotations[label.NydusTarfsParents]; ok {
723+
parents = strings.Split(parentList, ",")
724+
} else {
725+
if !config.GetTarfsMountOnHost() {
726+
rafs.SetMountpoint(upperDirPath)
666727
}
728+
return nil
729+
}
730+
731+
var devices []string
732+
for idx := 0; idx < len(parents); idx++ {
733+
snapshotID := parents[idx]
734+
st, err := t.waitLayerReady(snapshotID, true)
735+
if err != nil {
736+
return errors.Wrapf(err, "wait for tarfs conversion task")
737+
}
738+
739+
if st.dataLoopdev == nil {
740+
blobTarFilePath := t.layerTarFilePath(st.blobID)
741+
loopdev, err := t.attachLoopdev(blobTarFilePath)
742+
if err != nil {
743+
st.mutex.Unlock()
744+
return errors.Wrapf(err, "attach layer tar file %s to loopdev", blobTarFilePath)
745+
}
746+
st.dataLoopdev = loopdev
747+
}
748+
devices = append(devices, "device="+st.dataLoopdev.Name())
667749

668750
st.mutex.Unlock()
669751
}
752+
devices = append(devices, "ro")
670753
mountOpts := strings.Join(devices, ",")
671754

672755
st, err := t.getSnapshotStatus(snapshotID, true)
@@ -685,6 +768,7 @@ func (t *Manager) MountTarErofs(snapshotID string, s *storage.Snapshot, labels m
685768
}
686769

687770
if st.metaLoopdev == nil {
771+
mergedBootstrap := t.imageMetaFilePath(upperDirPath)
688772
loopdev, err := t.attachLoopdev(mergedBootstrap)
689773
if err != nil {
690774
return errors.Wrapf(err, "attach merged bootstrap %s to loopdev", mergedBootstrap)
@@ -767,6 +851,43 @@ func (t *Manager) DetachLayer(snapshotID string) error {
767851
return nil
768852
}
769853

854+
func (t *Manager) RecoverSnapshoInfo(ctx context.Context, id string, info snapshots.Info, upperPath string) error {
855+
t.mutex.Lock()
856+
defer t.mutex.Unlock()
857+
log.L.Infof("recover tarfs snapshot %s with path %s", id, upperPath)
858+
859+
if _, ok := t.snapshotMap[id]; ok {
860+
// RecoverSnapshotInfo() is called after RecoverRafsInstance(), so there may be some snapshots already exist.
861+
return nil
862+
}
863+
864+
layerMetaFilePath := t.layerMetaFilePath(upperPath)
865+
if _, err := os.Stat(layerMetaFilePath); err == nil {
866+
layerDigest := digest.Digest(info.Labels[label.CRILayerDigest])
867+
if layerDigest.Validate() != nil {
868+
return errors.Errorf("not found layer digest label")
869+
}
870+
ctx, cancel := context.WithCancel(context.Background())
871+
t.snapshotMap[id] = &snapshotStatus{
872+
status: TarfsStatusReady,
873+
blobID: layerDigest.Hex(),
874+
cancel: cancel,
875+
ctx: ctx,
876+
}
877+
} else {
878+
ctx, cancel := context.WithCancel(context.Background())
879+
wg := &sync.WaitGroup{}
880+
wg.Add(1)
881+
t.snapshotMap[id] = &snapshotStatus{
882+
status: TarfsStatusFailed,
883+
wg: wg,
884+
cancel: cancel,
885+
ctx: ctx,
886+
}
887+
}
888+
return nil
889+
}
890+
770891
// This method is called in single threaded mode during startup, so we do not lock `snapshotStatus` objects.
771892
func (t *Manager) RecoverRafsInstance(r *rafs.Rafs) error {
772893
t.mutex.Lock()

snapshot/process.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -157,7 +157,7 @@ func chooseProcessor(ctx context.Context, logger *logrus.Entry,
157157
// which have already been prepared by overlay snapshotter
158158

159159
logger.Infof("Prepare active snapshot %s in Nydus tarfs mode", key)
160-
err = sn.mergeTarfs(ctx, s, parent, pID, pInfo)
160+
err = sn.mergeTarfs(ctx, s, parent, pInfo)
161161
if err != nil {
162162
return nil, "", errors.Wrapf(err, "merge tarfs layers for snapshot %s", pID)
163163
}

snapshot/snapshot.go

Lines changed: 41 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@ import (
3434
"github.com/containerd/nydus-snapshotter/pkg/metrics"
3535
"github.com/containerd/nydus-snapshotter/pkg/metrics/collector"
3636
"github.com/containerd/nydus-snapshotter/pkg/pprof"
37+
"github.com/containerd/nydus-snapshotter/pkg/rafs"
3738
"github.com/containerd/nydus-snapshotter/pkg/referrer"
3839
"github.com/containerd/nydus-snapshotter/pkg/system"
3940
"github.com/containerd/nydus-snapshotter/pkg/tarfs"
@@ -225,8 +226,9 @@ func NewSnapshotter(ctx context.Context, cfg *config.SnapshotterConfig) (snapsho
225226
opts = append(opts, filesystem.WithReferrerManager(referrerMgr))
226227
}
227228

229+
var tarfsMgr *tarfs.Manager
228230
if cfg.Experimental.TarfsConfig.EnableTarfs {
229-
tarfsMgr := tarfs.NewManager(skipSSLVerify, cfg.Experimental.TarfsConfig.TarfsHint,
231+
tarfsMgr = tarfs.NewManager(skipSSLVerify, cfg.Experimental.TarfsConfig.TarfsHint,
230232
cacheConfig.CacheDir, cfg.DaemonConfig.NydusImagePath,
231233
int64(cfg.Experimental.TarfsConfig.MaxConcurrentProc))
232234
opts = append(opts, filesystem.WithTarfsManager(tarfsMgr))
@@ -284,7 +286,7 @@ func NewSnapshotter(ctx context.Context, cfg *config.SnapshotterConfig) (snapsho
284286
syncRemove = true
285287
}
286288

287-
return &snapshotter{
289+
snapshotter := &snapshotter{
288290
root: cfg.Root,
289291
nydusdPath: cfg.DaemonConfig.NydusdPath,
290292
ms: ms,
@@ -294,7 +296,41 @@ func NewSnapshotter(ctx context.Context, cfg *config.SnapshotterConfig) (snapsho
294296
enableNydusOverlayFS: cfg.SnapshotsConfig.EnableNydusOverlayFS,
295297
enableKataVolume: cfg.SnapshotsConfig.EnableKataVolume,
296298
cleanupOnClose: cfg.CleanupOnClose,
297-
}, nil
299+
}
300+
301+
// There's special requirement to recover tarfs instance because it depdens on `snapshotter.ms`
302+
// so it can't be done in `Filesystem.Recover()`
303+
if tarfsMgr != nil {
304+
snapshotter.recoverTarfs(ctx, tarfsMgr)
305+
}
306+
307+
return snapshotter, nil
308+
}
309+
310+
func (o *snapshotter) recoverTarfs(ctx context.Context, tarfsMgr *tarfs.Manager) {
311+
// First recover all snapshot information related to tarfs, mount operation depends on snapshots.
312+
_ = o.Walk(ctx, func(ctx context.Context, i snapshots.Info) error {
313+
if _, ok := i.Labels[label.NydusTarfsLayer]; ok {
314+
id, _, _, err := snapshot.GetSnapshotInfo(ctx, o.ms, i.Name)
315+
if err != nil {
316+
return errors.Wrapf(err, "get id for snapshot %s", i.Name)
317+
}
318+
log.L.Infof("found tarfs snapshot %s with name %s", id, i.Name)
319+
upperPath := o.upperPath(id)
320+
if err = tarfsMgr.RecoverSnapshoInfo(ctx, id, i, upperPath); err != nil {
321+
return errors.Wrapf(err, "get id for snapshot %s", i.Name)
322+
}
323+
}
324+
return nil
325+
})
326+
327+
for id, r := range tarfsMgr.RemountMap {
328+
log.L.Infof("remount tarfs snapshot %s", id)
329+
if err := tarfsMgr.RemountErofs(id, r); err != nil {
330+
log.L.Warnf("failed to remount EROFS filesystem for tarfs, %s", err)
331+
}
332+
}
333+
tarfsMgr.RemountMap = map[string]*rafs.Rafs{}
298334
}
299335

300336
func (o *snapshotter) Cleanup(ctx context.Context) error {
@@ -519,7 +555,7 @@ func (o *snapshotter) View(ctx context.Context, key, parent string, opts ...snap
519555

520556
if o.fs.TarfsEnabled() && label.IsTarfsDataLayer(pInfo.Labels) {
521557
log.L.Infof("Prepare view snapshot %s in Nydus tarfs mode", pID)
522-
err = o.mergeTarfs(ctx, s, parent, pID, pInfo)
558+
err = o.mergeTarfs(ctx, s, parent, pInfo)
523559
if err != nil {
524560
return nil, errors.Wrapf(err, "merge tarfs layers for snapshot %s", pID)
525561
}
@@ -796,7 +832,7 @@ func (o *snapshotter) createSnapshot(ctx context.Context, kind snapshots.Kind, k
796832
return &base, s, nil
797833
}
798834

799-
func (o *snapshotter) mergeTarfs(ctx context.Context, s storage.Snapshot, parent, pID string, pInfo snapshots.Info) error {
835+
func (o *snapshotter) mergeTarfs(ctx context.Context, s storage.Snapshot, parent string, pInfo snapshots.Info) error {
800836
infoGetter := func(ctx context.Context, id string) (string, snapshots.Info, error) {
801837
for {
802838
id2, info, _, err := snapshot.GetSnapshotInfo(ctx, o.ms, parent)

0 commit comments

Comments
 (0)