Add topology support to ceph-csi

Signed-off-by: ShyamsundarR <srangana@redhat.com>
This commit is contained in:
ShyamsundarR
2020-01-24 11:26:56 -05:00
committed by mergify[bot]
parent 5475022bc3
commit 5c4abf8347
31 changed files with 1017 additions and 273 deletions

View File

@ -29,6 +29,9 @@ import (
"k8s.io/klog"
)
// InvalidPoolID used to denote an invalid pool
const InvalidPoolID int64 = -1
// ExecCommand executes passed in program with args and returns separate stdout and stderr streams
func ExecCommand(program string, args ...string) (stdout, stderr []byte, err error) {
var (
@ -117,6 +120,26 @@ func GetPoolName(ctx context.Context, monitors string, cr *Credentials, poolID i
return "", ErrPoolNotFound{string(poolID), fmt.Errorf("pool ID (%d) not found in Ceph cluster", poolID)}
}
// GetPoolIDs searches a list of pools in a cluster and returns the IDs of the pools that matches
// the passed in pools
// TODO this should take in a list and return a map[string(poolname)]int64(poolID)
func GetPoolIDs(ctx context.Context, monitors, journalPool, imagePool string, cr *Credentials) (int64, int64, error) {
journalPoolID, err := GetPoolID(ctx, monitors, cr, journalPool)
if err != nil {
return InvalidPoolID, InvalidPoolID, err
}
imagePoolID := journalPoolID
if imagePool != journalPool {
imagePoolID, err = GetPoolID(ctx, monitors, cr, imagePool)
if err != nil {
return InvalidPoolID, InvalidPoolID, err
}
}
return journalPoolID, imagePoolID, nil
}
// SetOMapKeyValue sets the given key and value into the provided Ceph omap name
func SetOMapKeyValue(ctx context.Context, monitors string, cr *Credentials, poolName, namespace, oMapName, oMapKey, keyValue string) error {
// Command: "rados <options> setomapval oMapName oMapKey keyValue"

255
pkg/util/topology.go Normal file
View File

@ -0,0 +1,255 @@
/*
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 util
import (
"context"
"encoding/json"
"fmt"
"strings"
"github.com/container-storage-interface/spec/lib/go/csi"
metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/klog"
)
const (
keySeparator rune = '/'
labelSeparator string = ","
)
func k8sGetNodeLabels(nodeName string) (map[string]string, error) {
client := NewK8sClient()
node, err := client.CoreV1().Nodes().Get(context.TODO(), nodeName, metav1.GetOptions{})
if err != nil {
return nil, fmt.Errorf("failed to get node (%s) information : %v", nodeName, err)
}
return node.GetLabels(), nil
}
// GetTopologyFromDomainLabels returns the CSI topology map, determined from
// the domain labels and their values from the CO system
// Expects domainLabels in arg to be in the format "[prefix/]<name>,[prefix/]<name>,...",
func GetTopologyFromDomainLabels(domainLabels, nodeName, driverName string) (map[string]string, error) {
if domainLabels == "" {
return nil, nil
}
// size checks on domain label prefix
topologyPrefix := strings.ToLower("topology." + driverName)
if len(topologyPrefix) > 63 {
return nil, fmt.Errorf("computed topology label prefix (%s) for node exceeds length limits", topologyPrefix)
}
// driverName is validated, and we are adding a lowercase "topology." to it, so no validation for conformance
// Convert passed in labels to a map, and check for uniqueness
labelsToRead := strings.SplitN(domainLabels, labelSeparator, -1)
klog.Infof("passed in node labels for processing : %+v", labelsToRead)
labelsIn := make(map[string]bool)
labelCount := 0
for _, label := range labelsToRead {
// as we read the labels from k8s, and check for missing labels,
// no label conformance checks here
if _, ok := labelsIn[label]; ok {
return nil, fmt.Errorf("duplicate label (%s) found in domain labels", label)
}
labelsIn[label] = true
labelCount++
}
nodeLabels, err := k8sGetNodeLabels(nodeName)
if err != nil {
return nil, err
}
// Determine values for requested labels from node labels
domainMap := make(map[string]string)
found := 0
for key, value := range nodeLabels {
if _, ok := labelsIn[key]; !ok {
continue
}
// label found split name component and store value
nameIdx := strings.IndexRune(key, keySeparator)
domain := key[nameIdx+1:]
domainMap[domain] = value
labelsIn[key] = false
found++
}
// Ensure all labels are found
if found != labelCount {
missingLabels := []string{}
for key, missing := range labelsIn {
if missing {
missingLabels = append(missingLabels, key)
}
}
return nil, fmt.Errorf("missing domain labels %v on node (%s)", missingLabels, nodeName)
}
klog.Infof("list of domains processed : %+v", domainMap)
topology := make(map[string]string)
for domain, value := range domainMap {
topology[topologyPrefix+"/"+domain] = value
// TODO: when implementing domain takeover/giveback, enable a domain value that can remain pinned to the node
// topology["topology."+driverName+"/"+domain+"-pinned"] = value
}
return topology, nil
}
type topologySegment struct {
DomainLabel string `json:"domainLabel"`
DomainValue string `json:"value"`
}
// TopologyConstrainedPool stores the pool name and a list of its associated topology domain values
type TopologyConstrainedPool struct {
PoolName string `json:"poolName"`
DomainSegments []topologySegment `json:"domainSegments"`
}
// GetTopologyFromRequest extracts TopologyConstrainedPools and passed in accessibility constraints
// from a CSI CreateVolume request
func GetTopologyFromRequest(req *csi.CreateVolumeRequest) (*[]TopologyConstrainedPool, *csi.TopologyRequirement, error) {
var (
topologyPools []TopologyConstrainedPool
)
// check if parameters have pool configuration pertaining to topology
topologyPoolsStr := req.GetParameters()["topologyConstrainedPools"]
if topologyPoolsStr == "" {
return nil, nil, nil
}
// check if there are any accessibility requirements in the request
accessibilityRequirements := req.GetAccessibilityRequirements()
if accessibilityRequirements == nil {
return nil, nil, nil
}
// extract topology based pools configuration
err := json.Unmarshal([]byte(strings.Replace(topologyPoolsStr, "\n", " ", -1)), &topologyPools)
if err != nil {
return nil, nil, fmt.Errorf("failed to parse JSON encoded topology constrained pools parameter (%s): %v", topologyPoolsStr, err)
}
return &topologyPools, accessibilityRequirements, nil
}
// MatchTopologyForPool returns the topology map, if the passed in pool matches any
// passed in accessibility constraints
func MatchTopologyForPool(topologyPools *[]TopologyConstrainedPool,
accessibilityRequirements *csi.TopologyRequirement, poolName string) (map[string]string, error) {
var topologyPool []TopologyConstrainedPool
if topologyPools == nil || accessibilityRequirements == nil {
return nil, nil
}
// find the pool in the list of topology based pools
for _, value := range *topologyPools {
if value.PoolName == poolName {
topologyPool = append(topologyPool, value)
break
}
}
if len(topologyPool) == 0 {
return nil, fmt.Errorf("none of the configured topology pools (%+v) matched passed in pool name (%s)",
topologyPools, poolName)
}
_, topology, err := FindPoolAndTopology(&topologyPool, accessibilityRequirements)
return topology, err
}
// FindPoolAndTopology loops through passed in "topologyPools" and also related
// accessibility requirements, to determine which pool matches the requirement.
// The return variables are, image poolname, data poolname, and topology map of
// matched requirement
func FindPoolAndTopology(topologyPools *[]TopologyConstrainedPool,
accessibilityRequirements *csi.TopologyRequirement) (string, map[string]string, error) {
if topologyPools == nil || accessibilityRequirements == nil {
return "", nil, nil
}
// select pool that fits first topology constraint preferred requirements
for _, topology := range accessibilityRequirements.GetPreferred() {
poolName := matchPoolToTopology(topologyPools, topology)
if poolName != "" {
return poolName, topology.GetSegments(), nil
}
}
// If preferred mismatches, check requisite for a fit
for _, topology := range accessibilityRequirements.GetRequisite() {
poolName := matchPoolToTopology(topologyPools, topology)
if poolName != "" {
return poolName, topology.GetSegments(), nil
}
}
return "", nil, fmt.Errorf("none of the topology constrained pools matched requested "+
"topology constraints : pools (%+v) requested topology (%+v)",
*topologyPools, *accessibilityRequirements)
}
// matchPoolToTopology loops through passed in pools, and for each pool checks if all
// requested topology segments are present and match the request, returning the first pool
// that hence matches (or an empty string if none match)
func matchPoolToTopology(topologyPools *[]TopologyConstrainedPool, topology *csi.Topology) string {
domainMap := extractDomainsFromlabels(topology)
// check if any pool matches all the domain keys and values
for _, topologyPool := range *topologyPools {
mismatch := false
// match all pool topology labels to requested topology
for _, segment := range topologyPool.DomainSegments {
if domainValue, ok := domainMap[segment.DomainLabel]; !ok || domainValue != segment.DomainValue {
mismatch = true
break
}
}
if mismatch {
continue
}
return topologyPool.PoolName
}
return ""
}
// extractDomainsFromlabels returns the domain name map, from passed in domain segments,
// which is of the form [prefix/]<name>
func extractDomainsFromlabels(topology *csi.Topology) map[string]string {
domainMap := make(map[string]string)
for domainKey, value := range topology.GetSegments() {
domainIdx := strings.IndexRune(domainKey, keySeparator)
domain := domainKey[domainIdx+1:]
domainMap[domain] = value
}
return domainMap
}

View File

@ -81,6 +81,7 @@ type Config struct {
InstanceID string // unique ID distinguishing this instance of Ceph CSI
MetadataStorage string // metadata persistence method [node|k8s_configmap]
PluginPath string // location of cephcsi plugin
DomainLabels string // list of domain labels to read from the node
// cephfs related flags
MountCacheDir string // mount info cache save dir
@ -147,15 +148,19 @@ func ValidateDriverName(driverName string) error {
// GenerateVolID generates a volume ID based on passed in parameters and version, to be returned
// to the CO system
func GenerateVolID(ctx context.Context, monitors string, cr *Credentials, pool, clusterID, objUUID string, volIDVersion uint16) (string, error) {
poolID, err := GetPoolID(ctx, monitors, cr, pool)
if err != nil {
return "", err
func GenerateVolID(ctx context.Context, monitors string, cr *Credentials, locationID int64, pool, clusterID, objUUID string, volIDVersion uint16) (string, error) {
var err error
if locationID == InvalidPoolID {
locationID, err = GetPoolID(ctx, monitors, cr, pool)
if err != nil {
return "", err
}
}
// generate the volume ID to return to the CO system
vi := CSIIdentifier{
LocationID: poolID,
LocationID: locationID,
EncodingVersion: volIDVersion,
ClusterID: clusterID,
ObjectUUID: objUUID,

View File

@ -18,13 +18,19 @@ package util
import (
"context"
"encoding/binary"
"encoding/hex"
"fmt"
"strings"
"github.com/pborman/uuid"
"github.com/pkg/errors"
"k8s.io/klog"
)
// Length of string representation of uuid, xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx is 36 bytes
const uuidEncodedLength = 36
/*
RADOS omaps usage:
@ -103,6 +109,7 @@ const (
defaultSnapshotNamingPrefix string = "csi-snap-"
)
// CSIJournal defines the interface and the required key names for the above RADOS based OMaps
type CSIJournal struct {
// csiDirectory is the name of the CSI volumes object map that contains CSI volume-name (or
// snapshot name) based keys
@ -122,6 +129,10 @@ type CSIJournal struct {
// of this Ceph volume
csiImageKey string
// pool ID where csiDirectory is maintained, as it can be different from where the ceph volume
// object map is maintained, during topology based provisioning
csiJournalPool string
// source volume name key in per Ceph snapshot object map, containing Ceph source volume uuid
// for which the snapshot was created
cephSnapSourceKey string
@ -141,6 +152,7 @@ func NewCSIVolumeJournal() *CSIJournal {
cephUUIDDirectoryPrefix: "csi.volume.",
csiNameKey: "csi.volname",
csiImageKey: "csi.imagename",
csiJournalPool: "csi.journalpool",
cephSnapSourceKey: "",
namespace: "",
encryptKMSKey: "csi.volume.encryptKMS",
@ -155,6 +167,7 @@ func NewCSISnapshotJournal() *CSIJournal {
cephUUIDDirectoryPrefix: "csi.snap.",
csiNameKey: "csi.snapname",
csiImageKey: "csi.imagename",
csiJournalPool: "csi.journalpool",
cephSnapSourceKey: "csi.source",
namespace: "",
encryptKMSKey: "csi.volume.encryptKMS",
@ -183,6 +196,14 @@ func (cj *CSIJournal) SetNamespace(ns string) {
cj.namespace = ns
}
// ImageData contains image name and stored CSI properties
type ImageData struct {
ImageUUID string
ImagePool string
ImagePoolID int64
ImageAttributes *ImageAttributes
}
/*
CheckReservation checks if given request name contains a valid reservation
- If there is a valid reservation, then the corresponding UUID for the volume/snapshot is returned
@ -198,71 +219,114 @@ Return values:
there was no reservation found
- error: non-nil in case of any errors
*/
func (cj *CSIJournal) CheckReservation(ctx context.Context, monitors string, cr *Credentials, pool, reqName, namePrefix, parentName, kmsConf string) (string, error) {
var snapSource bool
func (cj *CSIJournal) CheckReservation(ctx context.Context, monitors string, cr *Credentials,
journalPool, reqName, namePrefix, parentName, kmsConfig string) (*ImageData, error) {
var (
snapSource bool
objUUID string
savedImagePool string
savedImagePoolID int64 = InvalidPoolID
)
if parentName != "" {
if cj.cephSnapSourceKey == "" {
err := errors.New("invalid request, cephSnapSourceKey is nil")
return "", err
return nil, err
}
snapSource = true
}
// check if request name is already part of the directory omap
objUUID, err := GetOMapValue(ctx, monitors, cr, pool, cj.namespace, cj.csiDirectory,
objUUIDAndPool, err := GetOMapValue(ctx, monitors, cr, journalPool, cj.namespace, cj.csiDirectory,
cj.csiNameKeyPrefix+reqName)
if err != nil {
// error should specifically be not found, for volume to be absent, any other error
// is not conclusive, and we should not proceed
switch err.(type) {
case ErrKeyNotFound, ErrPoolNotFound:
return "", nil
return nil, nil
}
return "", err
return nil, err
}
savedReqName, _, savedReqParentName, savedKms, err := cj.GetObjectUUIDData(ctx, monitors, cr, pool,
// check UUID only encoded value
if len(objUUIDAndPool) == uuidEncodedLength {
objUUID = objUUIDAndPool
savedImagePool = journalPool
} else { // check poolID/UUID encoding; extract the vol UUID and pool name
var buf64 []byte
components := strings.Split(objUUIDAndPool, "/")
objUUID = components[1]
savedImagePoolIDStr := components[0]
buf64, err = hex.DecodeString(savedImagePoolIDStr)
if err != nil {
return nil, err
}
savedImagePoolID = int64(binary.BigEndian.Uint64(buf64))
savedImagePool, err = GetPoolName(ctx, monitors, cr, savedImagePoolID)
if err != nil {
if _, ok := err.(ErrPoolNotFound); ok {
err = cj.UndoReservation(ctx, monitors, cr, journalPool, "", "", reqName)
}
return nil, err
}
}
savedImageAttributes, err := cj.GetImageAttributes(ctx, monitors, cr, savedImagePool,
objUUID, snapSource)
if err != nil {
// error should specifically be not found, for image to be absent, any other error
// is not conclusive, and we should not proceed
if _, ok := err.(ErrKeyNotFound); ok {
err = cj.UndoReservation(ctx, monitors, cr, pool, cj.GetNameForUUID(namePrefix, objUUID, snapSource), reqName)
err = cj.UndoReservation(ctx, monitors, cr, journalPool, savedImagePool,
cj.GetNameForUUID(namePrefix, objUUID, snapSource), reqName)
}
return "", err
return nil, err
}
// check if UUID key points back to the request name
if savedReqName != reqName {
if savedImageAttributes.RequestName != reqName {
// NOTE: This should never be possible, hence no cleanup, but log error
// and return, as cleanup may need to occur manually!
return "", fmt.Errorf("internal state inconsistent, omap names mismatch,"+
return nil, fmt.Errorf("internal state inconsistent, omap names mismatch,"+
" request name (%s) volume UUID (%s) volume omap name (%s)",
reqName, objUUID, savedReqName)
reqName, objUUID, savedImageAttributes.RequestName)
}
if kmsConf != "" {
if savedKms != kmsConf {
return "", fmt.Errorf("internal state inconsistent, omap encryption KMS"+
if kmsConfig != "" {
if savedImageAttributes.KmsID != kmsConfig {
return nil, fmt.Errorf("internal state inconsistent, omap encryption KMS"+
" mismatch, request KMS (%s) volume UUID (%s) volume omap KMS (%s)",
kmsConf, objUUID, savedKms)
kmsConfig, objUUID, savedImageAttributes.KmsID)
}
}
// TODO: skipping due to excessive poolID to poolname call, also this should never happen!
// check if journal pool points back to the passed in journal pool
// if savedJournalPoolID != journalPoolID {
if snapSource {
// check if source UUID key points back to the parent volume passed in
if savedReqParentName != parentName {
if savedImageAttributes.SourceName != parentName {
// NOTE: This can happen if there is a snapname conflict, and we already have a snapshot
// with the same name pointing to a different UUID as the source
err = fmt.Errorf("snapname points to different volume, request name (%s)"+
" source name (%s) saved source name (%s)",
reqName, parentName, savedReqParentName)
return "", ErrSnapNameConflict{reqName, err}
reqName, parentName, savedImageAttributes.SourceName)
return nil, ErrSnapNameConflict{reqName, err}
}
}
return objUUID, nil
imageData := &ImageData{
ImageUUID: objUUID,
ImagePool: savedImagePool,
ImagePoolID: savedImagePoolID,
ImageAttributes: savedImageAttributes,
}
return imageData, nil
}
/*
@ -274,30 +338,37 @@ prior to cleaning up the reservation
NOTE: As the function manipulates omaps, it should be called with a lock against the request name
held, to prevent parallel operations from modifying the state of the omaps for this request name.
Input arguments:
- csiJournalPool: Pool name that holds the CSI request name based journal
- volJournalPool: Pool name that holds the image/subvolume and the per-image journal (may be
different if image is created in a topology constrained pool)
*/
func (cj *CSIJournal) UndoReservation(ctx context.Context, monitors string, cr *Credentials, pool, volName, reqName string) error {
func (cj *CSIJournal) UndoReservation(ctx context.Context, monitors string, cr *Credentials,
csiJournalPool, volJournalPool, volName, reqName string) error {
// delete volume UUID omap (first, inverse of create order)
// TODO: Check cases where volName can be empty, and we need to just cleanup the reqName
if len(volName) < 36 {
return fmt.Errorf("unable to parse UUID from %s, too short", volName)
}
if volName != "" {
if len(volName) < 36 {
return fmt.Errorf("unable to parse UUID from %s, too short", volName)
}
imageUUID := volName[len(volName)-36:]
if valid := uuid.Parse(imageUUID); valid == nil {
return fmt.Errorf("failed parsing UUID in %s", volName)
}
imageUUID := volName[len(volName)-36:]
if valid := uuid.Parse(imageUUID); valid == nil {
return fmt.Errorf("failed parsing UUID in %s", volName)
}
err := RemoveObject(ctx, monitors, cr, pool, cj.namespace, cj.cephUUIDDirectoryPrefix+imageUUID)
if err != nil {
if _, ok := err.(ErrObjectNotFound); !ok {
klog.Errorf(Log(ctx, "failed removing oMap %s (%s)"), cj.cephUUIDDirectoryPrefix+imageUUID, err)
return err
err := RemoveObject(ctx, monitors, cr, volJournalPool, cj.namespace, cj.cephUUIDDirectoryPrefix+imageUUID)
if err != nil {
if _, ok := err.(ErrObjectNotFound); !ok {
klog.Errorf(Log(ctx, "failed removing oMap %s (%s)"), cj.cephUUIDDirectoryPrefix+imageUUID, err)
return err
}
}
}
// delete the request name key (last, inverse of create order)
err = RemoveOMapKey(ctx, monitors, cr, pool, cj.namespace, cj.csiDirectory,
err := RemoveOMapKey(ctx, monitors, cr, csiJournalPool, cj.namespace, cj.csiDirectory,
cj.csiNameKeyPrefix+reqName)
if err != nil {
klog.Errorf(Log(ctx, "failed removing oMap key %s (%s)"), cj.csiNameKeyPrefix+reqName, err)
@ -346,13 +417,31 @@ pointers to the CSI generated request names.
NOTE: As the function manipulates omaps, it should be called with a lock against the request name
held, to prevent parallel operations from modifying the state of the omaps for this request name.
Input arguments:
- journalPool: Pool where the CSI journal is stored (maybe different than the pool where the
image/subvolume is created duw to topology constraints)
- journalPoolID: pool ID of the journalPool
- imagePool: Pool where the image/subvolume is created
- imagePoolID: pool ID of the imagePool
- reqName: Name of the volume request received
- namePrefix: Prefix to use when generating the image/subvolume name (suffix is an auto-genetated UUID)
- parentName: Name of the parent image/subvolume if reservation is for a snapshot (optional)
- kmsConf: Name of the key management service used to encrypt the image (optional)
Return values:
- string: Contains the UUID that was reserved for the passed in reqName
- string: Contains the image name that was reserved for the passed in reqName
- error: non-nil in case of any errors
*/
func (cj *CSIJournal) ReserveName(ctx context.Context, monitors string, cr *Credentials, pool, reqName, namePrefix, parentName, kmsConf string) (string, string, error) {
var snapSource bool
func (cj *CSIJournal) ReserveName(ctx context.Context, monitors string, cr *Credentials,
journalPool string, journalPoolID int64,
imagePool string, imagePoolID int64,
reqName, namePrefix, parentName, kmsConf string) (string, string, error) {
// TODO: Take in-arg as ImageAttributes?
var (
snapSource bool
nameKeyVal string
)
if parentName != "" {
if cj.cephSnapSourceKey == "" {
@ -366,54 +455,81 @@ func (cj *CSIJournal) ReserveName(ctx context.Context, monitors string, cr *Cred
// NOTE: If any service loss occurs post creation of the UUID directory, and before
// setting the request name key (csiNameKey) to point back to the UUID directory, the
// UUID directory key will be leaked
volUUID, err := reserveOMapName(ctx, monitors, cr, pool, cj.namespace, cj.cephUUIDDirectoryPrefix)
volUUID, err := reserveOMapName(ctx, monitors, cr, imagePool, cj.namespace, cj.cephUUIDDirectoryPrefix)
if err != nil {
return "", "", err
}
imageName := cj.GetNameForUUID(namePrefix, volUUID, snapSource)
// Create request name (csiNameKey) key in csiDirectory and store the UUId based
// volume name into it
err = SetOMapKeyValue(ctx, monitors, cr, pool, cj.namespace, cj.csiDirectory,
cj.csiNameKeyPrefix+reqName, volUUID)
// Create request name (csiNameKey) key in csiDirectory and store the UUID based
// volume name and optionally the image pool location into it
if journalPool != imagePool && imagePoolID != InvalidPoolID {
buf64 := make([]byte, 8)
binary.BigEndian.PutUint64(buf64, uint64(imagePoolID))
poolIDEncodedHex := hex.EncodeToString(buf64)
nameKeyVal = poolIDEncodedHex + "/" + volUUID
} else {
nameKeyVal = volUUID
}
err = SetOMapKeyValue(ctx, monitors, cr, journalPool, cj.namespace, cj.csiDirectory,
cj.csiNameKeyPrefix+reqName, nameKeyVal)
if err != nil {
return "", "", err
}
defer func() {
if err != nil {
klog.Warningf(Log(ctx, "reservation failed for volume: %s"), reqName)
errDefer := cj.UndoReservation(ctx, monitors, cr, pool, imageName, reqName)
errDefer := cj.UndoReservation(ctx, monitors, cr, imagePool, journalPool, imageName, reqName)
if errDefer != nil {
klog.Warningf(Log(ctx, "failed undoing reservation of volume: %s (%v)"), reqName, errDefer)
}
}
}()
// Update UUID directory to store CSI request name and image name
err = SetOMapKeyValue(ctx, monitors, cr, pool, cj.namespace, cj.cephUUIDDirectoryPrefix+volUUID,
// NOTE: UUID directory is stored on the same pool as the image, helps determine image attributes
// and also CSI journal pool, when only the VolumeID is passed in (e.g DeleteVolume/DeleteSnapshot,
// VolID during CreateSnapshot).
// Update UUID directory to store CSI request name
err = SetOMapKeyValue(ctx, monitors, cr, imagePool, cj.namespace, cj.cephUUIDDirectoryPrefix+volUUID,
cj.csiNameKey, reqName)
if err != nil {
return "", "", err
}
err = SetOMapKeyValue(ctx, monitors, cr, pool, cj.namespace, cj.cephUUIDDirectoryPrefix+volUUID,
// Update UUID directory to store image name
err = SetOMapKeyValue(ctx, monitors, cr, imagePool, cj.namespace, cj.cephUUIDDirectoryPrefix+volUUID,
cj.csiImageKey, imageName)
if err != nil {
return "", "", err
}
// Update UUID directory to store encryption values
if kmsConf != "" {
err = SetOMapKeyValue(ctx, monitors, cr, pool, cj.namespace, cj.cephUUIDDirectoryPrefix+volUUID,
err = SetOMapKeyValue(ctx, monitors, cr, imagePool, cj.namespace, cj.cephUUIDDirectoryPrefix+volUUID,
cj.encryptKMSKey, kmsConf)
if err != nil {
return "", "", err
}
}
if journalPool != imagePool && journalPoolID != InvalidPoolID {
buf64 := make([]byte, 8)
binary.BigEndian.PutUint64(buf64, uint64(journalPoolID))
journalPoolIDStr := hex.EncodeToString(buf64)
// Update UUID directory to store CSI journal pool name (prefer ID instead of name to be pool rename proof)
err = SetOMapKeyValue(ctx, monitors, cr, imagePool, cj.namespace, cj.cephUUIDDirectoryPrefix+volUUID,
cj.csiJournalPool, journalPoolIDStr)
if err != nil {
return "", "", err
}
}
if snapSource {
// Update UUID directory to store source volume UUID in case of snapshots
err = SetOMapKeyValue(ctx, monitors, cr, pool, cj.namespace, cj.cephUUIDDirectoryPrefix+volUUID,
err = SetOMapKeyValue(ctx, monitors, cr, imagePool, cj.namespace, cj.cephUUIDDirectoryPrefix+volUUID,
cj.cephSnapSourceKey, parentName)
if err != nil {
return "", "", err
@ -423,70 +539,89 @@ func (cj *CSIJournal) ReserveName(ctx context.Context, monitors string, cr *Cred
return volUUID, imageName, nil
}
/*
GetObjectUUIDData fetches all keys from a UUID directory
Return values:
- string: Contains the request name for the passed in UUID
- string: Contains the rbd image name for the passed in UUID
- string: Contains the parent image name for the passed in UUID, if it is a snapshot
- string: Contains encryption KMS, if it is an encrypted image
- error: non-nil in case of any errors
*/
func (cj *CSIJournal) GetObjectUUIDData(ctx context.Context, monitors string, cr *Credentials, pool, objectUUID string, snapSource bool) (string, string, string, string, error) {
var sourceName string
// ImageAttributes contains all CSI stored image attributes, typically as OMap keys
type ImageAttributes struct {
RequestName string // Contains the request name for the passed in UUID
SourceName string // Contains the parent image name for the passed in UUID, if it is a snapshot
ImageName string // Contains the image or subvolume name for the passed in UUID
KmsID string // Contains encryption KMS, if it is an encrypted image
JournalPoolID int64 // Pool ID of the CSI journal pool, stored in big endian format (on-disk data)
}
// GetImageAttributes fetches all keys and their values, from a UUID directory, returning ImageAttributes structure
func (cj *CSIJournal) GetImageAttributes(ctx context.Context, monitors string, cr *Credentials, pool, objectUUID string, snapSource bool) (*ImageAttributes, error) {
var (
err error
imageAttributes *ImageAttributes = &ImageAttributes{}
)
if snapSource && cj.cephSnapSourceKey == "" {
err := errors.New("invalid request, cephSnapSourceKey is nil")
return "", "", "", "", err
err = errors.New("invalid request, cephSnapSourceKey is nil")
return nil, err
}
// TODO: fetch all omap vals in one call, than make multiple listomapvals
requestName, err := GetOMapValue(ctx, monitors, cr, pool, cj.namespace,
imageAttributes.RequestName, err = GetOMapValue(ctx, monitors, cr, pool, cj.namespace,
cj.cephUUIDDirectoryPrefix+objectUUID, cj.csiNameKey)
if err != nil {
return "", "", "", "", err
return nil, err
}
// image key was added at some point, so not all volumes will have this key set
// when ceph-csi was upgraded
imageName, err := GetOMapValue(ctx, monitors, cr, pool, cj.namespace,
imageAttributes.ImageName, err = GetOMapValue(ctx, monitors, cr, pool, cj.namespace,
cj.cephUUIDDirectoryPrefix+objectUUID, cj.csiImageKey)
if err != nil {
// if the key was not found, assume the default key + UUID
// otherwise return error
switch err.(type) {
default:
return "", "", "", "", err
return nil, err
case ErrKeyNotFound, ErrPoolNotFound:
}
if snapSource {
imageName = defaultSnapshotNamingPrefix + objectUUID
imageAttributes.ImageName = defaultSnapshotNamingPrefix + objectUUID
} else {
imageName = defaultVolumeNamingPrefix + objectUUID
imageAttributes.ImageName = defaultVolumeNamingPrefix + objectUUID
}
}
encryptionKmsConfig := ""
encryptionKmsConfig, err = GetOMapValue(ctx, monitors, cr, pool, cj.namespace,
imageAttributes.KmsID, err = GetOMapValue(ctx, monitors, cr, pool, cj.namespace,
cj.cephUUIDDirectoryPrefix+objectUUID, cj.encryptKMSKey)
if err != nil {
// ErrKeyNotFound means no encryption KMS was used
switch err.(type) {
default:
return "", "", "", "", fmt.Errorf("OMapVal for %s/%s failed to get encryption KMS value: %s",
return nil, fmt.Errorf("OMapVal for %s/%s failed to get encryption KMS value: %s",
pool, cj.cephUUIDDirectoryPrefix+objectUUID, err)
case ErrKeyNotFound, ErrPoolNotFound:
}
}
journalPoolIDStr, err := GetOMapValue(ctx, monitors, cr, pool, cj.namespace,
cj.cephUUIDDirectoryPrefix+objectUUID, cj.csiJournalPool)
if err != nil {
if _, ok := err.(ErrKeyNotFound); !ok {
return nil, err
}
imageAttributes.JournalPoolID = InvalidPoolID
} else {
var buf64 []byte
buf64, err = hex.DecodeString(journalPoolIDStr)
if err != nil {
return nil, err
}
imageAttributes.JournalPoolID = int64(binary.BigEndian.Uint64(buf64))
}
if snapSource {
sourceName, err = GetOMapValue(ctx, monitors, cr, pool, cj.namespace,
imageAttributes.SourceName, err = GetOMapValue(ctx, monitors, cr, pool, cj.namespace,
cj.cephUUIDDirectoryPrefix+objectUUID, cj.cephSnapSourceKey)
if err != nil {
return "", "", "", "", err
return nil, err
}
}
return requestName, imageName, sourceName, encryptionKmsConfig, nil
return imageAttributes, nil
}