diff --git a/api/model.go b/api/model.go index 8808ac8142..6e61e048dd 100644 --- a/api/model.go +++ b/api/model.go @@ -365,6 +365,7 @@ type Node struct { Region string `json:"region"` Zone string `json:"zone"` InstanceManagerCPURequest int `json:"instanceManagerCPURequest"` + AutoEvicting bool `json:"autoEvicting"` } type DiskStatus struct { @@ -1871,6 +1872,7 @@ func toNodeResource(node *longhorn.Node, address string, apiContext *api.ApiCont Region: node.Status.Region, Zone: node.Status.Zone, InstanceManagerCPURequest: node.Spec.InstanceManagerCPURequest, + AutoEvicting: node.Status.AutoEvicting, } disks := map[string]DiskInfo{} diff --git a/client/generated_node.go b/client/generated_node.go index 7254b2a6e0..12bc002bbf 100644 --- a/client/generated_node.go +++ b/client/generated_node.go @@ -11,6 +11,8 @@ type Node struct { AllowScheduling bool `json:"allowScheduling,omitempty" yaml:"allow_scheduling,omitempty"` + AutoEvicting bool `json:"autoEvicting,omitempty" yaml:"auto_evicting,omitempty"` + Conditions map[string]interface{} `json:"conditions,omitempty" yaml:"conditions,omitempty"` Disks map[string]interface{} `json:"disks,omitempty" yaml:"disks,omitempty"` diff --git a/constant/events.go b/constant/events.go index de48a0b3ba..12dc2305e1 100644 --- a/constant/events.go +++ b/constant/events.go @@ -33,14 +33,18 @@ const ( EventReasonSucceededExpansion = "SucceededExpansion" EventReasonCanceledExpansion = "CanceledExpansion" - EventReasonAttached = "Attached" - EventReasonDetached = "Detached" - EventReasonHealthy = "Healthy" - EventReasonFaulted = "Faulted" - EventReasonDegraded = "Degraded" - EventReasonOrphaned = "Orphaned" - EventReasonUnknown = "Unknown" - EventReasonFailedEviction = "FailedEviction" + EventReasonAttached = "Attached" + EventReasonDetached = "Detached" + EventReasonHealthy = "Healthy" + EventReasonFaulted = "Faulted" + EventReasonDegraded = "Degraded" + EventReasonOrphaned = "Orphaned" + EventReasonUnknown = "Unknown" + + EventReasonEvictionAutomatic = "EvictionAutomatic" + EventReasonEvictionUserRequested = "EvictionUserRequested" + EventReasonEvictionCanceled = "EvictionCanceled" + EventReasonEvictionFailed = "EvictionFailed" EventReasonDetachedUnexpectedly = "DetachedUnexpectedly" EventReasonRemount = "Remount" diff --git a/controller/controller_test.go b/controller/controller_test.go index f1748452f9..f0fabffa0c 100644 --- a/controller/controller_test.go +++ b/controller/controller_test.go @@ -502,6 +502,7 @@ func newNode(name, namespace string, allowScheduling bool, status longhorn.Condi StorageReserved: 0, }, }, + Name: name, }, Status: longhorn.NodeStatus{ Conditions: []longhorn.Condition{ diff --git a/controller/instance_manager_controller.go b/controller/instance_manager_controller.go index ee6b4fe450..ff11cf003f 100644 --- a/controller/instance_manager_controller.go +++ b/controller/instance_manager_controller.go @@ -566,7 +566,7 @@ func (imc *InstanceManagerController) syncInstanceManagerPDB(im *longhorn.Instan return err } - imPDB, err := imc.ds.GetPDBRO(imc.getPDBName(im)) + imPDB, err := imc.ds.GetPDBRO(types.GetPDBName(im)) if err != nil && !datastore.ErrorIsNotFound(err) { return err } @@ -666,7 +666,7 @@ func (imc *InstanceManagerController) cleanUpPDBForNonExistingIM() error { if labelValue != types.LonghornLabelInstanceManager { continue } - if _, ok := ims[getIMNameFromPDBName(pdbName)]; ok { + if _, ok := ims[types.GetIMNameFromPDBName(pdbName)]; ok { continue } if err := imc.ds.DeletePDB(pdbName); err != nil { @@ -680,7 +680,7 @@ func (imc *InstanceManagerController) cleanUpPDBForNonExistingIM() error { } func (imc *InstanceManagerController) deleteInstanceManagerPDB(im *longhorn.InstanceManager) error { - name := imc.getPDBName(im) + name := types.GetPDBName(im) imc.logger.Infof("Deleting %v PDB", name) err := imc.ds.DeletePDB(name) if err != nil && !datastore.ErrorIsNotFound(err) { @@ -732,6 +732,11 @@ func (imc *InstanceManagerController) canDeleteInstanceManagerPDB(im *longhorn.I return false, err } + if nodeDrainingPolicy == string(types.NodeDrainPolicyBlockForEviction) && len(replicasOnCurrentNode) > 0 { + // We must wait for ALL replicas to be evicted before removing the PDB. + return false, nil + } + targetReplicas := []*longhorn.Replica{} if nodeDrainingPolicy == string(types.NodeDrainPolicyAllowIfReplicaIsStopped) { for _, replica := range replicasOnCurrentNode { @@ -743,62 +748,31 @@ func (imc *InstanceManagerController) canDeleteInstanceManagerPDB(im *longhorn.I targetReplicas = replicasOnCurrentNode } - // For each replica in the target replica list, - // find out whether there is a PDB protected healthy replica of the same - // volume on another schedulable node. + // For each replica in the target replica list, find out whether there is a PDB protected healthy replica of the + // same volume on another schedulable node. for _, replica := range targetReplicas { - vol, err := imc.ds.GetVolumeRO(replica.Spec.VolumeName) - if err != nil { - return false, err - } + hasPDBOnAnotherNode := false + isUnusedReplicaOnCurrentNode := false - replicas, err := imc.ds.ListVolumeReplicasRO(vol.Name) + pdbProtectedHealthyReplicas, err := imc.ds.ListVolumePDBProtectedHealthyReplicasRO(replica.Spec.VolumeName) if err != nil { return false, err } - - hasPDBOnAnotherNode := false - isUnusedReplicaOnCurrentNode := false - for _, r := range replicas { - hasOtherHealthyReplicas := r.Spec.HealthyAt != "" && r.Spec.FailedAt == "" && r.Spec.NodeID != im.Spec.NodeID - if hasOtherHealthyReplicas { - unschedulable, err := imc.ds.IsKubeNodeUnschedulable(r.Spec.NodeID) - if err != nil { - return false, err - } - if unschedulable { - continue - } - - var rIM *longhorn.InstanceManager - rIM, err = imc.getRunningReplicaInstanceManagerRO(r) - if err != nil { - return false, err - } - if rIM == nil { - continue - } - - pdb, err := imc.ds.GetPDBRO(imc.getPDBName(rIM)) - if err != nil && !datastore.ErrorIsNotFound(err) { - return false, err - } - if pdb != nil { - hasPDBOnAnotherNode = true - break - } - } - // If a replica has never been started, there is no data stored in this replica, and - // retaining it makes no sense for HA. - // Hence Longhorn doesn't need to block the PDB removal for the replica. - // This case typically happens on a newly created volume that hasn't been attached to any node. - // https://github.com/longhorn/longhorn/issues/2673 - isUnusedReplicaOnCurrentNode = r.Spec.HealthyAt == "" && r.Spec.FailedAt == "" && r.Spec.NodeID == im.Spec.NodeID - if isUnusedReplicaOnCurrentNode { + for _, pdbProtectedHealthyReplica := range pdbProtectedHealthyReplicas { + if pdbProtectedHealthyReplica.Spec.NodeID != im.Spec.NodeID { + hasPDBOnAnotherNode = true break } } + // If a replica has never been started, there is no data stored in this replica, and retaining it makes no sense + // for HA. Hence Longhorn doesn't need to block the PDB removal for the replica. This case typically happens on + // a newly created volume that hasn't been attached to any node. + // https://github.com/longhorn/longhorn/issues/2673 + isUnusedReplicaOnCurrentNode = replica.Spec.HealthyAt == "" && + replica.Spec.FailedAt == "" && + replica.Spec.NodeID == im.Spec.NodeID + if !hasPDBOnAnotherNode && !isUnusedReplicaOnCurrentNode { return false, nil } @@ -807,24 +781,6 @@ func (imc *InstanceManagerController) canDeleteInstanceManagerPDB(im *longhorn.I return true, nil } -func (imc *InstanceManagerController) getRunningReplicaInstanceManagerRO(r *longhorn.Replica) (im *longhorn.InstanceManager, err error) { - if r.Status.InstanceManagerName == "" { - im, err = imc.ds.GetInstanceManagerByInstanceRO(r) - if err != nil && !types.ErrorIsNotFound(err) { - return nil, err - } - } else { - im, err = imc.ds.GetInstanceManagerRO(r.Status.InstanceManagerName) - if err != nil && !apierrors.IsNotFound(err) { - return nil, err - } - } - if im == nil || im.Status.CurrentState != longhorn.InstanceManagerStateRunning { - return nil, nil - } - return im, nil -} - func (imc *InstanceManagerController) areAllVolumesDetachedFromNode(nodeName string) (bool, error) { detached, err := imc.areAllInstanceRemovedFromNodeByType(nodeName, longhorn.InstanceManagerTypeEngine) if err != nil { @@ -874,7 +830,7 @@ func (imc *InstanceManagerController) createInstanceManagerPDB(im *longhorn.Inst func (imc *InstanceManagerController) generateInstanceManagerPDBManifest(im *longhorn.InstanceManager) *policyv1.PodDisruptionBudget { return &policyv1.PodDisruptionBudget{ ObjectMeta: metav1.ObjectMeta{ - Name: imc.getPDBName(im), + Name: types.GetPDBName(im), Namespace: imc.namespace, }, Spec: policyv1.PodDisruptionBudgetSpec{ @@ -886,18 +842,6 @@ func (imc *InstanceManagerController) generateInstanceManagerPDBManifest(im *lon } } -func (imc *InstanceManagerController) getPDBName(im *longhorn.InstanceManager) string { - return getPDBNameFromIMName(im.Name) -} - -func getPDBNameFromIMName(imName string) string { - return imName -} - -func getIMNameFromPDBName(pdbName string) string { - return pdbName -} - func (imc *InstanceManagerController) enqueueInstanceManager(instanceManager interface{}) { key, err := controller.KeyFunc(instanceManager) if err != nil { diff --git a/controller/node_controller.go b/controller/node_controller.go index 45bc3005cd..45ee54eff2 100644 --- a/controller/node_controller.go +++ b/controller/node_controller.go @@ -181,7 +181,8 @@ func (nc *NodeController) isResponsibleForSetting(obj interface{}) bool { return types.SettingName(setting.Name) == types.SettingNameStorageMinimalAvailablePercentage || types.SettingName(setting.Name) == types.SettingNameBackingImageCleanupWaitInterval || - types.SettingName(setting.Name) == types.SettingNameOrphanAutoDeletion + types.SettingName(setting.Name) == types.SettingNameOrphanAutoDeletion || + types.SettingName(setting.Name) == types.SettingNameNodeDrainPolicy } func (nc *NodeController) isResponsibleForReplica(obj interface{}) bool { @@ -552,6 +553,10 @@ func (nc *NodeController) syncNode(key string) (err error) { return err } + if err = nc.syncReplicaEvictionRequested(node, kubeNode); err != nil { + return err + } + return nil } @@ -565,6 +570,16 @@ func (nc *NodeController) enqueueNode(obj interface{}) { nc.queue.Add(key) } +func (nc *NodeController) enqueueNodeRateLimited(obj interface{}) { + key, err := controller.KeyFunc(obj) + if err != nil { + utilruntime.HandleError(fmt.Errorf("couldn't get key for object %#v: %v", obj, err)) + return + } + + nc.queue.AddRateLimited(key) +} + func (nc *NodeController) enqueueSetting(obj interface{}) { nodes, err := nc.ds.ListNodesRO() if err != nil { @@ -1441,3 +1456,107 @@ func (nc *NodeController) createSnapshotMonitor() (mon monitor.Monitor, err erro return mon, nil } + +func (nc *NodeController) syncReplicaEvictionRequested(node *longhorn.Node, kubeNode *corev1.Node) error { + log := getLoggerForNode(nc.logger, node) + node.Status.AutoEvicting = false + nodeDrainPolicy, err := nc.ds.GetSettingValueExisted(types.SettingNameNodeDrainPolicy) + if err != nil { + return errors.Wrapf(err, "failed to get %v setting", types.SettingNameNodeDrainPolicy) + } + + type replicaToSync struct { + *longhorn.Replica + syncReason string + } + replicasToSync := []replicaToSync{} + + for diskName, diskSpec := range node.Spec.Disks { + diskStatus := node.Status.DiskStatus[diskName] + for replicaName := range diskStatus.ScheduledReplica { + replica, err := nc.ds.GetReplica(replicaName) + if err != nil { + return err + } + shouldEvictReplica, reason, err := nc.shouldEvictReplica(node, kubeNode, &diskSpec, replica, + nodeDrainPolicy) + if err != nil { + return err + } + if replica.Spec.EvictionRequested != shouldEvictReplica { + replica.Spec.EvictionRequested = shouldEvictReplica + replicasToSync = append(replicasToSync, replicaToSync{replica, reason}) + } + + if replica.Spec.EvictionRequested && !node.Spec.EvictionRequested && !diskSpec.EvictionRequested { + // We don't consider the node to be auto evicting if eviction was manually requested. + node.Status.AutoEvicting = true + } + } + } + + for _, replicaToSync := range replicasToSync { + replicaLog := log.WithField("replica", replicaToSync.Name).WithField("disk", replicaToSync.Spec.DiskID) + if replicaToSync.Spec.EvictionRequested { + replicaLog.Infof("Requesting replica eviction") + if _, err := nc.ds.UpdateReplica(replicaToSync.Replica); err != nil { + replicaLog.Warn("Failed to request replica eviction, will enqueue then resync node") + nc.enqueueNodeRateLimited(node) + continue + } + nc.eventRecorder.Eventf(replicaToSync.Replica, corev1.EventTypeNormal, replicaToSync.syncReason, "Requesting replica %v eviction from node %v and disk %v", replicaToSync.Name, node.Spec.Name, replicaToSync.Spec.DiskID) + } else { + replicaLog.Infof("Cancelling replica eviction") + if _, err := nc.ds.UpdateReplica(replicaToSync.Replica); err != nil { + replicaLog.Warn("Failed to cancel replica eviction, will enqueue then resync node") + nc.enqueueNodeRateLimited(node) + continue + } + nc.eventRecorder.Eventf(replicaToSync.Replica, corev1.EventTypeNormal, replicaToSync.syncReason, "Cancelling replica %v eviction from node %v and disk %v", replicaToSync.Name, node.Spec.Name, replicaToSync.Spec.DiskID) + } + } + + return nil +} + +func (nc *NodeController) shouldEvictReplica(node *longhorn.Node, kubeNode *corev1.Node, diskSpec *longhorn.DiskSpec, + replica *longhorn.Replica, nodeDrainPolicy string) (bool, string, error) { + // Replica eviction was cancelled on down or deleted nodes in previous implementations. It seems safest to continue + // this behavior unless we find a reason to change it. + if isDownOrDeleted, err := nc.ds.IsNodeDownOrDeleted(node.Spec.Name); err != nil { + return false, "", err + } else if isDownOrDeleted { + return false, longhorn.NodeConditionReasonKubernetesNodeNotReady, nil + } + + if node.Spec.EvictionRequested || diskSpec.EvictionRequested { + return true, constant.EventReasonEvictionUserRequested, nil + } + if !kubeNode.Spec.Unschedulable { + // Node drain policy only takes effect on cordoned nodes. + return false, constant.EventReasonEvictionCanceled, nil + } + if nodeDrainPolicy == string(types.NodeDrainPolicyBlockForEviction) { + return true, constant.EventReasonEvictionAutomatic, nil + } + if nodeDrainPolicy != string(types.NodeDrainPolicyBlockForEvictionIfContainsLastReplica) { + return false, constant.EventReasonEvictionCanceled, nil + } + + pdbProtectedHealthyReplicas, err := nc.ds.ListVolumePDBProtectedHealthyReplicasRO(replica.Spec.VolumeName) + if err != nil { + return false, "", err + } + hasPDBOnAnotherNode := false + for _, pdbProtectedHealthyReplica := range pdbProtectedHealthyReplicas { + if pdbProtectedHealthyReplica.Spec.NodeID != replica.Spec.NodeID { + hasPDBOnAnotherNode = true + break + } + } + if !hasPDBOnAnotherNode { + return true, constant.EventReasonEvictionAutomatic, nil + } + + return false, constant.EventReasonEvictionCanceled, nil +} diff --git a/controller/replica_controller.go b/controller/replica_controller.go index ebb05a8aed..a362cbc691 100644 --- a/controller/replica_controller.go +++ b/controller/replica_controller.go @@ -195,69 +195,6 @@ func getLoggerForReplica(logger logrus.FieldLogger, r *longhorn.Replica) *logrus ) } -// From replica to check Node.Spec.EvictionRequested of the node -// this replica first, then check Node.Spec.Disks.EvictionRequested -func (rc *ReplicaController) isEvictionRequested(replica *longhorn.Replica) bool { - // Return false if this replica has not been assigned to a node. - if replica.Spec.NodeID == "" { - return false - } - - log := getLoggerForReplica(rc.logger, replica) - - if isDownOrDeleted, err := rc.ds.IsNodeDownOrDeleted(replica.Spec.NodeID); err != nil { - log.WithError(err).Warn("Failed to check if node is down or deleted") - return false - } else if isDownOrDeleted { - return false - } - - node, err := rc.ds.GetNodeRO(replica.Spec.NodeID) - if err != nil { - log.WithError(err).Warn("Failed to get node information") - return false - } - - // Check if node has been request eviction. - if node.Spec.EvictionRequested { - return true - } - - // Check if disk has been request eviction. - for diskName, diskStatus := range node.Status.DiskStatus { - if diskStatus.DiskUUID != replica.Spec.DiskID { - continue - } - diskSpec, ok := node.Spec.Disks[diskName] - if !ok { - log.Warnf("Cannot continue handling replica eviction since there is no spec for disk name %v on node %v", diskName, node.Name) - return false - } - return diskSpec.EvictionRequested - } - - return false -} - -func (rc *ReplicaController) UpdateReplicaEvictionStatus(replica *longhorn.Replica) { - log := getLoggerForReplica(rc.logger, replica) - - // Check if eviction has been requested on this replica - if rc.isEvictionRequested(replica) && - !replica.Status.EvictionRequested { - replica.Status.EvictionRequested = true - log.Info("Replica has requested eviction") - } - - // Check if eviction has been cancelled on this replica - if !rc.isEvictionRequested(replica) && - replica.Status.EvictionRequested { - replica.Status.EvictionRequested = false - log.Info("Replica has cancelled eviction") - } - -} - func (rc *ReplicaController) syncReplica(key string) (err error) { defer func() { err = errors.Wrapf(err, "failed to sync replica for %v", key) @@ -346,8 +283,8 @@ func (rc *ReplicaController) syncReplica(key string) (err error) { } }() - // Update `Replica.Status.EvictionRequested` field - rc.UpdateReplicaEvictionStatus(replica) + // Deprecated and no longer used by Longhorn, but maybe someone's external tooling uses it? Remove in v1.7.0. + replica.Status.EvictionRequested = replica.Spec.EvictionRequested return rc.instanceHandler.ReconcileInstanceState(replica, &replica.Spec.InstanceSpec, &replica.Status.InstanceStatus) } diff --git a/controller/utils.go b/controller/utils.go index add79557c8..4ca8f47419 100644 --- a/controller/utils.go +++ b/controller/utils.go @@ -10,7 +10,7 @@ import ( func hasReplicaEvictionRequested(rs map[string]*longhorn.Replica) bool { for _, r := range rs { - if r.Status.EvictionRequested { + if r.Spec.EvictionRequested { return true } } diff --git a/controller/volume_controller.go b/controller/volume_controller.go index 2614154746..9fcf60fcf0 100644 --- a/controller/volume_controller.go +++ b/controller/volume_controller.go @@ -515,7 +515,7 @@ func (c *VolumeController) EvictReplicas(v *longhorn.Volume, hasNewReplica := false healthyNonEvictingCount := healthyCount for _, replica := range rs { - if replica.Status.EvictionRequested && + if replica.Spec.EvictionRequested && e.Status.ReplicaModeMap[replica.Name] == longhorn.ReplicaModeRW { healthyNonEvictingCount-- } @@ -529,7 +529,7 @@ func (c *VolumeController) EvictReplicas(v *longhorn.Volume, log.Info("Creating one more replica for eviction") if err := c.replenishReplicas(v, e, rs, ""); err != nil { c.eventRecorder.Eventf(v, corev1.EventTypeWarning, - constant.EventReasonFailedEviction, + constant.EventReasonEvictionFailed, "volume %v failed to create one more replica", v.Name) return errors.Wrap(err, "failed to create new replica for replica eviction") } @@ -1030,7 +1030,7 @@ func (c *VolumeController) cleanupEvictionRequestedReplicas(v *longhorn.Volume, if !datastore.IsAvailableHealthyReplica(r) { continue } - if !r.Status.EvictionRequested { + if !r.Spec.EvictionRequested { hasNonEvictingHealthyReplica = true break } @@ -1038,7 +1038,7 @@ func (c *VolumeController) cleanupEvictionRequestedReplicas(v *longhorn.Volume, } for _, r := range rs { - if !r.Status.EvictionRequested { + if !r.Spec.EvictionRequested { continue } if !hasNonEvictingHealthyReplica && r.Name == evictingHealthyReplica { @@ -1046,10 +1046,13 @@ func (c *VolumeController) cleanupEvictionRequestedReplicas(v *longhorn.Volume, continue } if err := c.deleteReplica(r, rs); err != nil { - c.eventRecorder.Eventf(v, corev1.EventTypeWarning, - constant.EventReasonFailedEviction, - "volume %v failed to evict replica %v", - v.Name, r.Name) + if !apierrors.IsConflict(err) && !apierrors.IsNotFound(err) { + c.eventRecorder.Eventf(v, corev1.EventTypeWarning, + constant.EventReasonEvictionFailed, + "volume %v failed to evict replica %v", + v.Name, r.Name) + return false, err + } return false, err } log.Infof("Evicted replica %v in disk %v of node %v ", r.Name, r.Spec.DiskID, r.Spec.NodeID) @@ -2484,7 +2487,7 @@ func (c *VolumeController) getReplenishReplicasCount(v *longhorn.Volume, rs map[ continue } // Skip the replica has been requested eviction. - if r.Spec.FailedAt == "" && (!r.Status.EvictionRequested) && r.Spec.Active { + if r.Spec.FailedAt == "" && (!r.Spec.EvictionRequested) && r.Spec.Active { usableCount++ } } diff --git a/datastore/longhorn.go b/datastore/longhorn.go index 8306862b9e..c1dd630fc5 100644 --- a/datastore/longhorn.go +++ b/datastore/longhorn.go @@ -1457,6 +1457,64 @@ func IsAvailableHealthyReplica(r *longhorn.Replica) bool { return true } +func (s *DataStore) ListVolumePDBProtectedHealthyReplicasRO(volumeName string) (map[string]*longhorn.Replica, error) { + pdbProtectedHealthyReplicas := map[string]*longhorn.Replica{} + replicas, err := s.ListVolumeReplicasRO(volumeName) + if err != nil { + return nil, err + } + + for _, replica := range replicas { + if replica.Spec.HealthyAt == "" || replica.Spec.FailedAt != "" { + continue + } + + unschedulable, err := s.IsKubeNodeUnschedulable(replica.Spec.NodeID) + if err != nil { + return map[string]*longhorn.Replica{}, err + } + if unschedulable { + continue + } + + instanceManager, err := s.getRunningReplicaInstanceManagerRO(replica) + if err != nil { + return map[string]*longhorn.Replica{}, err + } + if instanceManager == nil { + continue + } + + pdb, err := s.GetPDBRO(types.GetPDBName(instanceManager)) + if err != nil && !ErrorIsNotFound(err) { + return map[string]*longhorn.Replica{}, err + } + if pdb != nil { + pdbProtectedHealthyReplicas[replica.Name] = replica + } + } + + return pdbProtectedHealthyReplicas, nil +} + +func (s *DataStore) getRunningReplicaInstanceManagerRO(r *longhorn.Replica) (im *longhorn.InstanceManager, err error) { + if r.Status.InstanceManagerName == "" { + im, err = s.GetInstanceManagerByInstanceRO(r) + if err != nil && !types.ErrorIsNotFound(err) { + return nil, err + } + } else { + im, err = s.GetInstanceManagerRO(r.Status.InstanceManagerName) + if err != nil && !ErrorIsNotFound(err) { + return nil, err + } + } + if im == nil || im.Status.CurrentState != longhorn.InstanceManagerStateRunning { + return nil, nil + } + return im, nil +} + // IsReplicaRebuildingFailed returns true if the rebuilding replica failed not caused by network issues. func IsReplicaRebuildingFailed(reusableFailedReplica *longhorn.Replica) bool { replicaRebuildFailedCondition := types.GetCondition(reusableFailedReplica.Status.Conditions, longhorn.ReplicaConditionTypeRebuildFailed) diff --git a/k8s/crds.yaml b/k8s/crds.yaml index 5cfb966b81..af8e9e91cc 100644 --- a/k8s/crds.yaml +++ b/k8s/crds.yaml @@ -1944,6 +1944,8 @@ spec: status: description: NodeStatus defines the observed state of the Longhorn node properties: + autoEvicting: + type: boolean conditions: items: properties: @@ -2434,6 +2436,8 @@ spec: type: string engineName: type: string + evictionRequested: + type: boolean failedAt: type: string hardNodeAffinity: @@ -2492,6 +2496,7 @@ spec: currentState: type: string evictionRequested: + description: 'Deprecated: Replaced by field `spec.evictionRequested`.' type: boolean instanceManagerName: type: string diff --git a/k8s/pkg/apis/longhorn/v1beta2/node.go b/k8s/pkg/apis/longhorn/v1beta2/node.go index 4f1fe6a1bb..ae23fde816 100644 --- a/k8s/pkg/apis/longhorn/v1beta2/node.go +++ b/k8s/pkg/apis/longhorn/v1beta2/node.go @@ -122,6 +122,8 @@ type NodeStatus struct { Zone string `json:"zone"` // +optional SnapshotCheckStatus SnapshotCheckStatus `json:"snapshotCheckStatus"` + // +optional + AutoEvicting bool `json:"autoEvicting"` } // +genclient diff --git a/k8s/pkg/apis/longhorn/v1beta2/replica.go b/k8s/pkg/apis/longhorn/v1beta2/replica.go index ceab3a0628..e8ed37533b 100644 --- a/k8s/pkg/apis/longhorn/v1beta2/replica.go +++ b/k8s/pkg/apis/longhorn/v1beta2/replica.go @@ -46,11 +46,14 @@ type ReplicaSpec struct { UnmapMarkDiskChainRemovedEnabled bool `json:"unmapMarkDiskChainRemovedEnabled"` // +optional RebuildRetryCount int `json:"rebuildRetryCount"` + // +optional + EvictionRequested bool `json:"evictionRequested"` } // ReplicaStatus defines the observed state of the Longhorn replica type ReplicaStatus struct { InstanceStatus `json:""` + // Deprecated: Replaced by field `spec.evictionRequested`. // +optional EvictionRequested bool `json:"evictionRequested"` } diff --git a/manager/volume.go b/manager/volume.go index 823b0011c0..a8f9fdcff1 100644 --- a/manager/volume.go +++ b/manager/volume.go @@ -718,7 +718,7 @@ func (m *VolumeManager) DeleteReplica(volumeName, replicaName string) error { if !datastore.IsAvailableHealthyReplica(r) { continue } - if r.Status.EvictionRequested { + if r.Spec.EvictionRequested { continue } healthyReplica = r.Name diff --git a/scheduler/replica_scheduler.go b/scheduler/replica_scheduler.go index 6360e68866..eeb7d6013a 100644 --- a/scheduler/replica_scheduler.go +++ b/scheduler/replica_scheduler.go @@ -134,7 +134,7 @@ func (rcs *ReplicaScheduler) getNodeCandidates(nodesInfo map[string]*longhorn.No func getNodesWithEvictingReplicas(replicas map[string]*longhorn.Replica, nodeInfo map[string]*longhorn.Node) map[string]*longhorn.Node { nodesWithEvictingReplicas := map[string]*longhorn.Node{} for _, r := range replicas { - if r.Status.EvictionRequested { + if r.Spec.EvictionRequested { if node, ok := nodeInfo[r.Spec.NodeID]; ok { nodesWithEvictingReplicas[r.Spec.NodeID] = node } @@ -584,7 +584,7 @@ func (rcs *ReplicaScheduler) isFailedReplicaReusable(r *longhorn.Replica, v *lon if r.Spec.RebuildRetryCount >= FailedReplicaMaxRetryCount { return false, nil } - if r.Status.EvictionRequested { + if r.Spec.EvictionRequested { return false, nil } if hardNodeAffinity != "" && r.Spec.NodeID != hardNodeAffinity { @@ -668,7 +668,7 @@ func IsPotentiallyReusableReplica(r *longhorn.Replica, hardNodeAffinity string) if r.Spec.RebuildRetryCount >= FailedReplicaMaxRetryCount { return false } - if r.Status.EvictionRequested { + if r.Spec.EvictionRequested { return false } if hardNodeAffinity != "" && r.Spec.NodeID != hardNodeAffinity { @@ -802,7 +802,7 @@ func getCurrentNodesAndZones(replicas map[string]*longhorn.Replica, nodeInfo map for _, r := range replicas { if r.Spec.NodeID != "" && r.DeletionTimestamp == nil && r.Spec.FailedAt == "" { if node, ok := nodeInfo[r.Spec.NodeID]; ok { - if r.Status.EvictionRequested { + if r.Spec.EvictionRequested { if _, ok := usedNodes[r.Spec.NodeID]; !ok { // This is an evicting replica on a thus far unused node. We won't change this again unless we // find a non-evicting replica on this node. diff --git a/scheduler/replica_scheduler_test.go b/scheduler/replica_scheduler_test.go index dd33b7eca5..670ebe3b83 100644 --- a/scheduler/replica_scheduler_test.go +++ b/scheduler/replica_scheduler_test.go @@ -942,7 +942,7 @@ func (s *TestSuite) TestReplicaScheduler(c *C) { } alreadyScheduledReplica := newReplicaForVolume(tc.volume) alreadyScheduledReplica.Spec.NodeID = TestNode1 - alreadyScheduledReplica.Status.EvictionRequested = true + alreadyScheduledReplica.Spec.EvictionRequested = true tc.allReplicas[alreadyScheduledReplica.Name] = alreadyScheduledReplica node1.Status.DiskStatus = map[string]*longhorn.DiskStatus{ getDiskID(TestNode1, "1"): { @@ -1024,7 +1024,7 @@ func (s *TestSuite) TestReplicaScheduler(c *C) { } alreadyScheduledReplica = newReplicaForVolume(tc.volume) alreadyScheduledReplica.Spec.NodeID = TestNode1 - alreadyScheduledReplica.Status.EvictionRequested = true + alreadyScheduledReplica.Spec.EvictionRequested = true tc.allReplicas[alreadyScheduledReplica.Name] = alreadyScheduledReplica node1.Status.DiskStatus = map[string]*longhorn.DiskStatus{ getDiskID(TestNode1, "1"): { @@ -1047,7 +1047,7 @@ func (s *TestSuite) TestReplicaScheduler(c *C) { } alreadyScheduledReplica = newReplicaForVolume(tc.volume) alreadyScheduledReplica.Spec.NodeID = TestNode2 - alreadyScheduledReplica.Status.EvictionRequested = false + alreadyScheduledReplica.Spec.EvictionRequested = false tc.allReplicas[alreadyScheduledReplica.Name] = alreadyScheduledReplica node2.Status.DiskStatus = map[string]*longhorn.DiskStatus{ getDiskID(TestNode2, "1"): { @@ -1070,7 +1070,7 @@ func (s *TestSuite) TestReplicaScheduler(c *C) { } alreadyScheduledReplica = newReplicaForVolume(tc.volume) alreadyScheduledReplica.Spec.NodeID = TestNode3 - alreadyScheduledReplica.Status.EvictionRequested = false + alreadyScheduledReplica.Spec.EvictionRequested = false tc.allReplicas[alreadyScheduledReplica.Name] = alreadyScheduledReplica node3.Status.DiskStatus = map[string]*longhorn.DiskStatus{ getDiskID(TestNode3, "1"): { @@ -1360,8 +1360,6 @@ func (s *TestSuite) TestGetCurrentNodesAndZones(c *C) { InstanceSpec: longhorn.InstanceSpec{ NodeID: nodeName, }, - }, - Status: longhorn.ReplicaStatus{ EvictionRequested: evictionRequested, }, } diff --git a/types/setting.go b/types/setting.go index ebf38f8ff8..7e295d63a5 100644 --- a/types/setting.go +++ b/types/setting.go @@ -693,7 +693,9 @@ var ( SettingDefinitionNodeDrainPolicy = SettingDefinition{ DisplayName: "Node Drain Policy", - Description: "Define the policy to use when a node with the last healthy replica of a volume is drained. \n" + + Description: "Define the policy to use when a node with the last healthy replica of a volume is drained.\n" + + "- **block-for-eviction** Longhorn will automatically evict all replicas and block the drain until eviction is complete.\n" + + "- **block-for-eviction-if-contains-last-replica** Longhorn will automatically evict any replicas that don't have a healthy counterpart and block the drain until eviction is complete.\n" + "- **block-if-contains-last-replica** Longhorn will block the drain when the node contains the last healthy replica of a volume.\n" + "- **allow-if-replica-is-stopped** Longhorn will allow the drain when the node contains the last healthy replica of a volume but the replica is stopped. WARNING: possible data loss if the node is removed after draining. Select this option if you want to drain the node and do in-place upgrade/maintenance.\n" + "- **always-allow** Longhorn will allow the drain even though the node contains the last healthy replica of a volume. WARNING: possible data loss if the node is removed after draining. Also possible data corruption if the last replica was running during the draining.\n", @@ -703,6 +705,8 @@ var ( ReadOnly: false, Default: string(NodeDrainPolicyBlockIfContainsLastReplica), Choices: []string{ + string(NodeDrainPolicyBlockForEviction), + string(NodeDrainPolicyBlockForEvictionIfContainsLastReplica), string(NodeDrainPolicyBlockIfContainsLastReplica), string(NodeDrainPolicyAllowIfReplicaIsStopped), string(NodeDrainPolicyAlwaysAllow), @@ -1155,12 +1159,14 @@ const ( NodeDownPodDeletionPolicyDeleteBothStatefulsetAndDeploymentPod = NodeDownPodDeletionPolicy("delete-both-statefulset-and-deployment-pod") ) -type NodeWithLastHealthyReplicaDrainPolicy string +type NodeDrainPolicy string const ( - NodeDrainPolicyBlockIfContainsLastReplica = NodeWithLastHealthyReplicaDrainPolicy("block-if-contains-last-replica") - NodeDrainPolicyAllowIfReplicaIsStopped = NodeWithLastHealthyReplicaDrainPolicy("allow-if-replica-is-stopped") - NodeDrainPolicyAlwaysAllow = NodeWithLastHealthyReplicaDrainPolicy("always-allow") + NodeDrainPolicyBlockForEviction = NodeDrainPolicy("block-for-eviction") + NodeDrainPolicyBlockForEvictionIfContainsLastReplica = NodeDrainPolicy("block-for-eviction-if-contains-last-replica") + NodeDrainPolicyBlockIfContainsLastReplica = NodeDrainPolicy("block-if-contains-last-replica") + NodeDrainPolicyAllowIfReplicaIsStopped = NodeDrainPolicy("allow-if-replica-is-stopped") + NodeDrainPolicyAlwaysAllow = NodeDrainPolicy("always-allow") ) type SystemManagedPodsImagePullPolicy string diff --git a/types/types.go b/types/types.go index 891e21e591..1719bbb63c 100644 --- a/types/types.go +++ b/types/types.go @@ -1080,3 +1080,15 @@ func GetBackupTargetSchemeFromURL(backupTargetURL string) string { return ValueUnknown } } + +func GetPDBName(im *longhorn.InstanceManager) string { + return GetPDBNameFromIMName(im.Name) +} + +func GetPDBNameFromIMName(imName string) string { + return imName +} + +func GetIMNameFromPDBName(pdbName string) string { + return pdbName +} diff --git a/upgrade/v15xto160/upgrade.go b/upgrade/v15xto160/upgrade.go index 5c5ce25d9a..90145df477 100644 --- a/upgrade/v15xto160/upgrade.go +++ b/upgrade/v15xto160/upgrade.go @@ -111,6 +111,8 @@ func upgradeReplicas(namespace string, lhClient *lhclientset.Clientset, resource r.Spec.Image = r.Spec.EngineImage r.Spec.EngineImage = "" } + + r.Spec.EvictionRequested = r.Status.EvictionRequested } return nil