cleanup: move pkg/ to internal/

The internal/ directory in Go has a special meaning, and indicates that
those packages are not meant for external consumption. Ceph-CSI does
provide public APIs for other projects to consume. There is no plan to
keep the API of the internally used packages stable.

Closes: #903
Signed-off-by: Niels de Vos <ndevos@redhat.com>
This commit is contained in:
Niels de Vos
2020-04-17 11:23:49 +02:00
committed by mergify[bot]
parent d0abc3f5e6
commit 32839948ef
64 changed files with 37 additions and 37 deletions

View File

@ -0,0 +1,818 @@
/*
Copyright 2018 The Ceph-CSI Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package rbd
import (
"context"
"fmt"
csicommon "github.com/ceph/ceph-csi/internal/csi-common"
"github.com/ceph/ceph-csi/internal/util"
"github.com/container-storage-interface/spec/lib/go/csi"
"github.com/kubernetes-csi/csi-lib-utils/protosanitizer"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"k8s.io/klog"
)
const (
oneGB = 1073741824
)
// ControllerServer struct of rbd CSI driver with supported methods of CSI
// controller server spec.
type ControllerServer struct {
*csicommon.DefaultControllerServer
MetadataStore util.CachePersister
// A map storing all volumes with ongoing operations so that additional operations
// for that same volume (as defined by VolumeID/volume name) return an Aborted error
VolumeLocks *util.VolumeLocks
// A map storing all volumes with ongoing operations so that additional operations
// for that same snapshot (as defined by SnapshotID/snapshot name) return an Aborted error
SnapshotLocks *util.VolumeLocks
}
func (cs *ControllerServer) validateVolumeReq(ctx context.Context, req *csi.CreateVolumeRequest) error {
if err := cs.Driver.ValidateControllerServiceRequest(csi.ControllerServiceCapability_RPC_CREATE_DELETE_VOLUME); err != nil {
klog.Errorf(util.Log(ctx, "invalid create volume req: %v"), protosanitizer.StripSecrets(req))
return err
}
// Check sanity of request Name, Volume Capabilities
if req.Name == "" {
return status.Error(codes.InvalidArgument, "volume Name cannot be empty")
}
if req.VolumeCapabilities == nil {
return status.Error(codes.InvalidArgument, "volume Capabilities cannot be empty")
}
options := req.GetParameters()
if value, ok := options["clusterID"]; !ok || value == "" {
return status.Error(codes.InvalidArgument, "missing or empty cluster ID to provision volume from")
}
if value, ok := options["pool"]; !ok || value == "" {
return status.Error(codes.InvalidArgument, "missing or empty pool name to provision volume from")
}
if value, ok := options["dataPool"]; ok && value == "" {
return status.Error(codes.InvalidArgument, "empty datapool name to provision volume from")
}
if value, ok := options["volumeNamePrefix"]; ok && value == "" {
return status.Error(codes.InvalidArgument, "empty volume name prefix to provision volume from")
}
return nil
}
func (cs *ControllerServer) parseVolCreateRequest(ctx context.Context, req *csi.CreateVolumeRequest) (*rbdVolume, error) {
// TODO (sbezverk) Last check for not exceeding total storage capacity
isMultiNode := false
isBlock := false
for _, cap := range req.VolumeCapabilities {
// RO modes need to be handled independently (ie right now even if access mode is RO, they'll be RW upon attach)
if cap.GetAccessMode().GetMode() == csi.VolumeCapability_AccessMode_MULTI_NODE_MULTI_WRITER {
isMultiNode = true
}
if cap.GetBlock() != nil {
isBlock = true
}
}
// We want to fail early if the user is trying to create a RWX on a non-block type device
if isMultiNode && !isBlock {
return nil, status.Error(codes.InvalidArgument, "multi node access modes are only supported on rbd `block` type volumes")
}
// if it's NOT SINGLE_NODE_WRITER and it's BLOCK we'll set the parameter to ignore the in-use checks
rbdVol, err := genVolFromVolumeOptions(ctx, req.GetParameters(), req.GetSecrets(), (isMultiNode && isBlock), false)
if err != nil {
return nil, status.Error(codes.InvalidArgument, err.Error())
}
rbdVol.RequestName = req.GetName()
// Volume Size - Default is 1 GiB
volSizeBytes := int64(oneGB)
if req.GetCapacityRange() != nil {
volSizeBytes = req.GetCapacityRange().GetRequiredBytes()
}
// always round up the request size in bytes to the nearest MiB/GiB
rbdVol.VolSize = util.RoundOffBytes(volSizeBytes)
// start with pool the same as journal pool, in case there is a topology
// based split, pool for the image will be updated subsequently
rbdVol.JournalPool = rbdVol.Pool
// store topology information from the request
rbdVol.TopologyPools, rbdVol.TopologyRequirement, err = util.GetTopologyFromRequest(req)
if err != nil {
return nil, status.Error(codes.InvalidArgument, err.Error())
}
// NOTE: rbdVol does not contain VolID and RbdImageName populated, everything
// else is populated post create request parsing
return rbdVol, nil
}
// CreateVolume creates the volume in backend
func (cs *ControllerServer) CreateVolume(ctx context.Context, req *csi.CreateVolumeRequest) (*csi.CreateVolumeResponse, error) {
if err := cs.validateVolumeReq(ctx, req); err != nil {
return nil, err
}
// TODO: create/get a connection from the the ConnPool, and do not pass
// the credentials to any of the utility functions.
cr, err := util.NewUserCredentials(req.GetSecrets())
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
defer cr.DeleteCredentials()
rbdVol, err := cs.parseVolCreateRequest(ctx, req)
if err != nil {
return nil, err
}
defer rbdVol.Destroy()
// Existence and conflict checks
if acquired := cs.VolumeLocks.TryAcquire(req.GetName()); !acquired {
klog.Errorf(util.Log(ctx, util.VolumeOperationAlreadyExistsFmt), req.GetName())
return nil, status.Errorf(codes.Aborted, util.VolumeOperationAlreadyExistsFmt, req.GetName())
}
defer cs.VolumeLocks.Release(req.GetName())
found, err := checkVolExists(ctx, rbdVol, cr)
if err != nil {
if _, ok := err.(ErrVolNameConflict); ok {
return nil, status.Error(codes.AlreadyExists, err.Error())
}
return nil, status.Error(codes.Internal, err.Error())
}
if found {
if rbdVol.Encrypted {
err = ensureEncryptionMetadataSet(ctx, cr, rbdVol)
if err != nil {
klog.Errorf(util.Log(ctx, err.Error()))
return nil, err
}
}
volumeContext := req.GetParameters()
volumeContext["pool"] = rbdVol.Pool
volumeContext["journalPool"] = rbdVol.JournalPool
volumeContext["imageName"] = rbdVol.RbdImageName
volume := &csi.Volume{
VolumeId: rbdVol.VolID,
CapacityBytes: rbdVol.VolSize,
VolumeContext: volumeContext,
ContentSource: req.GetVolumeContentSource(),
}
if rbdVol.Topology != nil {
volume.AccessibleTopology =
[]*csi.Topology{
{
Segments: rbdVol.Topology,
},
}
}
return &csi.CreateVolumeResponse{Volume: volume}, nil
}
rbdSnap, err := cs.checkSnapshotSource(ctx, req, cr)
if err != nil {
return nil, err
}
err = reserveVol(ctx, rbdVol, rbdSnap, cr)
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
defer func() {
if err != nil {
errDefer := undoVolReservation(ctx, rbdVol, cr)
if errDefer != nil {
klog.Warningf(util.Log(ctx, "failed undoing reservation of volume: %s (%s)"), req.GetName(), errDefer)
}
}
}()
err = createBackingImage(ctx, cr, rbdVol, rbdSnap)
if err != nil {
return nil, err
}
if rbdVol.Encrypted {
err = ensureEncryptionMetadataSet(ctx, cr, rbdVol)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to save encryption status, deleting image %s"),
rbdVol.RbdImageName)
if deleteErr := deleteImage(ctx, rbdVol, cr); err != nil {
klog.Errorf(util.Log(ctx, "failed to delete rbd image: %s/%s with error: %v"),
rbdVol.Pool, rbdVol.RbdImageName, deleteErr)
return nil, deleteErr
}
return nil, err
}
}
volumeContext := req.GetParameters()
volumeContext["pool"] = rbdVol.Pool
volumeContext["journalPool"] = rbdVol.JournalPool
volumeContext["imageName"] = rbdVol.RbdImageName
volume := &csi.Volume{
VolumeId: rbdVol.VolID,
CapacityBytes: rbdVol.VolSize,
VolumeContext: volumeContext,
ContentSource: req.GetVolumeContentSource(),
}
if rbdVol.Topology != nil {
volume.AccessibleTopology =
[]*csi.Topology{
{
Segments: rbdVol.Topology,
},
}
}
return &csi.CreateVolumeResponse{Volume: volume}, nil
}
func createBackingImage(ctx context.Context, cr *util.Credentials, rbdVol *rbdVolume, rbdSnap *rbdSnapshot) error {
var err error
if rbdSnap != nil {
err = restoreSnapshot(ctx, rbdVol, rbdSnap, cr)
if err != nil {
return err
}
klog.V(4).Infof(util.Log(ctx, "created volume %s from snapshot %s"), rbdVol.RequestName, rbdSnap.RbdSnapName)
return nil
}
err = createImage(ctx, rbdVol, cr)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to create volume: %v"), err)
return status.Error(codes.Internal, err.Error())
}
klog.V(4).Infof(util.Log(ctx, "created volume %s backed by image %s"), rbdVol.RequestName, rbdVol.RbdImageName)
return nil
}
func (cs *ControllerServer) checkSnapshotSource(ctx context.Context, req *csi.CreateVolumeRequest,
cr *util.Credentials) (*rbdSnapshot, error) {
if req.VolumeContentSource == nil {
return nil, nil
}
snapshot := req.VolumeContentSource.GetSnapshot()
if snapshot == nil {
return nil, status.Error(codes.InvalidArgument, "volume Snapshot cannot be empty")
}
snapshotID := snapshot.GetSnapshotId()
if snapshotID == "" {
return nil, status.Error(codes.InvalidArgument, "volume Snapshot ID cannot be empty")
}
rbdSnap := &rbdSnapshot{}
if err := genSnapFromSnapID(ctx, rbdSnap, snapshotID, cr); err != nil {
if _, ok := err.(ErrSnapNotFound); !ok {
return nil, status.Error(codes.Internal, err.Error())
}
if _, ok := err.(util.ErrPoolNotFound); ok {
klog.Errorf(util.Log(ctx, "failed to get backend snapshot for %s: %v"), snapshotID, err)
return nil, status.Error(codes.InvalidArgument, err.Error())
}
return nil, status.Error(codes.InvalidArgument, "missing requested Snapshot ID")
}
return rbdSnap, nil
}
// DeleteLegacyVolume deletes a volume provisioned using version 1.0.0 of the plugin
func (cs *ControllerServer) DeleteLegacyVolume(ctx context.Context, req *csi.DeleteVolumeRequest, cr *util.Credentials) (*csi.DeleteVolumeResponse, error) {
volumeID := req.GetVolumeId()
if cs.MetadataStore == nil {
return nil, status.Errorf(codes.InvalidArgument, "missing metadata store configuration to"+
" proceed with deleting legacy volume ID (%s)", volumeID)
}
if acquired := cs.VolumeLocks.TryAcquire(volumeID); !acquired {
klog.Errorf(util.Log(ctx, util.VolumeOperationAlreadyExistsFmt), volumeID)
return nil, status.Errorf(codes.Aborted, util.VolumeOperationAlreadyExistsFmt, volumeID)
}
defer cs.VolumeLocks.Release(volumeID)
rbdVol := &rbdVolume{}
defer rbdVol.Destroy()
if err := cs.MetadataStore.Get(volumeID, rbdVol); err != nil {
if err, ok := err.(*util.CacheEntryNotFound); ok {
klog.Warningf(util.Log(ctx, "metadata for legacy volume %s not found, assuming the volume to be already deleted (%v)"), volumeID, err)
return &csi.DeleteVolumeResponse{}, nil
}
return nil, status.Error(codes.Internal, err.Error())
}
// Fill up Monitors
if err := updateMons(rbdVol, nil, req.GetSecrets()); err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
// Update rbdImageName as the VolName when dealing with version 1 volumes
rbdVol.RbdImageName = rbdVol.VolName
klog.V(4).Infof(util.Log(ctx, "deleting legacy volume %s"), rbdVol.VolName)
if err := deleteImage(ctx, rbdVol, cr); err != nil {
// TODO: can we detect "already deleted" situations here and proceed?
klog.Errorf(util.Log(ctx, "failed to delete legacy rbd image: %s/%s with error: %v"), rbdVol.Pool, rbdVol.VolName, err)
return nil, status.Error(codes.Internal, err.Error())
}
if err := cs.MetadataStore.Delete(volumeID); err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
return &csi.DeleteVolumeResponse{}, nil
}
// DeleteVolume deletes the volume in backend and removes the volume metadata
// from store
func (cs *ControllerServer) DeleteVolume(ctx context.Context, req *csi.DeleteVolumeRequest) (*csi.DeleteVolumeResponse, error) {
if err := cs.Driver.ValidateControllerServiceRequest(csi.ControllerServiceCapability_RPC_CREATE_DELETE_VOLUME); err != nil {
klog.Errorf(util.Log(ctx, "invalid delete volume req: %v"), protosanitizer.StripSecrets(req))
return nil, err
}
cr, err := util.NewUserCredentials(req.GetSecrets())
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
defer cr.DeleteCredentials()
// For now the image get unconditionally deleted, but here retention policy can be checked
volumeID := req.GetVolumeId()
if volumeID == "" {
return nil, status.Error(codes.InvalidArgument, "empty volume ID in request")
}
if acquired := cs.VolumeLocks.TryAcquire(volumeID); !acquired {
klog.Errorf(util.Log(ctx, util.VolumeOperationAlreadyExistsFmt), volumeID)
return nil, status.Errorf(codes.Aborted, util.VolumeOperationAlreadyExistsFmt, volumeID)
}
defer cs.VolumeLocks.Release(volumeID)
rbdVol := &rbdVolume{}
defer rbdVol.Destroy()
if err = genVolFromVolID(ctx, rbdVol, volumeID, cr, req.GetSecrets()); err != nil {
if _, ok := err.(util.ErrPoolNotFound); ok {
klog.Warningf(util.Log(ctx, "failed to get backend volume for %s: %v"), volumeID, err)
return &csi.DeleteVolumeResponse{}, nil
}
// If error is ErrInvalidVolID it could be a version 1.0.0 or lower volume, attempt
// to process it as such
if _, ok := err.(ErrInvalidVolID); ok {
if isLegacyVolumeID(volumeID) {
klog.V(2).Infof(util.Log(ctx, "attempting deletion of potential legacy volume (%s)"), volumeID)
return cs.DeleteLegacyVolume(ctx, req, cr)
}
// Consider unknown volumeID as a successfully deleted volume
return &csi.DeleteVolumeResponse{}, nil
}
// if error is ErrKeyNotFound, then a previous attempt at deletion was complete
// or partially complete (image and imageOMap are garbage collected already), hence return
// success as deletion is complete
if _, ok := err.(util.ErrKeyNotFound); ok {
klog.Warningf(util.Log(ctx, "Failed to volume options for %s: %v"), volumeID, err)
return &csi.DeleteVolumeResponse{}, nil
}
// All errors other than ErrImageNotFound should return an error back to the caller
if _, ok := err.(ErrImageNotFound); !ok {
return nil, status.Error(codes.Internal, err.Error())
}
// If error is ErrImageNotFound then we failed to find the image, but found the imageOMap
// to lead us to the image, hence the imageOMap needs to be garbage collected, by calling
// unreserve for the same
if acquired := cs.VolumeLocks.TryAcquire(rbdVol.RequestName); !acquired {
klog.Errorf(util.Log(ctx, util.VolumeOperationAlreadyExistsFmt), rbdVol.RequestName)
return nil, status.Errorf(codes.Aborted, util.VolumeOperationAlreadyExistsFmt, rbdVol.RequestName)
}
defer cs.VolumeLocks.Release(rbdVol.RequestName)
if err = undoVolReservation(ctx, rbdVol, cr); err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
return &csi.DeleteVolumeResponse{}, nil
}
// lock out parallel create requests against the same volume name as we
// cleanup the image and associated omaps for the same
if acquired := cs.VolumeLocks.TryAcquire(rbdVol.RequestName); !acquired {
klog.Errorf(util.Log(ctx, util.VolumeOperationAlreadyExistsFmt), rbdVol.RequestName)
return nil, status.Errorf(codes.Aborted, util.VolumeOperationAlreadyExistsFmt, rbdVol.RequestName)
}
defer cs.VolumeLocks.Release(rbdVol.RequestName)
// Deleting rbd image
klog.V(4).Infof(util.Log(ctx, "deleting image %s"), rbdVol.RbdImageName)
if err = deleteImage(ctx, rbdVol, cr); err != nil {
klog.Errorf(util.Log(ctx, "failed to delete rbd image: %s/%s with error: %v"),
rbdVol.Pool, rbdVol.RbdImageName, err)
return nil, status.Error(codes.Internal, err.Error())
}
if err = undoVolReservation(ctx, rbdVol, cr); err != nil {
klog.Errorf(util.Log(ctx, "failed to remove reservation for volume (%s) with backing image (%s) (%s)"),
rbdVol.RequestName, rbdVol.RbdImageName, err)
return nil, status.Error(codes.Internal, err.Error())
}
if rbdVol.Encrypted {
if err = rbdVol.KMS.DeletePassphrase(rbdVol.VolID); err != nil {
klog.Warningf(util.Log(ctx, "failed to clean the passphrase for volume %s: %s"), rbdVol.VolID, err)
}
}
return &csi.DeleteVolumeResponse{}, nil
}
// ValidateVolumeCapabilities checks whether the volume capabilities requested
// are supported.
func (cs *ControllerServer) ValidateVolumeCapabilities(ctx context.Context, req *csi.ValidateVolumeCapabilitiesRequest) (*csi.ValidateVolumeCapabilitiesResponse, error) {
if req.GetVolumeId() == "" {
return nil, status.Error(codes.InvalidArgument, "empty volume ID in request")
}
if len(req.VolumeCapabilities) == 0 {
return nil, status.Error(codes.InvalidArgument, "empty volume capabilities in request")
}
for _, cap := range req.VolumeCapabilities {
if cap.GetAccessMode().GetMode() != csi.VolumeCapability_AccessMode_SINGLE_NODE_WRITER {
return &csi.ValidateVolumeCapabilitiesResponse{Message: ""}, nil
}
}
return &csi.ValidateVolumeCapabilitiesResponse{
Confirmed: &csi.ValidateVolumeCapabilitiesResponse_Confirmed{
VolumeCapabilities: req.VolumeCapabilities,
},
}, nil
}
// CreateSnapshot creates the snapshot in backend and stores metadata
// in store
// nolint: gocyclo
func (cs *ControllerServer) CreateSnapshot(ctx context.Context, req *csi.CreateSnapshotRequest) (*csi.CreateSnapshotResponse, error) {
if err := cs.validateSnapshotReq(ctx, req); err != nil {
return nil, err
}
cr, err := util.NewUserCredentials(req.GetSecrets())
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
defer cr.DeleteCredentials()
// Fetch source volume information
rbdVol := new(rbdVolume)
err = genVolFromVolID(ctx, rbdVol, req.GetSourceVolumeId(), cr, req.GetSecrets())
if err != nil {
if _, ok := err.(ErrImageNotFound); ok {
return nil, status.Errorf(codes.NotFound, "source Volume ID %s not found", req.GetSourceVolumeId())
}
if _, ok := err.(util.ErrPoolNotFound); ok {
klog.Errorf(util.Log(ctx, "failed to get backend volume for %s: %v"), req.GetSourceVolumeId(), err)
return nil, status.Errorf(codes.Internal, err.Error())
}
return nil, status.Errorf(codes.Internal, err.Error())
}
// TODO: re-encrypt snapshot with a new passphrase
if rbdVol.Encrypted {
return nil, status.Errorf(codes.Unimplemented, "source Volume %s is encrypted, "+
"snapshotting is not supported currently", rbdVol.VolID)
}
// Check if source volume was created with required image features for snaps
if !hasSnapshotFeature(rbdVol.ImageFeatures) {
return nil, status.Errorf(codes.InvalidArgument, "volume(%s) has not snapshot feature(layering)", req.GetSourceVolumeId())
}
// Create snap volume
rbdSnap := genSnapFromOptions(ctx, rbdVol, req.GetParameters())
rbdSnap.RbdImageName = rbdVol.RbdImageName
rbdSnap.SizeBytes = rbdVol.VolSize
rbdSnap.SourceVolumeID = req.GetSourceVolumeId()
rbdSnap.RequestName = req.GetName()
if acquired := cs.SnapshotLocks.TryAcquire(req.GetName()); !acquired {
klog.Errorf(util.Log(ctx, util.SnapshotOperationAlreadyExistsFmt), req.GetName())
return nil, status.Errorf(codes.Aborted, util.VolumeOperationAlreadyExistsFmt, req.GetName())
}
defer cs.SnapshotLocks.Release(req.GetName())
// Need to check for already existing snapshot name, and if found
// check for the requested source volume id and already allocated source volume id
found, err := checkSnapExists(ctx, rbdSnap, cr)
if err != nil {
if _, ok := err.(util.ErrSnapNameConflict); ok {
return nil, status.Error(codes.AlreadyExists, err.Error())
}
return nil, status.Errorf(codes.Internal, err.Error())
}
if found {
return &csi.CreateSnapshotResponse{
Snapshot: &csi.Snapshot{
SizeBytes: rbdSnap.SizeBytes,
SnapshotId: rbdSnap.SnapID,
SourceVolumeId: rbdSnap.SourceVolumeID,
CreationTime: rbdSnap.CreatedAt,
ReadyToUse: true,
},
}, nil
}
err = reserveSnap(ctx, rbdSnap, cr)
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
defer func() {
if err != nil {
errDefer := undoSnapReservation(ctx, rbdSnap, cr)
if errDefer != nil {
klog.Warningf(util.Log(ctx, "failed undoing reservation of snapshot: %s %v"), req.GetName(), errDefer)
}
}
}()
err = cs.doSnapshot(ctx, rbdSnap, cr)
if err != nil {
return nil, err
}
return &csi.CreateSnapshotResponse{
Snapshot: &csi.Snapshot{
SizeBytes: rbdSnap.SizeBytes,
SnapshotId: rbdSnap.SnapID,
SourceVolumeId: req.GetSourceVolumeId(),
CreationTime: rbdSnap.CreatedAt,
ReadyToUse: true,
},
}, nil
}
func (cs *ControllerServer) validateSnapshotReq(ctx context.Context, req *csi.CreateSnapshotRequest) error {
if err := cs.Driver.ValidateControllerServiceRequest(csi.ControllerServiceCapability_RPC_CREATE_DELETE_SNAPSHOT); err != nil {
klog.Errorf(util.Log(ctx, "invalid create snapshot req: %v"), protosanitizer.StripSecrets(req))
return err
}
// Check sanity of request Snapshot Name, Source Volume Id
if req.Name == "" {
return status.Error(codes.InvalidArgument, "snapshot Name cannot be empty")
}
if req.SourceVolumeId == "" {
return status.Error(codes.InvalidArgument, "source Volume ID cannot be empty")
}
options := req.GetParameters()
if value, ok := options["snapshotNamePrefix"]; ok && value == "" {
return status.Error(codes.InvalidArgument, "empty snapshot name prefix to provision snapshot from")
}
return nil
}
func (cs *ControllerServer) doSnapshot(ctx context.Context, rbdSnap *rbdSnapshot, cr *util.Credentials) (err error) {
err = createSnapshot(ctx, rbdSnap, cr)
// If snap creation fails, even due to snapname already used, fail, next attempt will get a new
// uuid for use as the snap name
if err != nil {
klog.Errorf(util.Log(ctx, "failed to create snapshot: %v"), err)
return status.Error(codes.Internal, err.Error())
}
defer func() {
if err != nil {
errDefer := deleteSnapshot(ctx, rbdSnap, cr)
if errDefer != nil {
klog.Errorf(util.Log(ctx, "failed to delete snapshot: %v"), errDefer)
err = fmt.Errorf("snapshot created but failed to delete snapshot due to"+
" other failures: %v", err)
}
err = status.Error(codes.Internal, err.Error())
}
}()
err = protectSnapshot(ctx, rbdSnap, cr)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to protect snapshot: %v"), err)
return status.Error(codes.Internal, err.Error())
}
defer func() {
if err != nil {
errDefer := unprotectSnapshot(ctx, rbdSnap, cr)
if errDefer != nil {
klog.Errorf(util.Log(ctx, "failed to unprotect snapshot: %v"), errDefer)
err = fmt.Errorf("snapshot created but failed to unprotect snapshot due to"+
" other failures: %v", err)
}
err = status.Error(codes.Internal, err.Error())
}
}()
err = getSnapshotMetadata(ctx, rbdSnap, cr)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to fetch snapshot metadata: %v"), err)
return status.Error(codes.Internal, err.Error())
}
return nil
}
// DeleteSnapshot deletes the snapshot in backend and removes the
// snapshot metadata from store
func (cs *ControllerServer) DeleteSnapshot(ctx context.Context, req *csi.DeleteSnapshotRequest) (*csi.DeleteSnapshotResponse, error) {
if err := cs.Driver.ValidateControllerServiceRequest(csi.ControllerServiceCapability_RPC_CREATE_DELETE_SNAPSHOT); err != nil {
klog.Errorf(util.Log(ctx, "invalid delete snapshot req: %v"), protosanitizer.StripSecrets(req))
return nil, err
}
cr, err := util.NewUserCredentials(req.GetSecrets())
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
defer cr.DeleteCredentials()
snapshotID := req.GetSnapshotId()
if snapshotID == "" {
return nil, status.Error(codes.InvalidArgument, "snapshot ID cannot be empty")
}
if acquired := cs.SnapshotLocks.TryAcquire(snapshotID); !acquired {
klog.Errorf(util.Log(ctx, util.SnapshotOperationAlreadyExistsFmt), snapshotID)
return nil, status.Errorf(codes.Aborted, util.VolumeOperationAlreadyExistsFmt, snapshotID)
}
defer cs.SnapshotLocks.Release(snapshotID)
rbdSnap := &rbdSnapshot{}
if err = genSnapFromSnapID(ctx, rbdSnap, snapshotID, cr); err != nil {
// if error is ErrPoolNotFound, the pool is already deleted we dont
// need to worry about deleting snapshot or omap data, return success
if _, ok := err.(util.ErrPoolNotFound); ok {
klog.Warningf(util.Log(ctx, "failed to get backend snapshot for %s: %v"), snapshotID, err)
return &csi.DeleteSnapshotResponse{}, nil
}
// if error is ErrKeyNotFound, then a previous attempt at deletion was complete
// or partially complete (snap and snapOMap are garbage collected already), hence return
// success as deletion is complete
if _, ok := err.(util.ErrKeyNotFound); ok {
return &csi.DeleteSnapshotResponse{}, nil
}
// All errors other than ErrSnapNotFound should return an error back to the caller
if _, ok := err.(ErrSnapNotFound); !ok {
return nil, status.Error(codes.Internal, err.Error())
}
// Consider missing snap as already deleted, and proceed to remove the omap values,
// safeguarding against parallel create or delete requests against the
// same name.
if acquired := cs.SnapshotLocks.TryAcquire(rbdSnap.RequestName); !acquired {
klog.Errorf(util.Log(ctx, util.SnapshotOperationAlreadyExistsFmt), rbdSnap.RequestName)
return nil, status.Errorf(codes.Aborted, util.VolumeOperationAlreadyExistsFmt, rbdSnap.RequestName)
}
defer cs.SnapshotLocks.Release(rbdSnap.RequestName)
if err = undoSnapReservation(ctx, rbdSnap, cr); err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
return &csi.DeleteSnapshotResponse{}, nil
}
// safeguard against parallel create or delete requests against the same
// name
if acquired := cs.SnapshotLocks.TryAcquire(rbdSnap.RequestName); !acquired {
klog.Errorf(util.Log(ctx, util.SnapshotOperationAlreadyExistsFmt), rbdSnap.RequestName)
return nil, status.Errorf(codes.Aborted, util.VolumeOperationAlreadyExistsFmt, rbdSnap.RequestName)
}
defer cs.SnapshotLocks.Release(rbdSnap.RequestName)
// Unprotect snapshot
err = unprotectSnapshot(ctx, rbdSnap, cr)
if err != nil {
return nil, status.Errorf(codes.FailedPrecondition,
"failed to unprotect snapshot: %s/%s with error: %v",
rbdSnap.Pool, rbdSnap.RbdSnapName, err)
}
// Deleting snapshot
klog.V(4).Infof(util.Log(ctx, "deleting Snaphot %s"), rbdSnap.RbdSnapName)
if err := deleteSnapshot(ctx, rbdSnap, cr); err != nil {
return nil, status.Errorf(codes.FailedPrecondition,
"failed to delete snapshot: %s/%s with error: %v",
rbdSnap.Pool, rbdSnap.RbdSnapName, err)
}
return &csi.DeleteSnapshotResponse{}, nil
}
// ControllerExpandVolume expand RBD Volumes on demand based on resizer request
func (cs *ControllerServer) ControllerExpandVolume(ctx context.Context, req *csi.ControllerExpandVolumeRequest) (*csi.ControllerExpandVolumeResponse, error) {
if err := cs.Driver.ValidateControllerServiceRequest(csi.ControllerServiceCapability_RPC_EXPAND_VOLUME); err != nil {
klog.Errorf(util.Log(ctx, "invalid expand volume req: %v"), protosanitizer.StripSecrets(req))
return nil, err
}
volID := req.GetVolumeId()
if volID == "" {
return nil, status.Error(codes.InvalidArgument, "volume ID cannot be empty")
}
capRange := req.GetCapacityRange()
if capRange == nil {
return nil, status.Error(codes.InvalidArgument, "capacityRange cannot be empty")
}
// lock out parallel requests against the same volume ID
if acquired := cs.VolumeLocks.TryAcquire(volID); !acquired {
klog.Errorf(util.Log(ctx, util.VolumeOperationAlreadyExistsFmt), volID)
return nil, status.Errorf(codes.Aborted, util.VolumeOperationAlreadyExistsFmt, volID)
}
defer cs.VolumeLocks.Release(volID)
cr, err := util.NewUserCredentials(req.GetSecrets())
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
defer cr.DeleteCredentials()
rbdVol := &rbdVolume{}
defer rbdVol.Destroy()
err = genVolFromVolID(ctx, rbdVol, volID, cr, req.GetSecrets())
if err != nil {
if _, ok := err.(ErrImageNotFound); ok {
return nil, status.Errorf(codes.NotFound, "volume ID %s not found", volID)
}
if _, ok := err.(util.ErrPoolNotFound); ok {
klog.Errorf(util.Log(ctx, "failed to get backend volume for %s: %v"), volID, err)
return nil, status.Errorf(codes.InvalidArgument, err.Error())
}
return nil, status.Errorf(codes.Internal, err.Error())
}
if rbdVol.Encrypted {
return nil, status.Errorf(codes.InvalidArgument, "encrypted volumes do not support resize (%s/%s)",
rbdVol.Pool, rbdVol.RbdImageName)
}
// always round up the request size in bytes to the nearest MiB/GiB
volSize := util.RoundOffBytes(req.GetCapacityRange().GetRequiredBytes())
// resize volume if required
nodeExpansion := false
if rbdVol.VolSize < volSize {
klog.V(4).Infof(util.Log(ctx, "rbd volume %s/%s size is %v,resizing to %v"), rbdVol.Pool, rbdVol.RbdImageName, rbdVol.VolSize, volSize)
rbdVol.VolSize = volSize
nodeExpansion = true
err = resizeRBDImage(rbdVol, cr)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to resize rbd image: %s/%s with error: %v"), rbdVol.Pool, rbdVol.RbdImageName, err)
return nil, status.Error(codes.Internal, err.Error())
}
}
return &csi.ControllerExpandVolumeResponse{
CapacityBytes: rbdVol.VolSize,
NodeExpansionRequired: nodeExpansion,
}, nil
}

170
internal/rbd/driver.go Normal file
View File

@ -0,0 +1,170 @@
/*
Copyright 2018 The Ceph-CSI Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package rbd
import (
csicommon "github.com/ceph/ceph-csi/internal/csi-common"
"github.com/ceph/ceph-csi/internal/util"
"github.com/container-storage-interface/spec/lib/go/csi"
"k8s.io/klog"
"k8s.io/utils/mount"
)
const (
// volIDVersion is the version number of volume ID encoding scheme
volIDVersion uint16 = 1
// csiConfigFile is the location of the CSI config file
csiConfigFile = "/etc/ceph-csi-config/config.json"
)
// Driver contains the default identity,node and controller struct
type Driver struct {
cd *csicommon.CSIDriver
ids *IdentityServer
ns *NodeServer
cs *ControllerServer
}
var (
// CSIInstanceID is the instance ID that is unique to an instance of CSI, used when sharing
// ceph clusters across CSI instances, to differentiate omap names per CSI instance
CSIInstanceID = "default"
// volJournal and snapJournal are used to maintain RADOS based journals for CO generated
// VolumeName to backing RBD images
volJournal *util.CSIJournal
snapJournal *util.CSIJournal
)
// NewDriver returns new rbd driver
func NewDriver() *Driver {
return &Driver{}
}
// NewIdentityServer initialize a identity server for rbd CSI driver
func NewIdentityServer(d *csicommon.CSIDriver) *IdentityServer {
return &IdentityServer{
DefaultIdentityServer: csicommon.NewDefaultIdentityServer(d),
}
}
// NewControllerServer initialize a controller server for rbd CSI driver
func NewControllerServer(d *csicommon.CSIDriver, cachePersister util.CachePersister) *ControllerServer {
return &ControllerServer{
DefaultControllerServer: csicommon.NewDefaultControllerServer(d),
MetadataStore: cachePersister,
VolumeLocks: util.NewVolumeLocks(),
SnapshotLocks: util.NewVolumeLocks(),
}
}
// NewNodeServer initialize a node server for rbd CSI driver.
func NewNodeServer(d *csicommon.CSIDriver, t string, topology map[string]string) (*NodeServer, error) {
mounter := mount.New("")
return &NodeServer{
DefaultNodeServer: csicommon.NewDefaultNodeServer(d, t, topology),
mounter: mounter,
VolumeLocks: util.NewVolumeLocks(),
}, nil
}
// Run start a non-blocking grpc controller,node and identityserver for
// rbd CSI driver which can serve multiple parallel requests
func (r *Driver) Run(conf *util.Config, cachePersister util.CachePersister) {
var err error
var topology map[string]string
// Create ceph.conf for use with CLI commands
if err = util.WriteCephConfig(); err != nil {
klog.Fatalf("failed to write ceph configuration file (%v)", err)
}
// Use passed in instance ID, if provided for omap suffix naming
if conf.InstanceID != "" {
CSIInstanceID = conf.InstanceID
}
// Get an instance of the volume and snapshot journal keys
volJournal = util.NewCSIVolumeJournal()
snapJournal = util.NewCSISnapshotJournal()
// Update keys with CSI instance suffix
volJournal.SetCSIDirectorySuffix(CSIInstanceID)
snapJournal.SetCSIDirectorySuffix(CSIInstanceID)
// Initialize default library driver
r.cd = csicommon.NewCSIDriver(conf.DriverName, util.DriverVersion, conf.NodeID)
if r.cd == nil {
klog.Fatalln("Failed to initialize CSI Driver.")
}
if conf.IsControllerServer || !conf.IsNodeServer {
r.cd.AddControllerServiceCapabilities([]csi.ControllerServiceCapability_RPC_Type{
csi.ControllerServiceCapability_RPC_CREATE_DELETE_VOLUME,
csi.ControllerServiceCapability_RPC_CREATE_DELETE_SNAPSHOT,
csi.ControllerServiceCapability_RPC_CLONE_VOLUME,
csi.ControllerServiceCapability_RPC_EXPAND_VOLUME,
})
// We only support the multi-writer option when using block, but it's a supported capability for the plugin in general
// In addition, we want to add the remaining modes like MULTI_NODE_READER_ONLY,
// MULTI_NODE_SINGLE_WRITER etc, but need to do some verification of RO modes first
// will work those as follow up features
r.cd.AddVolumeCapabilityAccessModes(
[]csi.VolumeCapability_AccessMode_Mode{csi.VolumeCapability_AccessMode_SINGLE_NODE_WRITER,
csi.VolumeCapability_AccessMode_MULTI_NODE_MULTI_WRITER})
}
// Create GRPC servers
r.ids = NewIdentityServer(r.cd)
if conf.IsNodeServer {
topology, err = util.GetTopologyFromDomainLabels(conf.DomainLabels, conf.NodeID, conf.DriverName)
if err != nil {
klog.Fatalln(err)
}
r.ns, err = NewNodeServer(r.cd, conf.Vtype, topology)
if err != nil {
klog.Fatalf("failed to start node server, err %v\n", err)
}
}
if conf.IsControllerServer {
r.cs = NewControllerServer(r.cd, cachePersister)
}
if !conf.IsControllerServer && !conf.IsNodeServer {
topology, err = util.GetTopologyFromDomainLabels(conf.DomainLabels, conf.NodeID, conf.DriverName)
if err != nil {
klog.Fatalln(err)
}
r.ns, err = NewNodeServer(r.cd, conf.Vtype, topology)
if err != nil {
klog.Fatalf("failed to start node server, err %v\n", err)
}
r.cs = NewControllerServer(r.cd, cachePersister)
}
s := csicommon.NewNonBlockingGRPCServer()
s.Start(conf.Endpoint, conf.HistogramOption, r.ids, r.cs, r.ns, conf.EnableGRPCMetrics)
if conf.EnableGRPCMetrics {
klog.Warning("EnableGRPCMetrics is deprecated")
go util.StartMetricsServer(conf)
}
s.Wait()
}

68
internal/rbd/errors.go Normal file
View File

@ -0,0 +1,68 @@
/*
Copyright 2019 The Ceph-CSI Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package rbd
// ErrImageNotFound is returned when image name is not found in the cluster on the given pool
type ErrImageNotFound struct {
imageName string
err error
}
func (e ErrImageNotFound) Error() string {
return e.err.Error()
}
// ErrSnapNotFound is returned when snap name passed is not found in the list of snapshots for the
// given image
type ErrSnapNotFound struct {
snapName string
err error
}
func (e ErrSnapNotFound) Error() string {
return e.err.Error()
}
// ErrVolNameConflict is generated when a requested CSI volume name already exists on RBD but with
// different properties, and hence is in conflict with the passed in CSI volume name
type ErrVolNameConflict struct {
requestName string
err error
}
func (e ErrVolNameConflict) Error() string {
return e.err.Error()
}
// ErrInvalidVolID is returned when a CSI passed VolumeID does not conform to any known volume ID
// formats
type ErrInvalidVolID struct {
err error
}
func (e ErrInvalidVolID) Error() string {
return e.err.Error()
}
// ErrMissingStash is returned when the image metadata stash file is not found
type ErrMissingStash struct {
err error
}
func (e ErrMissingStash) Error() string {
return e.err.Error()
}

View File

@ -0,0 +1,60 @@
/*
Copyright 2018 The Ceph-CSI Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package rbd
import (
"context"
csicommon "github.com/ceph/ceph-csi/internal/csi-common"
"github.com/container-storage-interface/spec/lib/go/csi"
)
// IdentityServer struct of rbd CSI driver with supported methods of CSI
// identity server spec.
type IdentityServer struct {
*csicommon.DefaultIdentityServer
}
// GetPluginCapabilities returns available capabilities of the rbd driver
func (is *IdentityServer) GetPluginCapabilities(ctx context.Context, req *csi.GetPluginCapabilitiesRequest) (*csi.GetPluginCapabilitiesResponse, error) {
return &csi.GetPluginCapabilitiesResponse{
Capabilities: []*csi.PluginCapability{
{
Type: &csi.PluginCapability_Service_{
Service: &csi.PluginCapability_Service{
Type: csi.PluginCapability_Service_CONTROLLER_SERVICE,
},
},
},
{
Type: &csi.PluginCapability_VolumeExpansion_{
VolumeExpansion: &csi.PluginCapability_VolumeExpansion{
Type: csi.PluginCapability_VolumeExpansion_ONLINE,
},
},
},
{
Type: &csi.PluginCapability_Service_{
Service: &csi.PluginCapability_Service{
Type: csi.PluginCapability_Service_VOLUME_ACCESSIBILITY_CONSTRAINTS,
},
},
},
},
}, nil
}

802
internal/rbd/nodeserver.go Normal file
View File

@ -0,0 +1,802 @@
/*
Copyright 2018 The Ceph-CSI Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package rbd
import (
"context"
"fmt"
"os"
"strconv"
"strings"
csicommon "github.com/ceph/ceph-csi/internal/csi-common"
"github.com/ceph/ceph-csi/internal/util"
"github.com/container-storage-interface/spec/lib/go/csi"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"k8s.io/klog"
"k8s.io/kubernetes/pkg/util/resizefs"
utilexec "k8s.io/utils/exec"
"k8s.io/utils/mount"
)
// NodeServer struct of ceph rbd driver with supported methods of CSI
// node server spec
type NodeServer struct {
*csicommon.DefaultNodeServer
mounter mount.Interface
// A map storing all volumes with ongoing operations so that additional operations
// for that same volume (as defined by VolumeID) return an Aborted error
VolumeLocks *util.VolumeLocks
}
// stageTransaction struct represents the state a transaction was when it either completed
// or failed
// this transaction state can be used to rollback the transaction
type stageTransaction struct {
// isStagePathCreated represents whether the mount path to stage the volume on was created or not
isStagePathCreated bool
// isMounted represents if the volume was mounted or not
isMounted bool
// isEncrypted represents if the volume was encrypted or not
isEncrypted bool
}
// NodeStageVolume mounts the volume to a staging path on the node.
// Implementation notes:
// - stagingTargetPath is the directory passed in the request where the volume needs to be staged
// - We stage the volume into a directory, named after the VolumeID inside stagingTargetPath if
// it is a file system
// - We stage the volume into a file, named after the VolumeID inside stagingTargetPath if it is
// a block volume
// - Order of operation execution: (useful for defer stacking and when Unstaging to ensure steps
// are done in reverse, this is done in undoStagingTransaction)
// - Stash image metadata under staging path
// - Map the image (creates a device)
// - Create the staging file/directory under staging path
// - Stage the device (mount the device mapped for image)
// nolint: gocyclo
func (ns *NodeServer) NodeStageVolume(ctx context.Context, req *csi.NodeStageVolumeRequest) (*csi.NodeStageVolumeResponse, error) {
if err := util.ValidateNodeStageVolumeRequest(req); err != nil {
return nil, err
}
isBlock := req.GetVolumeCapability().GetBlock() != nil
disableInUseChecks := false
// MULTI_NODE_MULTI_WRITER is supported by default for Block access type volumes
if req.VolumeCapability.AccessMode.Mode == csi.VolumeCapability_AccessMode_MULTI_NODE_MULTI_WRITER {
if !isBlock {
klog.Warningf(util.Log(ctx, "MULTI_NODE_MULTI_WRITER currently only supported with volumes of access type `block`, invalid AccessMode for volume: %v"), req.GetVolumeId())
return nil, status.Error(codes.InvalidArgument, "rbd: RWX access mode request is only valid for volumes with access type `block`")
}
disableInUseChecks = true
}
volID := req.GetVolumeId()
cr, err := util.NewUserCredentials(req.GetSecrets())
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
defer cr.DeleteCredentials()
if acquired := ns.VolumeLocks.TryAcquire(volID); !acquired {
klog.Errorf(util.Log(ctx, util.VolumeOperationAlreadyExistsFmt), volID)
return nil, status.Errorf(codes.Aborted, util.VolumeOperationAlreadyExistsFmt, volID)
}
defer ns.VolumeLocks.Release(volID)
stagingParentPath := req.GetStagingTargetPath()
stagingTargetPath := stagingParentPath + "/" + volID
// check is it a static volume
staticVol := false
val, ok := req.GetVolumeContext()["staticVolume"]
if ok {
if staticVol, err = strconv.ParseBool(val); err != nil {
return nil, status.Error(codes.InvalidArgument, err.Error())
}
}
var isNotMnt bool
// check if stagingPath is already mounted
isNotMnt, err = mount.IsNotMountPoint(ns.mounter, stagingTargetPath)
if err != nil && !os.IsNotExist(err) {
return nil, status.Error(codes.Internal, err.Error())
}
if !isNotMnt {
klog.V(4).Infof(util.Log(ctx, "rbd: volume %s is already mounted to %s, skipping"), volID, stagingTargetPath)
return &csi.NodeStageVolumeResponse{}, nil
}
isLegacyVolume := isLegacyVolumeID(volID)
volOptions, err := genVolFromVolumeOptions(ctx, req.GetVolumeContext(), req.GetSecrets(), disableInUseChecks, isLegacyVolume)
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
// get rbd image name from the volume journal
// for static volumes, the image name is actually the volume ID itself
// for legacy volumes (v1.0.0), the image name can be found in the staging path
switch {
case staticVol:
volOptions.RbdImageName = volID
case isLegacyVolume:
volOptions.RbdImageName, err = getLegacyVolumeName(stagingTargetPath)
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
default:
var vi util.CSIIdentifier
var imageAttributes *util.ImageAttributes
err = vi.DecomposeCSIID(volID)
if err != nil {
err = fmt.Errorf("error decoding volume ID (%s) (%s)", err, volID)
return nil, status.Error(codes.Internal, err.Error())
}
imageAttributes, err = volJournal.GetImageAttributes(ctx, volOptions.Monitors, cr,
volOptions.Pool, vi.ObjectUUID, false)
if err != nil {
err = fmt.Errorf("error fetching image attributes for volume ID (%s) (%s)", err, volID)
return nil, status.Error(codes.Internal, err.Error())
}
volOptions.RbdImageName = imageAttributes.ImageName
}
volOptions.VolID = volID
transaction := stageTransaction{}
devicePath := ""
// Stash image details prior to mapping the image (useful during Unstage as it has no
// voloptions passed to the RPC as per the CSI spec)
err = stashRBDImageMetadata(volOptions, stagingParentPath)
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
defer func() {
if err != nil {
ns.undoStagingTransaction(ctx, req, devicePath, transaction)
}
}()
// perform the actual staging and if this fails, have undoStagingTransaction
// cleans up for us
transaction, err = ns.stageTransaction(ctx, req, volOptions, staticVol)
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
klog.V(4).Infof(util.Log(ctx, "rbd: successfully mounted volume %s to stagingTargetPath %s"), req.GetVolumeId(), stagingTargetPath)
return &csi.NodeStageVolumeResponse{}, nil
}
func (ns *NodeServer) stageTransaction(ctx context.Context, req *csi.NodeStageVolumeRequest, volOptions *rbdVolume, staticVol bool) (stageTransaction, error) {
transaction := stageTransaction{}
var err error
var cr *util.Credentials
cr, err = util.NewUserCredentials(req.GetSecrets())
if err != nil {
return transaction, err
}
defer cr.DeleteCredentials()
// Mapping RBD image
var devicePath string
devicePath, err = attachRBDImage(ctx, volOptions, cr)
if err != nil {
return transaction, err
}
klog.V(4).Infof(util.Log(ctx, "rbd image: %s/%s was successfully mapped at %s\n"),
req.GetVolumeId(), volOptions.Pool, devicePath)
if volOptions.Encrypted {
devicePath, err = ns.processEncryptedDevice(ctx, volOptions, devicePath, cr)
if err != nil {
return transaction, err
}
transaction.isEncrypted = true
}
stagingTargetPath := getStagingTargetPath(req)
isBlock := req.GetVolumeCapability().GetBlock() != nil
err = ns.createStageMountPoint(ctx, stagingTargetPath, isBlock)
if err != nil {
return transaction, err
}
transaction.isStagePathCreated = true
// nodeStage Path
err = ns.mountVolumeToStagePath(ctx, req, staticVol, stagingTargetPath, devicePath)
if err != nil {
return transaction, err
}
transaction.isMounted = true
// #nosec - allow anyone to write inside the target path
err = os.Chmod(stagingTargetPath, 0777)
return transaction, err
}
func (ns *NodeServer) undoStagingTransaction(ctx context.Context, req *csi.NodeStageVolumeRequest, devicePath string, transaction stageTransaction) {
var err error
stagingTargetPath := getStagingTargetPath(req)
if transaction.isMounted {
err = ns.mounter.Unmount(stagingTargetPath)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to unmount stagingtargetPath: %s with error: %v"), stagingTargetPath, err)
return
}
}
// remove the file/directory created on staging path
if transaction.isStagePathCreated {
err = os.Remove(stagingTargetPath)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to remove stagingtargetPath: %s with error: %v"), stagingTargetPath, err)
// continue on failure to unmap the image, as leaving stale images causes more issues than a stale file/directory
}
}
volID := req.GetVolumeId()
// Unmapping rbd device
if devicePath != "" {
err = detachRBDDevice(ctx, devicePath, volID, transaction.isEncrypted)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to unmap rbd device: %s for volume %s with error: %v"), devicePath, volID, err)
// continue on failure to delete the stash file, as kubernetes will fail to delete the staging path otherwise
}
}
// Cleanup the stashed image metadata
if err = cleanupRBDImageMetadataStash(req.GetStagingTargetPath()); err != nil {
klog.Errorf(util.Log(ctx, "failed to cleanup image metadata stash (%v)"), err)
return
}
}
func (ns *NodeServer) createStageMountPoint(ctx context.Context, mountPath string, isBlock bool) error {
if isBlock {
pathFile, err := os.OpenFile(mountPath, os.O_CREATE|os.O_RDWR, 0600)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to create mountPath:%s with error: %v"), mountPath, err)
return status.Error(codes.Internal, err.Error())
}
if err = pathFile.Close(); err != nil {
klog.Errorf(util.Log(ctx, "failed to close mountPath:%s with error: %v"), mountPath, err)
return status.Error(codes.Internal, err.Error())
}
return nil
}
err := os.Mkdir(mountPath, 0750)
if err != nil {
if !os.IsExist(err) {
klog.Errorf(util.Log(ctx, "failed to create mountPath:%s with error: %v"), mountPath, err)
return status.Error(codes.Internal, err.Error())
}
}
return nil
}
// NodePublishVolume mounts the volume mounted to the device path to the target
// path
func (ns *NodeServer) NodePublishVolume(ctx context.Context, req *csi.NodePublishVolumeRequest) (*csi.NodePublishVolumeResponse, error) {
err := util.ValidateNodePublishVolumeRequest(req)
if err != nil {
return nil, err
}
targetPath := req.GetTargetPath()
isBlock := req.GetVolumeCapability().GetBlock() != nil
stagingPath := req.GetStagingTargetPath()
volID := req.GetVolumeId()
stagingPath += "/" + volID
if acquired := ns.VolumeLocks.TryAcquire(volID); !acquired {
klog.Errorf(util.Log(ctx, util.VolumeOperationAlreadyExistsFmt), volID)
return nil, status.Errorf(codes.Aborted, util.VolumeOperationAlreadyExistsFmt, volID)
}
defer ns.VolumeLocks.Release(volID)
// Check if that target path exists properly
notMnt, err := ns.createTargetMountPath(ctx, targetPath, isBlock)
if err != nil {
return nil, err
}
if !notMnt {
return &csi.NodePublishVolumeResponse{}, nil
}
// Publish Path
err = ns.mountVolume(ctx, stagingPath, req)
if err != nil {
return nil, err
}
klog.V(4).Infof(util.Log(ctx, "rbd: successfully mounted stagingPath %s to targetPath %s"), stagingPath, targetPath)
return &csi.NodePublishVolumeResponse{}, nil
}
func getLegacyVolumeName(mountPath string) (string, error) {
var volName string
if strings.HasSuffix(mountPath, "/globalmount") {
s := strings.Split(strings.TrimSuffix(mountPath, "/globalmount"), "/")
volName = s[len(s)-1]
return volName, nil
}
if strings.HasSuffix(mountPath, "/mount") {
s := strings.Split(strings.TrimSuffix(mountPath, "/mount"), "/")
volName = s[len(s)-1]
return volName, nil
}
// get volume name for block volume
s := strings.Split(mountPath, "/")
if len(s) == 0 {
return "", fmt.Errorf("rbd: malformed value of stage target path: %s", mountPath)
}
volName = s[len(s)-1]
return volName, nil
}
func (ns *NodeServer) mountVolumeToStagePath(ctx context.Context, req *csi.NodeStageVolumeRequest, staticVol bool, stagingPath, devicePath string) error {
fsType := req.GetVolumeCapability().GetMount().GetFsType()
diskMounter := &mount.SafeFormatAndMount{Interface: ns.mounter, Exec: utilexec.New()}
// rbd images are thin-provisioned and return zeros for unwritten areas. A freshly created
// image will not benefit from discard and we also want to avoid as much unnecessary zeroing
// as possible. Open-code mkfs here because FormatAndMount() doesn't accept custom mkfs
// options.
//
// Note that "freshly" is very important here. While discard is more of a nice to have,
// lazy_journal_init=1 is plain unsafe if the image has been written to before and hasn't
// been zeroed afterwards (unlike the name suggests, it leaves the journal completely
// uninitialized and carries a risk until the journal is overwritten and wraps around for
// the first time).
existingFormat, err := diskMounter.GetDiskFormat(devicePath)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to get disk format for path %s, error: %v"), devicePath, err)
return err
}
if existingFormat == "" && !staticVol {
args := []string{}
if fsType == "ext4" {
args = []string{"-m0", "-Enodiscard,lazy_itable_init=1,lazy_journal_init=1", devicePath}
} else if fsType == "xfs" {
args = []string{"-K", devicePath}
}
if len(args) > 0 {
cmdOut, cmdErr := diskMounter.Exec.Command("mkfs."+fsType, args...).CombinedOutput()
if cmdErr != nil {
klog.Errorf(util.Log(ctx, "failed to run mkfs error: %v, output: %v"), cmdErr, cmdOut)
return cmdErr
}
}
}
opt := []string{"_netdev"}
opt = csicommon.ConstructMountOptions(opt, req.GetVolumeCapability())
isBlock := req.GetVolumeCapability().GetBlock() != nil
if isBlock {
opt = append(opt, "bind")
err = diskMounter.Mount(devicePath, stagingPath, fsType, opt)
} else {
err = diskMounter.FormatAndMount(devicePath, stagingPath, fsType, opt)
}
if err != nil {
klog.Errorf(util.Log(ctx, "failed to mount device path (%s) to staging path (%s) for volume (%s) error %s"), devicePath, stagingPath, req.GetVolumeId(), err)
}
return err
}
func (ns *NodeServer) mountVolume(ctx context.Context, stagingPath string, req *csi.NodePublishVolumeRequest) error {
// Publish Path
fsType := req.GetVolumeCapability().GetMount().GetFsType()
readOnly := req.GetReadonly()
mountOptions := []string{"bind", "_netdev"}
isBlock := req.GetVolumeCapability().GetBlock() != nil
targetPath := req.GetTargetPath()
mountOptions = csicommon.ConstructMountOptions(mountOptions, req.GetVolumeCapability())
klog.V(4).Infof(util.Log(ctx, "target %v\nisBlock %v\nfstype %v\nstagingPath %v\nreadonly %v\nmountflags %v\n"),
targetPath, isBlock, fsType, stagingPath, readOnly, mountOptions)
if readOnly {
mountOptions = append(mountOptions, "ro")
}
if err := util.Mount(stagingPath, targetPath, fsType, mountOptions); err != nil {
return status.Error(codes.Internal, err.Error())
}
return nil
}
func (ns *NodeServer) createTargetMountPath(ctx context.Context, mountPath string, isBlock bool) (bool, error) {
// Check if that mount path exists properly
notMnt, err := mount.IsNotMountPoint(ns.mounter, mountPath)
if err != nil {
if os.IsNotExist(err) {
if isBlock {
// #nosec
pathFile, e := os.OpenFile(mountPath, os.O_CREATE|os.O_RDWR, 0750)
if e != nil {
klog.V(4).Infof(util.Log(ctx, "Failed to create mountPath:%s with error: %v"), mountPath, err)
return notMnt, status.Error(codes.Internal, e.Error())
}
if err = pathFile.Close(); err != nil {
klog.V(4).Infof(util.Log(ctx, "Failed to close mountPath:%s with error: %v"), mountPath, err)
return notMnt, status.Error(codes.Internal, err.Error())
}
} else {
// Create a directory
if err = util.CreateMountPoint(mountPath); err != nil {
return notMnt, status.Error(codes.Internal, err.Error())
}
}
notMnt = true
} else {
return false, status.Error(codes.Internal, err.Error())
}
}
return notMnt, err
}
// NodeUnpublishVolume unmounts the volume from the target path
func (ns *NodeServer) NodeUnpublishVolume(ctx context.Context, req *csi.NodeUnpublishVolumeRequest) (*csi.NodeUnpublishVolumeResponse, error) {
err := util.ValidateNodeUnpublishVolumeRequest(req)
if err != nil {
return nil, err
}
targetPath := req.GetTargetPath()
volID := req.GetVolumeId()
if acquired := ns.VolumeLocks.TryAcquire(volID); !acquired {
klog.Errorf(util.Log(ctx, util.VolumeOperationAlreadyExistsFmt), volID)
return nil, status.Errorf(codes.Aborted, util.VolumeOperationAlreadyExistsFmt, volID)
}
defer ns.VolumeLocks.Release(volID)
notMnt, err := mount.IsNotMountPoint(ns.mounter, targetPath)
if err != nil {
if os.IsNotExist(err) {
// targetPath has already been deleted
klog.V(4).Infof(util.Log(ctx, "targetPath: %s has already been deleted"), targetPath)
return &csi.NodeUnpublishVolumeResponse{}, nil
}
return nil, status.Error(codes.NotFound, err.Error())
}
if notMnt {
if err = os.RemoveAll(targetPath); err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
return &csi.NodeUnpublishVolumeResponse{}, nil
}
if err = ns.mounter.Unmount(targetPath); err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
if err = os.RemoveAll(targetPath); err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
klog.V(4).Infof(util.Log(ctx, "rbd: successfully unbound volume %s from %s"), req.GetVolumeId(), targetPath)
return &csi.NodeUnpublishVolumeResponse{}, nil
}
// getStagingTargetPath concats either NodeStageVolumeRequest's or
// NodeUnstageVolumeRequest's target path with the volumeID
func getStagingTargetPath(req interface{}) string {
switch vr := req.(type) {
case *csi.NodeStageVolumeRequest:
return vr.GetStagingTargetPath() + "/" + vr.GetVolumeId()
case *csi.NodeUnstageVolumeRequest:
return vr.GetStagingTargetPath() + "/" + vr.GetVolumeId()
}
return ""
}
// NodeUnstageVolume unstages the volume from the staging path
func (ns *NodeServer) NodeUnstageVolume(ctx context.Context, req *csi.NodeUnstageVolumeRequest) (*csi.NodeUnstageVolumeResponse, error) {
var err error
if err = util.ValidateNodeUnstageVolumeRequest(req); err != nil {
return nil, err
}
volID := req.GetVolumeId()
if acquired := ns.VolumeLocks.TryAcquire(volID); !acquired {
klog.Errorf(util.Log(ctx, util.VolumeOperationAlreadyExistsFmt), volID)
return nil, status.Errorf(codes.Aborted, util.VolumeOperationAlreadyExistsFmt, volID)
}
defer ns.VolumeLocks.Release(volID)
stagingParentPath := req.GetStagingTargetPath()
stagingTargetPath := getStagingTargetPath(req)
notMnt, err := mount.IsNotMountPoint(ns.mounter, stagingTargetPath)
if err != nil {
if !os.IsNotExist(err) {
return nil, status.Error(codes.NotFound, err.Error())
}
// Continue on ENOENT errors as we may still have the image mapped
notMnt = true
}
if !notMnt {
// Unmounting the image
err = ns.mounter.Unmount(stagingTargetPath)
if err != nil {
klog.V(3).Infof(util.Log(ctx, "failed to unmount targetPath: %s with error: %v"), stagingTargetPath, err)
return nil, status.Error(codes.Internal, err.Error())
}
}
if err = os.Remove(stagingTargetPath); err != nil {
// Any error is critical as Staging path is expected to be empty by Kubernetes, it otherwise
// keeps invoking Unstage. Hence any errors removing files within this path is a critical
// error
if !os.IsNotExist(err) {
klog.Errorf(util.Log(ctx, "failed to remove staging target path (%s): (%v)"), stagingTargetPath, err)
return nil, status.Error(codes.Internal, err.Error())
}
}
imgInfo, err := lookupRBDImageMetadataStash(stagingParentPath)
if err != nil {
klog.V(2).Infof(util.Log(ctx, "failed to find image metadata: %v"), err)
// It is an error if it was mounted, as we should have found the image metadata file with
// no errors
if !notMnt {
return nil, status.Error(codes.Internal, err.Error())
}
// If not mounted, and error is anything other than metadata file missing, it is an error
if _, ok := err.(ErrMissingStash); !ok {
return nil, status.Error(codes.Internal, err.Error())
}
// It was not mounted and image metadata is also missing, we are done as the last step in
// in the staging transaction is complete
return &csi.NodeUnstageVolumeResponse{}, nil
}
// Unmapping rbd device
imageSpec := imgInfo.Pool + "/" + imgInfo.ImageName
if err = detachRBDImageOrDeviceSpec(ctx, imageSpec, true, imgInfo.NbdAccess, imgInfo.Encrypted, req.GetVolumeId()); err != nil {
klog.Errorf(util.Log(ctx, "error unmapping volume (%s) from staging path (%s): (%v)"), req.GetVolumeId(), stagingTargetPath, err)
return nil, status.Error(codes.Internal, err.Error())
}
klog.V(4).Infof(util.Log(ctx, "successfully unmounted volume (%s) from staging path (%s)"),
req.GetVolumeId(), stagingTargetPath)
if err = cleanupRBDImageMetadataStash(stagingParentPath); err != nil {
klog.Errorf(util.Log(ctx, "failed to cleanup image metadata stash (%v)"), err)
return nil, status.Error(codes.Internal, err.Error())
}
return &csi.NodeUnstageVolumeResponse{}, nil
}
// NodeExpandVolume resizes rbd volumes
func (ns *NodeServer) NodeExpandVolume(ctx context.Context, req *csi.NodeExpandVolumeRequest) (*csi.NodeExpandVolumeResponse, error) {
volumeID := req.GetVolumeId()
if volumeID == "" {
return nil, status.Error(codes.InvalidArgument, "volume ID must be provided")
}
volumePath := req.GetVolumePath()
if volumePath == "" {
return nil, status.Error(codes.InvalidArgument, "volume path must be provided")
}
if acquired := ns.VolumeLocks.TryAcquire(volumeID); !acquired {
klog.Errorf(util.Log(ctx, util.VolumeOperationAlreadyExistsFmt), volumeID)
return nil, status.Errorf(codes.Aborted, util.VolumeOperationAlreadyExistsFmt, volumeID)
}
defer ns.VolumeLocks.Release(volumeID)
// volumePath is targetPath for block PVC and stagingPath for filesystem.
// check the path is mountpoint or not, if it is
// mountpoint treat this as block PVC or else it is filesystem PVC
// TODO remove this once ceph-csi supports CSI v1.2.0 spec
notMnt, err := mount.IsNotMountPoint(ns.mounter, volumePath)
if err != nil {
if os.IsNotExist(err) {
return nil, status.Error(codes.NotFound, err.Error())
}
return nil, status.Error(codes.Internal, err.Error())
}
if !notMnt {
return &csi.NodeExpandVolumeResponse{}, nil
}
devicePath, err := getDevicePath(ctx, volumePath)
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
diskMounter := &mount.SafeFormatAndMount{Interface: ns.mounter, Exec: utilexec.New()}
// TODO check size and return success or error
volumePath += "/" + volumeID
resizer := resizefs.NewResizeFs(diskMounter)
ok, err := resizer.Resize(devicePath, volumePath)
if !ok {
return nil, fmt.Errorf("rbd: resize failed on path %s, error: %v", req.GetVolumePath(), err)
}
return &csi.NodeExpandVolumeResponse{}, nil
}
func getDevicePath(ctx context.Context, volumePath string) (string, error) {
imgInfo, err := lookupRBDImageMetadataStash(volumePath)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to find image metadata: %v"), err)
}
device, found := findDeviceMappingImage(ctx, imgInfo.Pool, imgInfo.ImageName, imgInfo.NbdAccess)
if found {
return device, nil
}
return "", fmt.Errorf("failed to get device for stagingtarget path %v", volumePath)
}
// NodeGetCapabilities returns the supported capabilities of the node server
func (ns *NodeServer) NodeGetCapabilities(ctx context.Context, req *csi.NodeGetCapabilitiesRequest) (*csi.NodeGetCapabilitiesResponse, error) {
return &csi.NodeGetCapabilitiesResponse{
Capabilities: []*csi.NodeServiceCapability{
{
Type: &csi.NodeServiceCapability_Rpc{
Rpc: &csi.NodeServiceCapability_RPC{
Type: csi.NodeServiceCapability_RPC_STAGE_UNSTAGE_VOLUME,
},
},
},
{
Type: &csi.NodeServiceCapability_Rpc{
Rpc: &csi.NodeServiceCapability_RPC{
Type: csi.NodeServiceCapability_RPC_GET_VOLUME_STATS,
},
},
},
{
Type: &csi.NodeServiceCapability_Rpc{
Rpc: &csi.NodeServiceCapability_RPC{
Type: csi.NodeServiceCapability_RPC_EXPAND_VOLUME,
},
},
},
},
}, nil
}
func (ns *NodeServer) processEncryptedDevice(ctx context.Context, volOptions *rbdVolume, devicePath string, cr *util.Credentials) (string, error) {
imageSpec := volOptions.Pool + "/" + volOptions.RbdImageName
encrypted, err := util.CheckRbdImageEncrypted(ctx, cr, volOptions.Monitors, imageSpec)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to get encryption status for rbd image %s: %v"),
imageSpec, err)
return "", err
}
if encrypted == rbdImageRequiresEncryption {
diskMounter := &mount.SafeFormatAndMount{Interface: ns.mounter, Exec: utilexec.New()}
// TODO: update this when adding support for static (pre-provisioned) PVs
var existingFormat string
existingFormat, err = diskMounter.GetDiskFormat(devicePath)
if err != nil {
return "", fmt.Errorf("failed to get disk format for path %s, error: %v", devicePath, err)
}
switch existingFormat {
case "":
err = encryptDevice(ctx, volOptions, cr, devicePath)
if err != nil {
return "", fmt.Errorf("failed to encrypt rbd image %s: %v", imageSpec, err)
}
case "crypt":
klog.Warningf(util.Log(ctx, "rbd image %s is encrypted, but encryption state was not updated"),
imageSpec)
err = util.SaveRbdImageEncryptionStatus(
ctx, cr, volOptions.Monitors, imageSpec, rbdImageEncrypted)
if err != nil {
return "", fmt.Errorf("failed to update encryption state for rbd image %s", imageSpec)
}
default:
return "", fmt.Errorf("can not encrypt rbdImage %s that already has file system: %s",
imageSpec, existingFormat)
}
} else if encrypted != rbdImageEncrypted {
return "", fmt.Errorf("rbd image %s found mounted with unexpected encryption status %s",
imageSpec, encrypted)
}
devicePath, err = openEncryptedDevice(ctx, volOptions, devicePath)
if err != nil {
return "", err
}
return devicePath, nil
}
func encryptDevice(ctx context.Context, rbdVol *rbdVolume, cr *util.Credentials, devicePath string) error {
passphrase, err := util.GetCryptoPassphrase(ctx, rbdVol.VolID, rbdVol.KMS)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to get crypto passphrase for %s/%s: %v"),
rbdVol.Pool, rbdVol.RbdImageName, err)
return err
}
if err = util.EncryptVolume(ctx, devicePath, passphrase); err != nil {
err = fmt.Errorf("failed to encrypt volume %s/%s: %v", rbdVol.Pool, rbdVol.RbdImageName, err)
klog.Errorf(util.Log(ctx, err.Error()))
return err
}
imageSpec := rbdVol.Pool + "/" + rbdVol.RbdImageName
err = util.SaveRbdImageEncryptionStatus(ctx, cr, rbdVol.Monitors, imageSpec, rbdImageEncrypted)
return err
}
func openEncryptedDevice(ctx context.Context, volOptions *rbdVolume, devicePath string) (string, error) {
passphrase, err := util.GetCryptoPassphrase(ctx, volOptions.VolID, volOptions.KMS)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to get passphrase for encrypted device %s/%s: %v"),
volOptions.Pool, volOptions.RbdImageName, err)
return "", status.Error(codes.Internal, err.Error())
}
mapperFile, mapperFilePath := util.VolumeMapper(volOptions.VolID)
isOpen, err := util.IsDeviceOpen(ctx, mapperFilePath)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to check device %s encryption status: %s"), devicePath, err)
return devicePath, err
}
if isOpen {
klog.V(4).Infof(util.Log(ctx, "encrypted device is already open at %s"), mapperFilePath)
} else {
err = util.OpenEncryptedVolume(ctx, devicePath, mapperFile, passphrase)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to open device %s/%s: %v"),
volOptions.Pool, volOptions.RbdImageName, err)
return devicePath, err
}
}
return mapperFilePath, nil
}

View File

@ -0,0 +1,76 @@
/*
Copyright 2020 The Ceph-CSI Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package rbd
import (
"testing"
"github.com/container-storage-interface/spec/lib/go/csi"
)
func TestGetLegacyVolumeName(t *testing.T) {
tests := []struct {
mountPath string
volName string
}{
{"csi/vol/56a0cc34-a5c9-44ab-ad33-ed53dd2bd5ea/globalmount", "56a0cc34-a5c9-44ab-ad33-ed53dd2bd5ea"},
{"csi/vol/9fdb7491-3469-4414-8fe2-ea96be6f7f72/mount", "9fdb7491-3469-4414-8fe2-ea96be6f7f72"},
{"csi/vol/82cd91c4-4582-47b3-bb08-a84f8c5716d6", "82cd91c4-4582-47b3-bb08-a84f8c5716d6"},
}
for _, test := range tests {
if got, err := getLegacyVolumeName(test.mountPath); err != nil {
t.Errorf("getLegacyVolumeName(%s) returned error when it shouldn't: %s", test.mountPath, err.Error())
} else if got != test.volName {
t.Errorf("getLegacyVolumeName(%s) = %s, want %s", test.mountPath, got, test.volName)
}
}
}
func TestGetStagingPath(t *testing.T) {
var stagingPath string
// test with nodestagevolumerequest
nsvr := &csi.NodeStageVolumeRequest{
VolumeId: "758978be-6331-4925-b25e-e490fe99c9eb",
StagingTargetPath: "/path/to/stage",
}
expect := "/path/to/stage/758978be-6331-4925-b25e-e490fe99c9eb"
stagingPath = getStagingTargetPath(nsvr)
if stagingPath != expect {
t.Errorf("getStagingTargetPath() = %s, got %s", stagingPath, expect)
}
// test with nodestagevolumerequest
nuvr := &csi.NodeUnstageVolumeRequest{
VolumeId: "622cfdeb-69bf-4de6-9bd7-5fa0b71a603e",
StagingTargetPath: "/path/to/unstage",
}
expect = "/path/to/unstage/622cfdeb-69bf-4de6-9bd7-5fa0b71a603e"
stagingPath = getStagingTargetPath(nuvr)
if stagingPath != expect {
t.Errorf("getStagingTargetPath() = %s, got %s", stagingPath, expect)
}
// test with non-handled interface
expect = ""
stagingPath = getStagingTargetPath("")
if stagingPath != expect {
t.Errorf("getStagingTargetPath() = %s, got %s", stagingPath, expect)
}
}

324
internal/rbd/rbd_attach.go Normal file
View File

@ -0,0 +1,324 @@
/*
Copyright 2018 The Ceph-CSI Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package rbd
import (
"context"
"encoding/json"
"fmt"
"os"
"strconv"
"strings"
"time"
"github.com/ceph/ceph-csi/internal/util"
"k8s.io/apimachinery/pkg/util/wait"
"k8s.io/klog"
)
const (
rbdTonbd = "rbd-nbd"
moduleNbd = "nbd"
accessTypeKRbd = "krbd"
accessTypeNbd = "nbd"
rbd = "rbd"
// Output strings returned during invocation of "rbd unmap --device-type... <imageSpec>" when
// image is not found to be mapped. Used to ignore errors when attempting to unmap such images.
// The %s format specifier should contain the <imageSpec> string
// NOTE: When using devicePath instead of imageSpec, the error strings are different
rbdUnmapCmdkRbdMissingMap = "rbd: %s: not a mapped image or snapshot"
rbdUnmapCmdNbdMissingMap = "rbd-nbd: %s is not mapped"
rbdMapConnectionTimeout = "Connection timed out"
)
var hasNBD = false
func init() {
hasNBD = checkRbdNbdTools()
}
// rbdDeviceInfo strongly typed JSON spec for rbd device list output (of type krbd)
type rbdDeviceInfo struct {
ID string `json:"id"`
Pool string `json:"pool"`
Name string `json:"name"`
Device string `json:"device"`
}
// nbdDeviceInfo strongly typed JSON spec for rbd-nbd device list output (of type nbd)
// NOTE: There is a bug in rbd output that returns id as number for nbd, and string for krbd, thus
// requiring 2 different JSON structures to unmarshal the output.
// NOTE: image key is "name" in krbd output and "image" in nbd output, which is another difference
type nbdDeviceInfo struct {
ID int64 `json:"id"`
Pool string `json:"pool"`
Name string `json:"image"`
Device string `json:"device"`
}
// rbdGetDeviceList queries rbd about mapped devices and returns a list of rbdDeviceInfo
// It will selectively list devices mapped using krbd or nbd as specified by accessType
func rbdGetDeviceList(accessType string) ([]rbdDeviceInfo, error) {
// rbd device list --format json --device-type [krbd|nbd]
var (
rbdDeviceList []rbdDeviceInfo
nbdDeviceList []nbdDeviceInfo
)
stdout, _, err := util.ExecCommand(rbd, "device", "list", "--format="+"json", "--device-type", accessType)
if err != nil {
return nil, fmt.Errorf("error getting device list from rbd for devices of type (%s): (%v)", accessType, err)
}
if accessType == accessTypeKRbd {
err = json.Unmarshal(stdout, &rbdDeviceList)
} else {
err = json.Unmarshal(stdout, &nbdDeviceList)
}
if err != nil {
return nil, fmt.Errorf("error to parse JSON output of device list for devices of type (%s): (%v)", accessType, err)
}
// convert output to a rbdDeviceInfo list for consumers
if accessType == accessTypeNbd {
for _, device := range nbdDeviceList {
rbdDeviceList = append(
rbdDeviceList,
rbdDeviceInfo{
ID: strconv.FormatInt(device.ID, 10),
Pool: device.Pool,
Name: device.Name,
Device: device.Device,
})
}
}
return rbdDeviceList, nil
}
// findDeviceMappingImage finds a devicePath, if available, based on image spec (pool/image) on the node.
func findDeviceMappingImage(ctx context.Context, pool, image string, useNbdDriver bool) (string, bool) {
accessType := accessTypeKRbd
if useNbdDriver {
accessType = accessTypeNbd
}
rbdDeviceList, err := rbdGetDeviceList(accessType)
if err != nil {
klog.Warningf(util.Log(ctx, "failed to determine if image (%s/%s) is mapped to a device (%v)"), pool, image, err)
return "", false
}
for _, device := range rbdDeviceList {
if device.Name == image && device.Pool == pool {
return device.Device, true
}
}
return "", false
}
// Stat a path, if it doesn't exist, retry maxRetries times.
func waitForPath(ctx context.Context, pool, image string, maxRetries int, useNbdDriver bool) (string, bool) {
for i := 0; i < maxRetries; i++ {
if i != 0 {
time.Sleep(time.Second)
}
device, found := findDeviceMappingImage(ctx, pool, image, useNbdDriver)
if found {
return device, found
}
}
return "", false
}
// Check if rbd-nbd tools are installed.
func checkRbdNbdTools() bool {
// check if the module is loaded or compiled in
_, err := os.Stat(fmt.Sprintf("/sys/module/%s", moduleNbd))
if os.IsNotExist(err) {
// try to load the module
_, err = execCommand("modprobe", []string{moduleNbd})
if err != nil {
klog.V(3).Infof("rbd-nbd: nbd modprobe failed with error %v", err)
return false
}
}
if _, err := execCommand(rbdTonbd, []string{"--version"}); err != nil {
klog.V(3).Infof("rbd-nbd: running rbd-nbd --version failed with error %v", err)
return false
}
klog.V(3).Infof("rbd-nbd tools were found.")
return true
}
func attachRBDImage(ctx context.Context, volOptions *rbdVolume, cr *util.Credentials) (string, error) {
var err error
image := volOptions.RbdImageName
useNBD := false
if volOptions.Mounter == rbdTonbd && hasNBD {
useNBD = true
}
devicePath, found := waitForPath(ctx, volOptions.Pool, image, 1, useNBD)
if !found {
backoff := wait.Backoff{
Duration: rbdImageWatcherInitDelay,
Factor: rbdImageWatcherFactor,
Steps: rbdImageWatcherSteps,
}
err = waitForrbdImage(ctx, backoff, volOptions, cr)
if err != nil {
return "", err
}
devicePath, err = createPath(ctx, volOptions, cr)
}
return devicePath, err
}
func createPath(ctx context.Context, volOpt *rbdVolume, cr *util.Credentials) (string, error) {
isNbd := false
image := volOpt.RbdImageName
imagePath := fmt.Sprintf("%s/%s", volOpt.Pool, image)
klog.V(5).Infof(util.Log(ctx, "rbd: map mon %s"), volOpt.Monitors)
// Map options
mapOptions := []string{
"--id", cr.ID,
"-m", volOpt.Monitors,
"--keyfile=" + cr.KeyFile,
"map", imagePath,
}
// Choose access protocol
accessType := accessTypeKRbd
if volOpt.Mounter == rbdTonbd && hasNBD {
isNbd = true
accessType = accessTypeNbd
}
// Update options with device type selection
mapOptions = append(mapOptions, "--device-type", accessType)
// Execute map
output, err := execCommand(rbd, mapOptions)
if err != nil {
klog.Warningf(util.Log(ctx, "rbd: map error %v, rbd output: %s"), err, string(output))
// unmap rbd image if connection timeout
if strings.Contains(err.Error(), rbdMapConnectionTimeout) {
detErr := detachRBDImageOrDeviceSpec(ctx, imagePath, true, isNbd, volOpt.Encrypted, volOpt.VolID)
if detErr != nil {
klog.Warningf(util.Log(ctx, "rbd: %s unmap error %v"), imagePath, detErr)
}
}
return "", fmt.Errorf("rbd: map failed %v, rbd output: %s", err, string(output))
}
devicePath := strings.TrimSuffix(string(output), "\n")
return devicePath, nil
}
func waitForrbdImage(ctx context.Context, backoff wait.Backoff, volOptions *rbdVolume, cr *util.Credentials) error {
image := volOptions.RbdImageName
imagePath := fmt.Sprintf("%s/%s", volOptions.Pool, image)
err := wait.ExponentialBackoff(backoff, func() (bool, error) {
used, rbdOutput, err := rbdStatus(ctx, volOptions, cr)
if err != nil {
return false, fmt.Errorf("fail to check rbd image status with: (%v), rbd output: (%s)", err, rbdOutput)
}
if (volOptions.DisableInUseChecks) && (used) {
klog.V(2).Info(util.Log(ctx, "valid multi-node attach requested, ignoring watcher in-use result"))
return used, nil
}
return !used, nil
})
// return error if rbd image has not become available for the specified timeout
if err == wait.ErrWaitTimeout {
return fmt.Errorf("rbd image %s is still being used", imagePath)
}
// return error if any other errors were encountered during waiting for the image to become available
return err
}
func detachRBDDevice(ctx context.Context, devicePath, volumeID string, encrypted bool) error {
nbdType := false
if strings.HasPrefix(devicePath, "/dev/nbd") {
nbdType = true
}
return detachRBDImageOrDeviceSpec(ctx, devicePath, false, nbdType, encrypted, volumeID)
}
// detachRBDImageOrDeviceSpec detaches an rbd imageSpec or devicePath, with additional checking
// when imageSpec is used to decide if image is already unmapped
func detachRBDImageOrDeviceSpec(ctx context.Context, imageOrDeviceSpec string, isImageSpec, ndbType, encrypted bool, volumeID string) error {
var output []byte
if encrypted {
mapperFile, mapperPath := util.VolumeMapper(volumeID)
mappedDevice, mapper, err := util.DeviceEncryptionStatus(ctx, mapperPath)
if err != nil {
klog.Errorf(util.Log(ctx, "error determining LUKS device on %s, %s: %s"),
mapperPath, imageOrDeviceSpec, err)
return err
}
if len(mapper) > 0 {
// mapper found, so it is open Luks device
err = util.CloseEncryptedVolume(ctx, mapperFile)
if err != nil {
klog.Errorf(util.Log(ctx, "error closing LUKS device on %s, %s: %s"),
mapperPath, imageOrDeviceSpec, err)
return err
}
imageOrDeviceSpec = mappedDevice
}
}
accessType := accessTypeKRbd
if ndbType {
accessType = accessTypeNbd
}
options := []string{"unmap", "--device-type", accessType, imageOrDeviceSpec}
output, err := execCommand(rbd, options)
if err != nil {
// Messages for krbd and nbd differ, hence checking either of them for missing mapping
// This is not applicable when a device path is passed in
if isImageSpec &&
(strings.Contains(string(output), fmt.Sprintf(rbdUnmapCmdkRbdMissingMap, imageOrDeviceSpec)) ||
strings.Contains(string(output), fmt.Sprintf(rbdUnmapCmdNbdMissingMap, imageOrDeviceSpec))) {
// Devices found not to be mapped are treated as a successful detach
klog.V(5).Infof(util.Log(ctx, "image or device spec (%s) not mapped"), imageOrDeviceSpec)
return nil
}
return fmt.Errorf("rbd: unmap for spec (%s) failed (%v): (%s)", imageOrDeviceSpec, err, string(output))
}
return nil
}

353
internal/rbd/rbd_journal.go Normal file
View File

@ -0,0 +1,353 @@
/*
Copyright 2019 The Ceph-CSI Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package rbd
import (
"context"
"fmt"
"github.com/ceph/ceph-csi/internal/util"
"github.com/pkg/errors"
"k8s.io/klog"
)
func validateNonEmptyField(field, fieldName, structName string) error {
if field == "" {
return fmt.Errorf("value '%s' in '%s' structure cannot be empty", fieldName, structName)
}
return nil
}
func validateRbdSnap(rbdSnap *rbdSnapshot) error {
var err error
if err = validateNonEmptyField(rbdSnap.RequestName, "RequestName", "rbdSnapshot"); err != nil {
return err
}
if err = validateNonEmptyField(rbdSnap.Monitors, "Monitors", "rbdSnapshot"); err != nil {
return err
}
if err = validateNonEmptyField(rbdSnap.Pool, "Pool", "rbdSnapshot"); err != nil {
return err
}
if err = validateNonEmptyField(rbdSnap.RbdImageName, "RbdImageName", "rbdSnapshot"); err != nil {
return err
}
if err = validateNonEmptyField(rbdSnap.ClusterID, "ClusterID", "rbdSnapshot"); err != nil {
return err
}
return err
}
func validateRbdVol(rbdVol *rbdVolume) error {
var err error
if err = validateNonEmptyField(rbdVol.RequestName, "RequestName", "rbdVolume"); err != nil {
return err
}
if err = validateNonEmptyField(rbdVol.Monitors, "Monitors", "rbdVolume"); err != nil {
return err
}
if err = validateNonEmptyField(rbdVol.Pool, "Pool", "rbdVolume"); err != nil {
return err
}
if err = validateNonEmptyField(rbdVol.ClusterID, "ClusterID", "rbdVolume"); err != nil {
return err
}
if rbdVol.VolSize == 0 {
return errors.New("value 'VolSize' in 'rbdVolume' structure cannot be 0")
}
return err
}
/*
checkSnapExists, and its counterpart checkVolExists, function checks if the passed in rbdSnapshot
or rbdVolume exists on the backend.
**NOTE:** These functions manipulate the rados omaps that hold information regarding
volume names as requested by the CSI drivers. Hence, these need to be invoked only when the
respective CSI driver generated snapshot or volume name based locks are held, as otherwise racy
access to these omaps may end up leaving them in an inconsistent state.
These functions need enough information about cluster and pool (ie, Monitors, Pool, IDs filled in)
to operate. They further require that the RequestName element of the structure have a valid value
to operate on and determine if the said RequestName already exists on the backend.
These functions populate the snapshot or the image name, its attributes and the CSI snapshot/volume
ID for the same when successful.
These functions also cleanup omap reservations that are stale. I.e when omap entries exist and
backing images or snapshots are missing, or one of the omaps exist and the next is missing. This is
because, the order of omap creation and deletion are inverse of each other, and protected by the
request name lock, and hence any stale omaps are leftovers from incomplete transactions and are
hence safe to garbage collect.
*/
func checkSnapExists(ctx context.Context, rbdSnap *rbdSnapshot, cr *util.Credentials) (bool, error) {
err := validateRbdSnap(rbdSnap)
if err != nil {
return false, err
}
snapData, err := snapJournal.CheckReservation(ctx, rbdSnap.Monitors, cr, rbdSnap.JournalPool,
rbdSnap.RequestName, rbdSnap.NamePrefix, rbdSnap.RbdImageName, "")
if err != nil {
return false, err
}
if snapData == nil {
return false, nil
}
snapUUID := snapData.ImageUUID
rbdSnap.RbdSnapName = snapData.ImageAttributes.ImageName
// it should never happen that this disagrees, but check
if rbdSnap.Pool != snapData.ImagePool {
return false, fmt.Errorf("stored snapshot pool (%s) and expected snapshot pool (%s) mismatch",
snapData.ImagePool, rbdSnap.Pool)
}
// Fetch on-disk image attributes
err = updateSnapWithImageInfo(ctx, rbdSnap, cr)
if err != nil {
if _, ok := err.(ErrSnapNotFound); ok {
err = snapJournal.UndoReservation(ctx, rbdSnap.Monitors, cr, rbdSnap.JournalPool,
rbdSnap.Pool, rbdSnap.RbdSnapName, rbdSnap.RequestName)
return false, err
}
return false, err
}
// found a snapshot already available, process and return its information
rbdSnap.SnapID, err = util.GenerateVolID(ctx, rbdSnap.Monitors, cr, snapData.ImagePoolID, rbdSnap.Pool,
rbdSnap.ClusterID, snapUUID, volIDVersion)
if err != nil {
return false, err
}
klog.V(4).Infof(util.Log(ctx, "found existing snap (%s) with snap name (%s) for request (%s)"),
rbdSnap.SnapID, rbdSnap.RbdSnapName, rbdSnap.RequestName)
return true, nil
}
/*
Check comment on checkSnapExists, to understand how this function behaves
**NOTE:** These functions manipulate the rados omaps that hold information regarding
volume names as requested by the CSI drivers. Hence, these need to be invoked only when the
respective CSI snapshot or volume name based locks are held, as otherwise racy access to these
omaps may end up leaving the omaps in an inconsistent state.
*/
func checkVolExists(ctx context.Context, rbdVol *rbdVolume, cr *util.Credentials) (bool, error) {
err := validateRbdVol(rbdVol)
if err != nil {
return false, err
}
kmsID := ""
if rbdVol.Encrypted {
kmsID = rbdVol.KMS.GetID()
}
imageData, err := volJournal.CheckReservation(ctx, rbdVol.Monitors, cr, rbdVol.JournalPool,
rbdVol.RequestName, rbdVol.NamePrefix, "", kmsID)
if err != nil {
return false, err
}
if imageData == nil {
return false, nil
}
imageUUID := imageData.ImageUUID
rbdVol.RbdImageName = imageData.ImageAttributes.ImageName
// check if topology constraints match what is found
rbdVol.Topology, err = util.MatchTopologyForPool(rbdVol.TopologyPools,
rbdVol.TopologyRequirement, imageData.ImagePool)
if err != nil {
// TODO check if need any undo operation here, or ErrVolNameConflict
return false, err
}
// update Pool, if it was topology constrained
if rbdVol.Topology != nil {
rbdVol.Pool = imageData.ImagePool
}
// NOTE: Return volsize should be on-disk volsize, not request vol size, so
// save it for size checks before fetching image data
requestSize := rbdVol.VolSize
// Fetch on-disk image attributes and compare against request
err = updateVolWithImageInfo(ctx, rbdVol, cr)
if err != nil {
if _, ok := err.(ErrImageNotFound); ok {
err = volJournal.UndoReservation(ctx, rbdVol.Monitors, cr, rbdVol.JournalPool, rbdVol.Pool,
rbdVol.RbdImageName, rbdVol.RequestName)
return false, err
}
return false, err
}
// size checks
if rbdVol.VolSize < requestSize {
err = fmt.Errorf("image with the same name (%s) but with different size already exists",
rbdVol.RbdImageName)
return false, ErrVolNameConflict{rbdVol.RbdImageName, err}
}
// TODO: We should also ensure image features and format is the same
// found a volume already available, process and return it!
rbdVol.VolID, err = util.GenerateVolID(ctx, rbdVol.Monitors, cr, imageData.ImagePoolID, rbdVol.Pool,
rbdVol.ClusterID, imageUUID, volIDVersion)
if err != nil {
return false, err
}
klog.V(4).Infof(util.Log(ctx, "found existing volume (%s) with image name (%s) for request (%s)"),
rbdVol.VolID, rbdVol.RbdImageName, rbdVol.RequestName)
return true, nil
}
// reserveSnap is a helper routine to request a rbdSnapshot name reservation and generate the
// volume ID for the generated name
func reserveSnap(ctx context.Context, rbdSnap *rbdSnapshot, cr *util.Credentials) error {
var (
snapUUID string
err error
)
journalPoolID, imagePoolID, err := util.GetPoolIDs(ctx, rbdSnap.Monitors, rbdSnap.JournalPool, rbdSnap.Pool, cr)
if err != nil {
return err
}
snapUUID, rbdSnap.RbdSnapName, err = snapJournal.ReserveName(ctx, rbdSnap.Monitors, cr, rbdSnap.JournalPool, journalPoolID,
rbdSnap.Pool, imagePoolID, rbdSnap.RequestName, rbdSnap.NamePrefix, rbdSnap.RbdImageName, "")
if err != nil {
return err
}
rbdSnap.SnapID, err = util.GenerateVolID(ctx, rbdSnap.Monitors, cr, imagePoolID, rbdSnap.Pool,
rbdSnap.ClusterID, snapUUID, volIDVersion)
if err != nil {
return err
}
klog.V(4).Infof(util.Log(ctx, "generated Volume ID (%s) and image name (%s) for request name (%s)"),
rbdSnap.SnapID, rbdSnap.RbdSnapName, rbdSnap.RequestName)
return nil
}
func updateTopologyConstraints(rbdVol *rbdVolume, rbdSnap *rbdSnapshot) error {
var err error
if rbdSnap != nil {
// check if topology constraints matches snapshot pool
rbdVol.Topology, err = util.MatchTopologyForPool(rbdVol.TopologyPools,
rbdVol.TopologyRequirement, rbdSnap.Pool)
if err != nil {
return err
}
// update Pool, if it was topology constrained
if rbdVol.Topology != nil {
rbdVol.Pool = rbdSnap.Pool
}
return nil
}
// update request based on topology constrained parameters (if present)
poolName, dataPoolName, topology, err := util.FindPoolAndTopology(rbdVol.TopologyPools, rbdVol.TopologyRequirement)
if err != nil {
return err
}
if poolName != "" {
rbdVol.Pool = poolName
rbdVol.DataPool = dataPoolName
rbdVol.Topology = topology
}
return nil
}
// reserveVol is a helper routine to request a rbdVolume name reservation and generate the
// volume ID for the generated name
func reserveVol(ctx context.Context, rbdVol *rbdVolume, rbdSnap *rbdSnapshot, cr *util.Credentials) error {
var (
imageUUID string
err error
)
err = updateTopologyConstraints(rbdVol, rbdSnap)
if err != nil {
return err
}
journalPoolID, imagePoolID, err := util.GetPoolIDs(ctx, rbdVol.Monitors, rbdVol.JournalPool, rbdVol.Pool, cr)
if err != nil {
return err
}
kmsID := ""
if rbdVol.Encrypted {
kmsID = rbdVol.KMS.GetID()
}
imageUUID, rbdVol.RbdImageName, err = volJournal.ReserveName(ctx, rbdVol.Monitors, cr, rbdVol.JournalPool, journalPoolID,
rbdVol.Pool, imagePoolID, rbdVol.RequestName, rbdVol.NamePrefix, "", kmsID)
if err != nil {
return err
}
rbdVol.VolID, err = util.GenerateVolID(ctx, rbdVol.Monitors, cr, imagePoolID, rbdVol.Pool,
rbdVol.ClusterID, imageUUID, volIDVersion)
if err != nil {
return err
}
klog.V(4).Infof(util.Log(ctx, "generated Volume ID (%s) and image name (%s) for request name (%s)"),
rbdVol.VolID, rbdVol.RbdImageName, rbdVol.RequestName)
return nil
}
// undoSnapReservation is a helper routine to undo a name reservation for rbdSnapshot
func undoSnapReservation(ctx context.Context, rbdSnap *rbdSnapshot, cr *util.Credentials) error {
err := snapJournal.UndoReservation(ctx, rbdSnap.Monitors, cr, rbdSnap.JournalPool, rbdSnap.Pool,
rbdSnap.RbdSnapName, rbdSnap.RequestName)
return err
}
// undoVolReservation is a helper routine to undo a name reservation for rbdVolume
func undoVolReservation(ctx context.Context, rbdVol *rbdVolume, cr *util.Credentials) error {
err := volJournal.UndoReservation(ctx, rbdVol.Monitors, cr, rbdVol.JournalPool, rbdVol.Pool,
rbdVol.RbdImageName, rbdVol.RequestName)
return err
}

999
internal/rbd/rbd_util.go Normal file
View File

@ -0,0 +1,999 @@
/*
Copyright 2018 The Ceph-CSI Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package rbd
import (
"context"
"encoding/json"
"fmt"
"io/ioutil"
"os"
"os/exec"
"path/filepath"
"strconv"
"strings"
"time"
"github.com/ceph/ceph-csi/internal/util"
"github.com/ceph/go-ceph/rados"
librbd "github.com/ceph/go-ceph/rbd"
"github.com/container-storage-interface/spec/lib/go/csi"
"github.com/golang/protobuf/ptypes"
"github.com/golang/protobuf/ptypes/timestamp"
"github.com/pborman/uuid"
"github.com/pkg/errors"
"k8s.io/apimachinery/pkg/util/sets"
"k8s.io/cloud-provider/volume/helpers"
"k8s.io/klog"
)
const (
imageWatcherStr = "watcher="
// The following three values are used for 30 seconds timeout
// while waiting for RBD Watcher to expire.
rbdImageWatcherInitDelay = 1 * time.Second
rbdImageWatcherFactor = 1.4
rbdImageWatcherSteps = 10
rbdDefaultMounter = "rbd"
// Output strings returned during invocation of "ceph rbd task add remove <imagespec>" when
// command is not supported by ceph manager. Used to check errors and recover when the command
// is unsupported.
rbdTaskRemoveCmdInvalidString1 = "no valid command found"
rbdTaskRemoveCmdInvalidString2 = "Error EINVAL: invalid command"
rbdTaskRemoveCmdAccessDeniedMessage = "Error EACCES:"
// Encryption statuses for RbdImage
rbdImageEncrypted = "encrypted"
rbdImageRequiresEncryption = "requiresEncryption"
)
// rbdVolume represents a CSI volume and its RBD image specifics
type rbdVolume struct {
// RbdImageName is the name of the RBD image backing this rbdVolume. This does not have a
// JSON tag as it is not stashed in JSON encoded config maps in v1.0.0
// VolID is the volume ID that is exchanged with CSI drivers, identifying this rbdVol
// RequestName is the CSI generated volume name for the rbdVolume. This does not have a
// JSON tag as it is not stashed in JSON encoded config maps in v1.0.0
// VolName and MonValueFromSecret are retained from older plugin versions (<= 1.0.0)
// for backward compatibility reasons
// JournalPool is the ceph pool in which the CSI Journal is stored
// Pool is where the image journal and image is stored, and could be the same as `JournalPool`
// (retained as Pool instead of renaming to ImagePool or such, as this is referenced in the code extensively)
// DataPool is where the data for images in `Pool` are stored, this is used as the `--data-pool`
// argument when the pool is created, and is not used anywhere else
TopologyPools *[]util.TopologyConstrainedPool
TopologyRequirement *csi.TopologyRequirement
Topology map[string]string
RbdImageName string
NamePrefix string
VolID string `json:"volID"`
Monitors string `json:"monitors"`
JournalPool string
Pool string `json:"pool"`
DataPool string
ImageFeatures string `json:"imageFeatures"`
AdminID string `json:"adminId"`
UserID string `json:"userId"`
Mounter string `json:"mounter"`
ClusterID string `json:"clusterId"`
RequestName string
VolName string `json:"volName"`
MonValueFromSecret string `json:"monValueFromSecret"`
VolSize int64 `json:"volSize"`
DisableInUseChecks bool `json:"disableInUseChecks"`
Encrypted bool
KMS util.EncryptionKMS
// connection
conn *rados.Conn
}
// rbdSnapshot represents a CSI snapshot and its RBD snapshot specifics
type rbdSnapshot struct {
// SourceVolumeID is the volume ID of RbdImageName, that is exchanged with CSI drivers
// RbdImageName is the name of the RBD image, that is this rbdSnapshot's source image
// RbdSnapName is the name of the RBD snapshot backing this rbdSnapshot
// SnapID is the snapshot ID that is exchanged with CSI drivers, identifying this rbdSnapshot
// RequestName is the CSI generated snapshot name for the rbdSnapshot
// JournalPool is the ceph pool in which the CSI snapshot Journal is stored
// Pool is where the image snapshot journal and snapshot is stored, and could be the same as `JournalPool`
SourceVolumeID string
RbdImageName string
NamePrefix string
RbdSnapName string
SnapID string
Monitors string
JournalPool string
Pool string
CreatedAt *timestamp.Timestamp
SizeBytes int64
ClusterID string
RequestName string
}
var (
supportedFeatures = sets.NewString("layering")
// large interval and timeout, it should be longer than the maximum
// time an operation can take (until refcounting of the connections is
// available)
cpInterval = 15 * time.Minute
cpExpiry = 10 * time.Minute
connPool = util.NewConnPool(cpInterval, cpExpiry)
)
// createImage creates a new ceph image with provision and volume options.
func createImage(ctx context.Context, pOpts *rbdVolume, cr *util.Credentials) error {
volSzMiB := fmt.Sprintf("%dM", util.RoundOffVolSize(pOpts.VolSize))
options := librbd.NewRbdImageOptions()
logMsg := "rbd: create %s size %s (features: %s) using mon %s, pool %s "
if pOpts.DataPool != "" {
logMsg += fmt.Sprintf("data pool %s", pOpts.DataPool)
err := options.SetString(librbd.RbdImageOptionDataPool, pOpts.DataPool)
if err != nil {
return errors.Wrapf(err, "failed to set data pool")
}
}
klog.V(4).Infof(util.Log(ctx, logMsg),
pOpts.RbdImageName, volSzMiB, pOpts.ImageFeatures, pOpts.Monitors, pOpts.Pool)
if pOpts.ImageFeatures != "" {
features := imageFeaturesToUint64(ctx, pOpts.ImageFeatures)
err := options.SetUint64(librbd.RbdImageOptionFeatures, features)
if err != nil {
return errors.Wrapf(err, "failed to set image features")
}
}
ioctx, err := pOpts.getIoctx(cr)
if err != nil {
return errors.Wrapf(err, "failed to get IOContext")
}
defer ioctx.Destroy()
err = librbd.CreateImage(ioctx, pOpts.RbdImageName,
uint64(util.RoundOffVolSize(pOpts.VolSize)*helpers.MiB), options)
if err != nil {
return errors.Wrapf(err, "failed to create rbd image")
}
return nil
}
func (rv *rbdVolume) getIoctx(cr *util.Credentials) (*rados.IOContext, error) {
if rv.conn == nil {
conn, err := connPool.Get(rv.Pool, rv.Monitors, cr.ID, cr.KeyFile)
if err != nil {
return nil, errors.Wrapf(err, "failed to get connection")
}
rv.conn = conn
}
ioctx, err := rv.conn.OpenIOContext(rv.Pool)
if err != nil {
connPool.Put(rv.conn)
return nil, errors.Wrapf(err, "failed to open IOContext for pool %s", rv.Pool)
}
return ioctx, nil
}
func (rv *rbdVolume) Destroy() {
if rv.conn != nil {
connPool.Put(rv.conn)
}
}
// rbdStatus checks if there is watcher on the image.
// It returns true if there is a watcher on the image, otherwise returns false.
func rbdStatus(ctx context.Context, pOpts *rbdVolume, cr *util.Credentials) (bool, string, error) {
var output string
var cmd []byte
image := pOpts.RbdImageName
klog.V(4).Infof(util.Log(ctx, "rbd: status %s using mon %s, pool %s"), image, pOpts.Monitors, pOpts.Pool)
args := []string{"status", image, "--pool", pOpts.Pool, "-m", pOpts.Monitors, "--id", cr.ID, "--keyfile=" + cr.KeyFile}
cmd, err := execCommand("rbd", args)
output = string(cmd)
if err, ok := err.(*exec.Error); ok {
if err.Err == exec.ErrNotFound {
klog.Errorf(util.Log(ctx, "rbd cmd not found"))
// fail fast if command not found
return false, output, err
}
}
// If command never succeed, returns its last error.
if err != nil {
return false, output, err
}
if strings.Contains(output, imageWatcherStr) {
klog.V(4).Infof(util.Log(ctx, "rbd: watchers on %s: %s"), image, output)
return true, output, nil
}
klog.Warningf(util.Log(ctx, "rbd: no watchers on %s"), image)
return false, output, nil
}
// rbdManagerTaskDelete adds a ceph manager task to delete an rbd image, thus deleting
// it asynchronously. If command is not found returns a bool set to false
func rbdManagerTaskDeleteImage(ctx context.Context, pOpts *rbdVolume, cr *util.Credentials) (bool, error) {
var output []byte
args := []string{"rbd", "task", "add", "remove",
pOpts.Pool + "/" + pOpts.RbdImageName,
"--id", cr.ID,
"--keyfile=" + cr.KeyFile,
"-m", pOpts.Monitors,
}
output, err := execCommand("ceph", args)
if err != nil {
switch {
case strings.Contains(string(output), rbdTaskRemoveCmdInvalidString1) &&
strings.Contains(string(output), rbdTaskRemoveCmdInvalidString2):
klog.Warningf(util.Log(ctx, "cluster with cluster ID (%s) does not support Ceph manager based rbd image"+
" deletion (minimum ceph version required is v14.2.3)"), pOpts.ClusterID)
case strings.HasPrefix(string(output), rbdTaskRemoveCmdAccessDeniedMessage):
klog.Warningf(util.Log(ctx, "access denied to Ceph MGR-based RBD image deletion "+
"on cluster ID (%s)"), pOpts.ClusterID)
default:
klog.Warningf(util.Log(ctx, "uncaught error while scheduling an image deletion task: %s"), err)
}
return false, err
}
return true, err
}
// deleteImage deletes a ceph image with provision and volume options.
func deleteImage(ctx context.Context, pOpts *rbdVolume, cr *util.Credentials) error {
var output []byte
image := pOpts.RbdImageName
found, _, err := rbdStatus(ctx, pOpts, cr)
if err != nil {
return err
}
if found {
klog.Errorf(util.Log(ctx, "rbd is still being used "), image)
return fmt.Errorf("rbd %s is still being used", image)
}
klog.V(4).Infof(util.Log(ctx, "rbd: rm %s using mon %s, pool %s"), image, pOpts.Monitors, pOpts.Pool)
// attempt to use Ceph manager based deletion support if available
rbdCephMgrSupported, err := rbdManagerTaskDeleteImage(ctx, pOpts, cr)
if rbdCephMgrSupported && err != nil {
klog.Errorf(util.Log(ctx, "failed to add task to delete rbd image: %s/%s, %v"), pOpts.Pool, image, err)
return err
}
if !rbdCephMgrSupported {
// attempt older style deletion
args := []string{"rm", image, "--pool", pOpts.Pool, "--id", cr.ID, "-m", pOpts.Monitors,
"--keyfile=" + cr.KeyFile}
output, err = execCommand("rbd", args)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to delete rbd image: %s/%s, error: %v, command output: %s"), pOpts.Pool, image, err, string(output))
}
}
return err
}
// updateSnapWithImageInfo updates provided rbdSnapshot with information from on-disk data
// regarding the same
func updateSnapWithImageInfo(ctx context.Context, rbdSnap *rbdSnapshot, cr *util.Credentials) error {
snapInfo, err := getSnapInfo(ctx, rbdSnap.Monitors, cr, rbdSnap.Pool,
rbdSnap.RbdImageName, rbdSnap.RbdSnapName)
if err != nil {
return err
}
rbdSnap.SizeBytes = snapInfo.Size
tm, err := time.Parse(time.ANSIC, snapInfo.Timestamp)
if err != nil {
return err
}
rbdSnap.CreatedAt, err = ptypes.TimestampProto(tm)
return err
}
// updateVolWithImageInfo updates provided rbdVolume with information from on-disk data
// regarding the same
func updateVolWithImageInfo(ctx context.Context, rbdVol *rbdVolume, cr *util.Credentials) error {
imageInfo, err := getImageInfo(ctx, rbdVol.Monitors, cr, rbdVol.Pool, rbdVol.RbdImageName)
if err != nil {
return err
}
rbdVol.VolSize = imageInfo.Size
rbdVol.ImageFeatures = strings.Join(imageInfo.Features, ",")
return nil
}
// genSnapFromSnapID generates a rbdSnapshot structure from the provided identifier, updating
// the structure with elements from on-disk snapshot metadata as well
func genSnapFromSnapID(ctx context.Context, rbdSnap *rbdSnapshot, snapshotID string, cr *util.Credentials) error {
var (
options map[string]string
vi util.CSIIdentifier
)
options = make(map[string]string)
rbdSnap.SnapID = snapshotID
err := vi.DecomposeCSIID(rbdSnap.SnapID)
if err != nil {
klog.Errorf(util.Log(ctx, "error decoding snapshot ID (%s) (%s)"), err, rbdSnap.SnapID)
return err
}
rbdSnap.ClusterID = vi.ClusterID
options["clusterID"] = rbdSnap.ClusterID
rbdSnap.Monitors, _, err = getMonsAndClusterID(ctx, options)
if err != nil {
return err
}
rbdSnap.Pool, err = util.GetPoolName(ctx, rbdSnap.Monitors, cr, vi.LocationID)
if err != nil {
return err
}
rbdSnap.JournalPool = rbdSnap.Pool
imageAttributes, err := snapJournal.GetImageAttributes(ctx, rbdSnap.Monitors,
cr, rbdSnap.Pool, vi.ObjectUUID, true)
if err != nil {
return err
}
rbdSnap.RequestName = imageAttributes.RequestName
rbdSnap.RbdImageName = imageAttributes.SourceName
rbdSnap.RbdSnapName = imageAttributes.ImageName
// convert the journal pool ID to name, for use in DeleteSnapshot cases
if imageAttributes.JournalPoolID != util.InvalidPoolID {
rbdSnap.JournalPool, err = util.GetPoolName(ctx, rbdSnap.Monitors, cr, imageAttributes.JournalPoolID)
if err != nil {
// TODO: If pool is not found we may leak the image (as DeleteSnapshot will return success)
return err
}
}
err = updateSnapWithImageInfo(ctx, rbdSnap, cr)
return err
}
// genVolFromVolID generates a rbdVolume structure from the provided identifier, updating
// the structure with elements from on-disk image metadata as well
func genVolFromVolID(ctx context.Context, rbdVol *rbdVolume, volumeID string, cr *util.Credentials, secrets map[string]string) error {
var (
options map[string]string
vi util.CSIIdentifier
)
options = make(map[string]string)
// rbdVolume fields that are not filled up in this function are:
// Mounter, MultiNodeWritable
rbdVol.VolID = volumeID
err := vi.DecomposeCSIID(rbdVol.VolID)
if err != nil {
err = fmt.Errorf("error decoding volume ID (%s) (%s)", err, rbdVol.VolID)
return ErrInvalidVolID{err}
}
rbdVol.ClusterID = vi.ClusterID
options["clusterID"] = rbdVol.ClusterID
rbdVol.Monitors, _, err = getMonsAndClusterID(ctx, options)
if err != nil {
return err
}
rbdVol.Pool, err = util.GetPoolName(ctx, rbdVol.Monitors, cr, vi.LocationID)
if err != nil {
return err
}
rbdVol.JournalPool = rbdVol.Pool
imageAttributes, err := volJournal.GetImageAttributes(ctx, rbdVol.Monitors, cr,
rbdVol.Pool, vi.ObjectUUID, false)
if err != nil {
return err
}
if imageAttributes.KmsID != "" {
rbdVol.Encrypted = true
rbdVol.KMS, err = util.GetKMS(imageAttributes.KmsID, secrets)
if err != nil {
return err
}
}
rbdVol.RequestName = imageAttributes.RequestName
rbdVol.RbdImageName = imageAttributes.ImageName
// convert the journal pool ID to name, for use in DeleteVolume cases
if imageAttributes.JournalPoolID >= 0 {
rbdVol.JournalPool, err = util.GetPoolName(ctx, rbdVol.Monitors, cr, imageAttributes.JournalPoolID)
if err != nil {
// TODO: If pool is not found we may leak the image (as DeleteVolume will return success)
return err
}
}
err = updateVolWithImageInfo(ctx, rbdVol, cr)
return err
}
func execCommand(command string, args []string) ([]byte, error) {
// #nosec
cmd := exec.Command(command, args...)
return cmd.CombinedOutput()
}
func getMonsAndClusterID(ctx context.Context, options map[string]string) (monitors, clusterID string, err error) {
var ok bool
if clusterID, ok = options["clusterID"]; !ok {
err = errors.New("clusterID must be set")
return
}
if monitors, err = util.Mons(csiConfigFile, clusterID); err != nil {
klog.Errorf(util.Log(ctx, "failed getting mons (%s)"), err)
err = errors.Wrapf(err, "failed to fetch monitor list using clusterID (%s)", clusterID)
return
}
return
}
// isLegacyVolumeID checks if passed in volume ID string conforms to volume ID naming scheme used
// by the version 1.0.0 (legacy) of the plugin, and returns true if found to be conforming
func isLegacyVolumeID(volumeID string) bool {
// Version 1.0.0 volumeID format: "csi-rbd-vol-" + UUID string
// length: 12 ("csi-rbd-vol-") + 36 (UUID string)
// length check
if len(volumeID) != 48 {
return false
}
// Header check
if !strings.HasPrefix(volumeID, "csi-rbd-vol-") {
return false
}
// Trailer UUID format check
if uuid.Parse(volumeID[12:]) == nil {
return false
}
return true
}
// upadateMons function is used to update the rbdVolume.Monitors for volumes that were provisioned
// using the 1.0.0 version (legacy) of the plugin.
func updateMons(rbdVol *rbdVolume, options, credentials map[string]string) error {
var ok bool
// read monitors and MonValueFromSecret from options, else check passed in rbdVolume for
// MonValueFromSecret key in credentials
monInSecret := ""
if options != nil {
if rbdVol.Monitors, ok = options["monitors"]; !ok {
rbdVol.Monitors = ""
}
if monInSecret, ok = options["monValueFromSecret"]; !ok {
monInSecret = ""
}
} else {
monInSecret = rbdVol.MonValueFromSecret
}
// if monitors are present in secrets and we have the credentials, use monitors from the
// credentials overriding monitors from other sources
if monInSecret != "" && credentials != nil {
monsFromSecret, ok := credentials[monInSecret]
if ok {
rbdVol.Monitors = monsFromSecret
}
}
if rbdVol.Monitors == "" {
return errors.New("either monitors or monValueFromSecret must be set")
}
return nil
}
func genVolFromVolumeOptions(ctx context.Context, volOptions, credentials map[string]string, disableInUseChecks, isLegacyVolume bool) (*rbdVolume, error) {
var (
ok bool
err error
namePrefix string
encrypted string
)
rbdVol := &rbdVolume{}
rbdVol.Pool, ok = volOptions["pool"]
if !ok {
return nil, errors.New("missing required parameter pool")
}
rbdVol.DataPool = volOptions["dataPool"]
if namePrefix, ok = volOptions["volumeNamePrefix"]; ok {
rbdVol.NamePrefix = namePrefix
}
if isLegacyVolume {
err = updateMons(rbdVol, volOptions, credentials)
if err != nil {
return nil, err
}
} else {
rbdVol.Monitors, rbdVol.ClusterID, err = getMonsAndClusterID(ctx, volOptions)
if err != nil {
return nil, err
}
}
// if no image features is provided, it results in empty string
// which disable all RBD image features as we expected
imageFeatures, found := volOptions["imageFeatures"]
if found {
arr := strings.Split(imageFeatures, ",")
for _, f := range arr {
if !supportedFeatures.Has(f) {
return nil, fmt.Errorf("invalid feature %q for volume csi-rbdplugin, supported"+
" features are: %v", f, supportedFeatures)
}
}
rbdVol.ImageFeatures = imageFeatures
}
klog.V(3).Infof(util.Log(ctx, "setting disableInUseChecks on rbd volume to: %v"), disableInUseChecks)
rbdVol.DisableInUseChecks = disableInUseChecks
rbdVol.Mounter, ok = volOptions["mounter"]
if !ok {
rbdVol.Mounter = rbdDefaultMounter
}
rbdVol.Encrypted = false
encrypted, ok = volOptions["encrypted"]
if ok {
rbdVol.Encrypted, err = strconv.ParseBool(encrypted)
if err != nil {
return nil, fmt.Errorf(
"invalid value set in 'encrypted': %s (should be \"true\" or \"false\")", encrypted)
}
if rbdVol.Encrypted {
// deliberately ignore if parsing failed as GetKMS will return default
// implementation of kmsID is empty
kmsID := volOptions["encryptionKMSID"]
rbdVol.KMS, err = util.GetKMS(kmsID, credentials)
if err != nil {
return nil, fmt.Errorf("invalid encryption kms configuration: %s", err)
}
}
}
return rbdVol, nil
}
func genSnapFromOptions(ctx context.Context, rbdVol *rbdVolume, snapOptions map[string]string) *rbdSnapshot {
var err error
rbdSnap := &rbdSnapshot{}
rbdSnap.Pool = rbdVol.Pool
rbdSnap.JournalPool = rbdVol.JournalPool
rbdSnap.Monitors, rbdSnap.ClusterID, err = getMonsAndClusterID(ctx, snapOptions)
if err != nil {
rbdSnap.Monitors = rbdVol.Monitors
rbdSnap.ClusterID = rbdVol.ClusterID
}
if namePrefix, ok := snapOptions["snapshotNamePrefix"]; ok {
rbdSnap.NamePrefix = namePrefix
}
return rbdSnap
}
func hasSnapshotFeature(imageFeatures string) bool {
arr := strings.Split(imageFeatures, ",")
for _, f := range arr {
if f == "layering" {
return true
}
}
return false
}
// imageFeaturesToUint64 takes the comma separated image features and converts
// them to a RbdImageOptionFeatures value.
func imageFeaturesToUint64(ctx context.Context, imageFeatures string) uint64 {
features := uint64(0)
for _, f := range strings.Split(imageFeatures, ",") {
if f == "layering" {
features |= librbd.RbdFeatureLayering
} else {
klog.Warningf(util.Log(ctx, "rbd: image feature %s not recognized, skipping"), f)
}
}
return features
}
func protectSnapshot(ctx context.Context, pOpts *rbdSnapshot, cr *util.Credentials) error {
var output []byte
image := pOpts.RbdImageName
snapName := pOpts.RbdSnapName
klog.V(4).Infof(util.Log(ctx, "rbd: snap protect %s using mon %s, pool %s "), image, pOpts.Monitors, pOpts.Pool)
args := []string{"snap", "protect", "--pool", pOpts.Pool, "--snap", snapName, image, "--id",
cr.ID, "-m", pOpts.Monitors, "--keyfile=" + cr.KeyFile}
output, err := execCommand("rbd", args)
if err != nil {
return errors.Wrapf(err, "failed to protect snapshot, command output: %s", string(output))
}
return nil
}
func createSnapshot(ctx context.Context, pOpts *rbdSnapshot, cr *util.Credentials) error {
var output []byte
image := pOpts.RbdImageName
snapName := pOpts.RbdSnapName
klog.V(4).Infof(util.Log(ctx, "rbd: snap create %s using mon %s, pool %s"), image, pOpts.Monitors, pOpts.Pool)
args := []string{"snap", "create", "--pool", pOpts.Pool, "--snap", snapName, image,
"--id", cr.ID, "-m", pOpts.Monitors, "--keyfile=" + cr.KeyFile}
output, err := execCommand("rbd", args)
if err != nil {
return errors.Wrapf(err, "failed to create snapshot, command output: %s", string(output))
}
return nil
}
func unprotectSnapshot(ctx context.Context, pOpts *rbdSnapshot, cr *util.Credentials) error {
var output []byte
image := pOpts.RbdImageName
snapName := pOpts.RbdSnapName
klog.V(4).Infof(util.Log(ctx, "rbd: snap unprotect %s using mon %s, pool %s"), image, pOpts.Monitors, pOpts.Pool)
args := []string{"snap", "unprotect", "--pool", pOpts.Pool, "--snap", snapName, image, "--id",
cr.ID, "-m", pOpts.Monitors, "--keyfile=" + cr.KeyFile}
output, err := execCommand("rbd", args)
if err != nil {
return errors.Wrapf(err, "failed to unprotect snapshot, command output: %s", string(output))
}
return nil
}
func deleteSnapshot(ctx context.Context, pOpts *rbdSnapshot, cr *util.Credentials) error {
var output []byte
image := pOpts.RbdImageName
snapName := pOpts.RbdSnapName
klog.V(4).Infof(util.Log(ctx, "rbd: snap rm %s using mon %s, pool %s"), image, pOpts.Monitors, pOpts.Pool)
args := []string{"snap", "rm", "--pool", pOpts.Pool, "--snap", snapName, image, "--id",
cr.ID, "-m", pOpts.Monitors, "--keyfile=" + cr.KeyFile}
output, err := execCommand("rbd", args)
if err != nil {
return errors.Wrapf(err, "failed to delete snapshot, command output: %s", string(output))
}
if err := undoSnapReservation(ctx, pOpts, cr); err != nil {
klog.Errorf(util.Log(ctx, "failed to remove reservation for snapname (%s) with backing snap (%s) on image (%s) (%s)"),
pOpts.RequestName, pOpts.RbdSnapName, pOpts.RbdImageName, err)
}
return nil
}
func restoreSnapshot(ctx context.Context, pVolOpts *rbdVolume, pSnapOpts *rbdSnapshot, cr *util.Credentials) error {
var output []byte
image := pVolOpts.RbdImageName
snapName := pSnapOpts.RbdSnapName
klog.V(4).Infof(util.Log(ctx, "rbd: clone %s using mon %s, pool %s"), image, pVolOpts.Monitors, pVolOpts.Pool)
args := []string{"clone", pSnapOpts.Pool + "/" + pSnapOpts.RbdImageName + "@" + snapName,
pVolOpts.Pool + "/" + image, "--id", cr.ID, "-m", pVolOpts.Monitors, "--keyfile=" + cr.KeyFile}
output, err := execCommand("rbd", args)
if err != nil {
return errors.Wrapf(err, "failed to restore snapshot, command output: %s", string(output))
}
return nil
}
// getSnapshotMetadata fetches on-disk metadata about the snapshot and populates the passed in
// rbdSnapshot structure
func getSnapshotMetadata(ctx context.Context, pSnapOpts *rbdSnapshot, cr *util.Credentials) error {
imageName := pSnapOpts.RbdImageName
snapName := pSnapOpts.RbdSnapName
snapInfo, err := getSnapInfo(ctx, pSnapOpts.Monitors, cr, pSnapOpts.Pool, imageName, snapName)
if err != nil {
return err
}
pSnapOpts.SizeBytes = snapInfo.Size
tm, err := time.Parse(time.ANSIC, snapInfo.Timestamp)
if err != nil {
return err
}
pSnapOpts.CreatedAt, err = ptypes.TimestampProto(tm)
if err != nil {
return err
}
return nil
}
// imageInfo strongly typed JSON spec for image info
type imageInfo struct {
ObjectUUID string `json:"name"`
Size int64 `json:"size"`
Features []string `json:"features"`
CreatedAt string `json:"create_timestamp"`
}
// getImageInfo queries rbd about the given image and returns its metadata, and returns
// ErrImageNotFound if provided image is not found
func getImageInfo(ctx context.Context, monitors string, cr *util.Credentials, poolName, imageName string) (imageInfo, error) {
// rbd --format=json info [image-spec | snap-spec]
var imgInfo imageInfo
stdout, stderr, err := util.ExecCommand(
"rbd",
"-m", monitors,
"--id", cr.ID,
"--keyfile="+cr.KeyFile,
"-c", util.CephConfigPath,
"--format="+"json",
"info", poolName+"/"+imageName)
if err != nil {
klog.Errorf(util.Log(ctx, "failed getting information for image (%s): (%s)"), poolName+"/"+imageName, err)
if strings.Contains(string(stderr), "rbd: error opening image "+imageName+
": (2) No such file or directory") {
return imgInfo, ErrImageNotFound{imageName, err}
}
return imgInfo, err
}
err = json.Unmarshal(stdout, &imgInfo)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to parse JSON output of image info (%s): (%s)"),
poolName+"/"+imageName, err)
return imgInfo, fmt.Errorf("unmarshal failed: %+v. raw buffer response: %s",
err, string(stdout))
}
return imgInfo, nil
}
// snapInfo strongly typed JSON spec for snap ls rbd output
type snapInfo struct {
ID int64 `json:"id"`
Name string `json:"name"`
Size int64 `json:"size"`
Timestamp string `json:"timestamp"`
}
/*
getSnapInfo queries rbd about the snapshots of the given image and returns its metadata, and
returns ErrImageNotFound if provided image is not found, and ErrSnapNotFound if provided snap
is not found in the images snapshot list
*/
func getSnapInfo(ctx context.Context, monitors string, cr *util.Credentials, poolName, imageName, snapName string) (snapInfo, error) {
// rbd --format=json snap ls [image-spec]
var (
snpInfo snapInfo
snaps []snapInfo
)
stdout, stderr, err := util.ExecCommand(
"rbd",
"-m", monitors,
"--id", cr.ID,
"--keyfile="+cr.KeyFile,
"-c", util.CephConfigPath,
"--format="+"json",
"snap", "ls", poolName+"/"+imageName)
if err != nil {
klog.Errorf(util.Log(ctx, "failed getting snap (%s) information from image (%s): (%s)"),
snapName, poolName+"/"+imageName, err)
if strings.Contains(string(stderr), "rbd: error opening image "+imageName+
": (2) No such file or directory") {
return snpInfo, ErrImageNotFound{imageName, err}
}
return snpInfo, err
}
err = json.Unmarshal(stdout, &snaps)
if err != nil {
klog.Errorf(util.Log(ctx, "failed to parse JSON output of image snap list (%s): (%s)"),
poolName+"/"+imageName, err)
return snpInfo, fmt.Errorf("unmarshal failed: %+v. raw buffer response: %s",
err, string(stdout))
}
for _, snap := range snaps {
if snap.Name == snapName {
return snap, nil
}
}
return snpInfo, ErrSnapNotFound{snapName, fmt.Errorf("snap (%s) for image (%s) not found",
snapName, poolName+"/"+imageName)}
}
// rbdImageMetadataStash strongly typed JSON spec for stashed RBD image metadata
type rbdImageMetadataStash struct {
Version int `json:"Version"`
Pool string `json:"pool"`
ImageName string `json:"image"`
NbdAccess bool `json:"accessType"`
Encrypted bool `json:"encrypted"`
}
// file name in which image metadata is stashed
const stashFileName = "image-meta.json"
// stashRBDImageMetadata stashes required fields into the stashFileName at the passed in path, in
// JSON format
func stashRBDImageMetadata(volOptions *rbdVolume, path string) error {
var imgMeta = rbdImageMetadataStash{
Version: 2, // there are no checks for this at present
Pool: volOptions.Pool,
ImageName: volOptions.RbdImageName,
Encrypted: volOptions.Encrypted,
}
imgMeta.NbdAccess = false
if volOptions.Mounter == rbdTonbd && hasNBD {
imgMeta.NbdAccess = true
}
encodedBytes, err := json.Marshal(imgMeta)
if err != nil {
return fmt.Errorf("failed to marshall JSON image metadata for image (%s) in pool (%s): (%v)",
volOptions.RbdImageName, volOptions.Pool, err)
}
fPath := filepath.Join(path, stashFileName)
err = ioutil.WriteFile(fPath, encodedBytes, 0600)
if err != nil {
return fmt.Errorf("failed to stash JSON image metadata for image (%s) in pool (%s) at path (%s): (%v)",
volOptions.RbdImageName, volOptions.Pool, fPath, err)
}
return nil
}
// lookupRBDImageMetadataStash reads and returns stashed image metadata at passed in path
func lookupRBDImageMetadataStash(path string) (rbdImageMetadataStash, error) {
var imgMeta rbdImageMetadataStash
fPath := filepath.Join(path, stashFileName)
encodedBytes, err := ioutil.ReadFile(fPath) // #nosec - intended reading from fPath
if err != nil {
if !os.IsNotExist(err) {
return imgMeta, fmt.Errorf("failed to read stashed JSON image metadata from path (%s): (%v)", fPath, err)
}
return imgMeta, ErrMissingStash{err}
}
err = json.Unmarshal(encodedBytes, &imgMeta)
if err != nil {
return imgMeta, fmt.Errorf("failed to unmarshall stashed JSON image metadata from path (%s): (%v)", fPath, err)
}
return imgMeta, nil
}
// cleanupRBDImageMetadataStash cleans up any stashed metadata at passed in path
func cleanupRBDImageMetadataStash(path string) error {
fPath := filepath.Join(path, stashFileName)
if err := os.Remove(fPath); err != nil {
return fmt.Errorf("failed to cleanup stashed JSON data (%s): (%v)", fPath, err)
}
return nil
}
// resizeRBDImage resizes the given volume to new size
func resizeRBDImage(rbdVol *rbdVolume, cr *util.Credentials) error {
var output []byte
mon := rbdVol.Monitors
image := rbdVol.RbdImageName
volSzMiB := fmt.Sprintf("%dM", util.RoundOffVolSize(rbdVol.VolSize))
args := []string{"resize", image, "--size", volSzMiB, "--pool", rbdVol.Pool, "--id", cr.ID, "-m", mon, "--keyfile=" + cr.KeyFile}
output, err := execCommand("rbd", args)
if err != nil {
return errors.Wrapf(err, "failed to resize rbd image, command output: %s", string(output))
}
return nil
}
func ensureEncryptionMetadataSet(ctx context.Context, cr *util.Credentials, rbdVol *rbdVolume) error {
var vi util.CSIIdentifier
err := vi.DecomposeCSIID(rbdVol.VolID)
if err != nil {
err = fmt.Errorf("error decoding volume ID (%s) (%s)", rbdVol.VolID, err)
return ErrInvalidVolID{err}
}
rbdImageName := volJournal.GetNameForUUID(rbdVol.NamePrefix, vi.ObjectUUID, false)
imageSpec := rbdVol.Pool + "/" + rbdImageName
err = util.SaveRbdImageEncryptionStatus(ctx, cr, rbdVol.Monitors, imageSpec, rbdImageRequiresEncryption)
if err != nil {
return fmt.Errorf("failed to save encryption status for %s: %v", imageSpec, err)
}
return nil
}

View File

@ -0,0 +1,56 @@
/*
Copyright 2020 The Ceph-CSI Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package rbd
import (
"testing"
)
func TestIsLegacyVolumeID(t *testing.T) {
tests := []struct {
volID string
isLegacy bool
}{
{"prefix-bda37d42-9979-420f-9389-74362f3f98f6", false},
{"csi-rbd-vo-f997e783-ff00-48b0-8cc7-30cb36c3df3d", false},
{"csi-rbd-vol-this-is-clearly-not-a-valid-UUID----", false},
{"csi-rbd-vol-b82f27de-3b3a-43f2-b5e7-9f8d0aad04e9", true},
}
for _, test := range tests {
if got := isLegacyVolumeID(test.volID); got != test.isLegacy {
t.Errorf("isLegacyVolumeID(%s) = %t, want %t", test.volID, got, test.isLegacy)
}
}
}
func TestHasSnapshotFeature(t *testing.T) {
tests := []struct {
features string
hasFeature bool
}{
{"foo", false},
{"foo,bar", false},
{"foo,layering,bar", true},
}
for _, test := range tests {
if got := hasSnapshotFeature(test.features); got != test.hasFeature {
t.Errorf("hasSnapshotFeature(%s) = %t, want %t", test.features, got, test.hasFeature)
}
}
}