diff --git a/api/v1beta1/flinkcluster_default_test.go b/api/v1beta1/flinkcluster_default_test.go index 938affd4..140866aa 100644 --- a/api/v1beta1/flinkcluster_default_test.go +++ b/api/v1beta1/flinkcluster_default_test.go @@ -45,6 +45,7 @@ func TestSetDefault(t *testing.T) { var defaultJmBlobPort = int32(6124) var defaultJmQueryPort = int32(6125) var defaultJmUIPort = int32(8081) + var defaultJmIngressTLSUse = false var defaultTmDataPort = int32(6121) var defaultTmRPCPort = int32(6122) var defaultTmQueryPort = int32(6125) @@ -52,11 +53,9 @@ func TestSetDefault(t *testing.T) { var defaultJobParallelism = int32(1) var defaultJobNoLoggingToStdout = false var defaultJobRestartPolicy = JobRestartPolicyNever - var defatulJobManagerIngressTLSUse = false var defaultMemoryOffHeapRatio = int32(25) var defaultMemoryOffHeapMin = resource.MustParse("600M") - defaultRecreateOnUpdate := new(bool) - *defaultRecreateOnUpdate = true + var defaultRecreateOnUpdate = true var expectedCluster = FlinkCluster{ TypeMeta: metav1.TypeMeta{}, ObjectMeta: metav1.ObjectMeta{}, @@ -70,7 +69,7 @@ func TestSetDefault(t *testing.T) { Replicas: &defaultJmReplicas, AccessScope: "Cluster", Ingress: &JobManagerIngressSpec{ - UseTLS: &defatulJobManagerIngressTLSUse, + UseTLS: &defaultJmIngressTLSUse, }, Ports: JobManagerPorts{ RPC: &defaultJmRPCPort, @@ -115,7 +114,7 @@ func TestSetDefault(t *testing.T) { MountPath: "/etc/hadoop/conf", }, EnvVars: nil, - RecreateOnUpdate: defaultRecreateOnUpdate, + RecreateOnUpdate: &defaultRecreateOnUpdate, }, Status: FlinkClusterStatus{}, } @@ -134,6 +133,7 @@ func TestSetNonDefault(t *testing.T) { var jmBlobPort = int32(8124) var jmQueryPort = int32(8125) var jmUIPort = int32(9081) + var jmIngressTLSUse = true var tmDataPort = int32(8121) var tmRPCPort = int32(8122) var tmQueryPort = int32(8125) @@ -141,16 +141,14 @@ func TestSetNonDefault(t *testing.T) { var jobParallelism = int32(2) var jobNoLoggingToStdout = true var jobRestartPolicy = JobRestartPolicyFromSavepointOnFailure - var jobManagerIngressTLSUse = true var memoryOffHeapRatio = int32(50) var memoryOffHeapMin = resource.MustParse("600M") + var recreateOnUpdate = false var securityContextUserGroup = int64(9999) var securityContext = corev1.PodSecurityContext{ RunAsUser: &securityContextUserGroup, RunAsGroup: &securityContextUserGroup, } - defaultRecreateOnUpdate := new(bool) - *defaultRecreateOnUpdate = true var cluster = FlinkCluster{ TypeMeta: metav1.TypeMeta{}, ObjectMeta: metav1.ObjectMeta{}, @@ -164,7 +162,7 @@ func TestSetNonDefault(t *testing.T) { Replicas: &jmReplicas, AccessScope: "Cluster", Ingress: &JobManagerIngressSpec{ - UseTLS: &jobManagerIngressTLSUse, + UseTLS: &jmIngressTLSUse, }, Ports: JobManagerPorts{ RPC: &jmRPCPort, @@ -208,7 +206,8 @@ func TestSetNonDefault(t *testing.T) { HadoopConfig: &HadoopConfig{ MountPath: "/opt/flink/hadoop/conf", }, - EnvVars: nil, + EnvVars: nil, + RecreateOnUpdate: &recreateOnUpdate, }, Status: FlinkClusterStatus{}, } @@ -228,7 +227,7 @@ func TestSetNonDefault(t *testing.T) { Replicas: &jmReplicas, AccessScope: "Cluster", Ingress: &JobManagerIngressSpec{ - UseTLS: &jobManagerIngressTLSUse, + UseTLS: &jmIngressTLSUse, }, Ports: JobManagerPorts{ RPC: &jmRPCPort, @@ -273,7 +272,7 @@ func TestSetNonDefault(t *testing.T) { MountPath: "/opt/flink/hadoop/conf", }, EnvVars: nil, - RecreateOnUpdate: defaultRecreateOnUpdate, + RecreateOnUpdate: &recreateOnUpdate, }, Status: FlinkClusterStatus{}, } diff --git a/api/v1beta1/flinkcluster_types.go b/api/v1beta1/flinkcluster_types.go index 14c0b441..b7b33f29 100644 --- a/api/v1beta1/flinkcluster_types.go +++ b/api/v1beta1/flinkcluster_types.go @@ -43,15 +43,17 @@ const ( // JobState defines states for a Flink job deployment. const ( - JobStatePending = "Pending" - JobStateRunning = "Running" - JobStateUpdating = "Updating" - JobStateSucceeded = "Succeeded" - JobStateFailed = "Failed" - JobStateCancelled = "Cancelled" - JobStateSuspended = "Suspended" - JobStateUnknown = "Unknown" - JobStateLost = "Lost" + JobStatePending = "Pending" + JobStateUpdating = "Updating" + JobStateRestarting = "Restarting" + JobStateDeploying = "Deploying" + JobStateDeployFailed = "DeployFailed" + JobStateRunning = "Running" + JobStateSucceeded = "Succeeded" + JobStateCancelled = "Cancelled" + JobStateFailed = "Failed" + JobStateLost = "Lost" + JobStateUnknown = "Unknown" ) // AccessScope defines the access scope of JobManager service. @@ -85,24 +87,23 @@ const ( ControlNameJobCancel = "job-cancel" // control state - ControlStateProgressing = "Progressing" - ControlStateSucceeded = "Succeeded" - ControlStateFailed = "Failed" + ControlStateRequested = "Requested" + ControlStateInProgress = "InProgress" + ControlStateSucceeded = "Succeeded" + ControlStateFailed = "Failed" ) // Savepoint status const ( - SavepointStateNotTriggered = "NotTriggered" SavepointStateInProgress = "InProgress" SavepointStateTriggerFailed = "TriggerFailed" SavepointStateFailed = "Failed" SavepointStateSucceeded = "Succeeded" - SavepointTriggerReasonUserRequested = "user requested" - SavepointTriggerReasonScheduled = "scheduled" - SavepointTriggerReasonScheduledInitial = "scheduled initial" // The first triggered savepoint has slightly different flow - SavepointTriggerReasonJobCancel = "job cancel" - SavepointTriggerReasonUpdate = "update" + SavepointTriggerReasonUserRequested = "user requested" + SavepointTriggerReasonJobCancel = "job cancel" + SavepointTriggerReasonScheduled = "scheduled" + SavepointTriggerReasonUpdate = "update" ) // ImageSpec defines Flink image of JobManager and TaskManager containers. @@ -348,12 +349,20 @@ type JobSpec struct { // Allow non-restored state, default: false. AllowNonRestoredState *bool `json:"allowNonRestoredState,omitempty"` - // Should take savepoint before upgrading the job, default: false. - TakeSavepointOnUpgrade *bool `json:"takeSavepointOnUpgrade,omitempty"` - // Savepoints dir where to store savepoints of the job. SavepointsDir *string `json:"savepointsDir,omitempty"` + // Should take savepoint before updating job, default: true. + // If this is set as false, maxStateAgeToRestoreSeconds must be provided to limit the savepoint age to restore. + TakeSavepointOnUpdate *bool `json:"takeSavepointOnUpdate,omitempty"` + + // Maximum age of the savepoint that allowed to restore state.. + // This is applied to auto restart on failure, update from stopped state and update without taking savepoint. + // If nil, job can be restarted only when the latest savepoint is the final job state (created by "stop with savepoint") + // - that is, only when job can be resumed from the suspended state. + // +kubebuilder:validation:Minimum=0 + MaxStateAgeToRestoreSeconds *int32 `json:"maxStateAgeToRestoreSeconds,omitempty"` + // Automatically take a savepoint to the `savepointsDir` every n seconds. AutoSavepointSeconds *int32 `json:"autoSavepointSeconds,omitempty"` @@ -555,7 +564,7 @@ type JobStatus struct { // The ID of the Flink job. ID string `json:"id,omitempty"` - // The state of the Kubernetes job. + // The state of the Flink job deployment. State string `json:"state"` // The actual savepoint from which this job started. @@ -571,21 +580,26 @@ type JobStatus struct { // Savepoint location. SavepointLocation string `json:"savepointLocation,omitempty"` - // Last savepoint trigger ID. - LastSavepointTriggerID string `json:"lastSavepointTriggerID,omitempty"` + // Last successful savepoint completed timestamp. + SavepointTime string `json:"savepointTime,omitempty"` - // Last savepoint trigger time. This is updated to make sure multiple - // savepoints will not be taken simultaneously. - LastSavepointTriggerTime string `json:"lastSavepointTriggerTime,omitempty"` + // The savepoint recorded in savepointLocation is the final state of the job. + FinalSavepoint bool `json:"finalSavepoint,omitempty"` - // Last successful or failed savepoint operation timestamp. - LastSavepointTime string `json:"lastSavepointTime,omitempty"` + // The timestamp of the Flink job deployment that creating job submitter. + DeployTime string `json:"deployTime,omitempty"` + + // The Flink job started timestamp. + StartTime string `json:"startTime,omitempty"` + + // The Flink job ended timestamp. + EndTime string `json:"endTime,omitempty"` // The number of restarts. RestartCount int32 `json:"restartCount,omitempty"` } -// SavepointStatus defines the status of savepoint progress +// SavepointStatus is the status of savepoint progress. type SavepointStatus struct { // The ID of the Flink job. JobID string `json:"jobID,omitempty"` @@ -599,8 +613,8 @@ type SavepointStatus struct { // Savepoint triggered reason. TriggerReason string `json:"triggerReason,omitempty"` - // Savepoint requested time. - RequestTime string `json:"requestTime,omitempty"` + // Savepoint status update time. + UpdateTime string `json:"requestTime,omitempty"` // Savepoint state. State string `json:"state"` @@ -609,6 +623,27 @@ type SavepointStatus struct { Message string `json:"message,omitempty"` } +type RevisionStatus struct { + // When the controller creates new ControllerRevision, it generates hash string from the FlinkCluster spec + // which is to be stored in ControllerRevision and uses it to compose the ControllerRevision name. + // Then the controller updates nextRevision to the ControllerRevision name. + // When update process is completed, the controller updates currentRevision as nextRevision. + // currentRevision and nextRevision is composed like this: + // -- + // e.g., myflinkcluster-c464ff7-5 + + // CurrentRevision indicates the version of FlinkCluster. + CurrentRevision string `json:"currentRevision,omitempty"` + + // NextRevision indicates the version of FlinkCluster updating. + NextRevision string `json:"nextRevision,omitempty"` + + // collisionCount is the count of hash collisions for the FlinkCluster. The controller + // uses this field as a collision avoidance mechanism when it needs to create the name for the + // newest ControllerRevision. + CollisionCount *int32 `json:"collisionCount,omitempty"` +} + // JobManagerIngressStatus defines the status of a JobManager ingress. type JobManagerIngressStatus struct { // The name of the Kubernetes ingress resource. @@ -644,30 +679,14 @@ type FlinkClusterStatus struct { // The status of the components. Components FlinkClusterComponentsStatus `json:"components"` - // The status of control requested by user + // The status of control requested by user. Control *FlinkClusterControlStatus `json:"control,omitempty"` - // The status of savepoint progress + // The status of savepoint progress. Savepoint *SavepointStatus `json:"savepoint,omitempty"` - // When the controller creates new ControllerRevision, it generates hash string from the FlinkCluster spec - // which is to be stored in ControllerRevision and uses it to compose the ControllerRevision name. - // Then the controller updates nextRevision to the ControllerRevision name. - // When update process is completed, the controller updates currentRevision as nextRevision. - // currentRevision and nextRevision is composed like this: - // -- - // e.g., myflinkcluster-c464ff7-5 - - // CurrentRevision indicates the version of FlinkCluster. - CurrentRevision string `json:"currentRevision,omitempty"` - - // NextRevision indicates the version of FlinkCluster updating. - NextRevision string `json:"nextRevision,omitempty"` - - // collisionCount is the count of hash collisions for the FlinkCluster. The controller - // uses this field as a collision avoidance mechanism when it needs to create the name for the - // newest ControllerRevision. - CollisionCount *int32 `json:"collisionCount,omitempty"` + // The status of revision. + Revision RevisionStatus `json:"revision,omitempty"` // Last update timestamp for this status. LastUpdateTime string `json:"lastUpdateTime,omitempty"` diff --git a/api/v1beta1/flinkcluster_types_util.go b/api/v1beta1/flinkcluster_types_util.go new file mode 100644 index 00000000..880160f0 --- /dev/null +++ b/api/v1beta1/flinkcluster_types_util.go @@ -0,0 +1,147 @@ +package v1beta1 + +import ( + "fmt" + "strings" + "time" +) + +func (j *JobStatus) IsActive() bool { + return j != nil && + (j.State == JobStateRunning || j.State == JobStateDeploying) +} + +func (j *JobStatus) IsPending() bool { + return j != nil && + (j.State == JobStatePending || + j.State == JobStateUpdating || + j.State == JobStateRestarting) +} + +func (j *JobStatus) IsFailed() bool { + return j != nil && + (j.State == JobStateFailed || + j.State == JobStateLost || + j.State == JobStateDeployFailed) +} + +func (j *JobStatus) IsStopped() bool { + return j != nil && + (j.State == JobStateSucceeded || + j.State == JobStateCancelled || + j.State == JobStateFailed || + j.State == JobStateLost || + j.State == JobStateDeployFailed) +} + +func (j *JobStatus) IsTerminated(spec *JobSpec) bool { + return j.IsStopped() && !j.ShouldRestart(spec) +} + +// Check if the recorded savepoint is up to date compared to maxStateAgeToRestoreSeconds. +// If maxStateAgeToRestoreSeconds is not set, +// the savepoint is up-to-date only when the recorded savepoint is the final job state. +func (j *JobStatus) IsSavepointUpToDate(spec *JobSpec, compareTime time.Time) bool { + if j.FinalSavepoint { + return true + } + if compareTime.IsZero() || + spec.MaxStateAgeToRestoreSeconds == nil || + j.SavepointLocation == "" || + j.SavepointTime == "" { + return false + } + + var stateMaxAge = int(*spec.MaxStateAgeToRestoreSeconds) + if !hasTimeElapsed(j.SavepointTime, compareTime, stateMaxAge) { + return true + } + return false +} + +// shouldRestartJob returns true if the controller should restart failed job. +// The controller can restart the job only if there is a savepoint that is close to the end time of the job. +func (j *JobStatus) ShouldRestart(spec *JobSpec) bool { + if j == nil || !j.IsFailed() || spec == nil { + return false + } + var tc TimeConverter + var restartEnabled = spec.RestartPolicy != nil && *spec.RestartPolicy == JobRestartPolicyFromSavepointOnFailure + var jobEndTime = tc.FromString(j.EndTime) + return restartEnabled && j.IsSavepointUpToDate(spec, jobEndTime) +} + +// Return true if job is ready to proceed update. +func (j *JobStatus) UpdateReady(spec *JobSpec, observeTime time.Time) bool { + var takeSavepointOnUpdate = spec.TakeSavepointOnUpdate == nil || *spec.TakeSavepointOnUpdate + switch { + case j == nil: + fallthrough + case !isBlank(spec.FromSavepoint): + return true + case j.IsActive(): + // When job is active and takeSavepointOnUpdate is true, only after taking savepoint with final job state, + // proceed job update. + if takeSavepointOnUpdate { + if j.FinalSavepoint { + return true + } + } else if j.IsSavepointUpToDate(spec, observeTime) { + return true + } + case j.State == JobStateUpdating && !takeSavepointOnUpdate: + return true + default: + // In other cases, check if savepoint is up-to-date compared to job end time. + var tc = TimeConverter{} + var jobEndTime time.Time + if j.EndTime != "" { + jobEndTime = tc.FromString(j.EndTime) + } + if j.IsSavepointUpToDate(spec, jobEndTime) { + return true + } + } + return false +} + +func (s *SavepointStatus) IsFailed() bool { + return s != nil && (s.State == SavepointStateTriggerFailed || s.State == SavepointStateFailed) +} + +func (r *RevisionStatus) IsUpdateTriggered() bool { + return r.CurrentRevision != r.NextRevision +} + +// TimeConverter converts between time.Time and string. +type TimeConverter struct{} + +// FromString converts string to time.Time. +func (tc *TimeConverter) FromString(timeStr string) time.Time { + timestamp, err := time.Parse( + time.RFC3339, timeStr) + if err != nil { + panic(fmt.Sprintf("Failed to parse time string: %s", timeStr)) + } + return timestamp +} + +// ToString converts time.Time to string. +func (tc *TimeConverter) ToString(timestamp time.Time) string { + return timestamp.Format(time.RFC3339) +} + +// Check time has passed +func hasTimeElapsed(timeToCheckStr string, now time.Time, intervalSec int) bool { + tc := &TimeConverter{} + timeToCheck := tc.FromString(timeToCheckStr) + intervalPassedTime := timeToCheck.Add(time.Duration(int64(intervalSec) * int64(time.Second))) + if now.After(intervalPassedTime) { + return true + } + return false +} + +func isBlank(s *string) bool { + return s == nil || strings.TrimSpace(*s) == "" +} diff --git a/api/v1beta1/flinkcluster_types_util_test.go b/api/v1beta1/flinkcluster_types_util_test.go new file mode 100644 index 00000000..1052e5fa --- /dev/null +++ b/api/v1beta1/flinkcluster_types_util_test.go @@ -0,0 +1,154 @@ +/* +Copyright 2019 Google LLC. + +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 v1beta1 + +import ( + "gotest.tools/assert" + "testing" + "time" +) + +func TestIsSavepointUpToDate(t *testing.T) { + var tc = &TimeConverter{} + var savepointTime = time.Now() + var jobEndTime = savepointTime.Add(time.Second * 100) + var maxStateAgeToRestoreSeconds = int32(300) + + // When maxStateAgeToRestoreSeconds is not provided + var jobSpec = JobSpec{ + MaxStateAgeToRestoreSeconds: nil, + } + var jobStatus = JobStatus{ + SavepointTime: tc.ToString(savepointTime), + SavepointLocation: "gs://my-bucket/savepoint-123", + } + var update = jobStatus.IsSavepointUpToDate(&jobSpec, jobEndTime) + assert.Equal(t, update, false) + + // Old savepoint + savepointTime = time.Now() + jobEndTime = savepointTime.Add(time.Second * 500) + jobSpec = JobSpec{ + MaxStateAgeToRestoreSeconds: &maxStateAgeToRestoreSeconds, + } + jobStatus = JobStatus{ + SavepointTime: tc.ToString(savepointTime), + SavepointLocation: "gs://my-bucket/savepoint-123", + EndTime: tc.ToString(jobEndTime), + } + update = jobStatus.IsSavepointUpToDate(&jobSpec, jobEndTime) + assert.Equal(t, update, false) + + // Fails without savepointLocation + savepointTime = time.Now() + jobEndTime = savepointTime.Add(time.Second * 100) + jobSpec = JobSpec{ + MaxStateAgeToRestoreSeconds: &maxStateAgeToRestoreSeconds, + } + jobStatus = JobStatus{ + SavepointTime: tc.ToString(savepointTime), + EndTime: tc.ToString(jobEndTime), + } + update = jobStatus.IsSavepointUpToDate(&jobSpec, jobEndTime) + assert.Equal(t, update, false) + + // Up-to-date savepoint + jobEndTime = savepointTime.Add(time.Second * 100) + jobSpec = JobSpec{ + MaxStateAgeToRestoreSeconds: &maxStateAgeToRestoreSeconds, + } + jobStatus = JobStatus{ + SavepointTime: tc.ToString(savepointTime), + SavepointLocation: "gs://my-bucket/savepoint-123", + } + update = jobStatus.IsSavepointUpToDate(&jobSpec, jobEndTime) + assert.Equal(t, update, true) + + // A savepoint of the final job state. + jobSpec = JobSpec{ + MaxStateAgeToRestoreSeconds: &maxStateAgeToRestoreSeconds, + } + jobStatus = JobStatus{ + FinalSavepoint: true, + } + update = jobStatus.IsSavepointUpToDate(&jobSpec, time.Time{}) + assert.Equal(t, update, true) +} + +func TestShouldRestartJob(t *testing.T) { + var tc = &TimeConverter{} + var restartOnFailure = JobRestartPolicyFromSavepointOnFailure + var neverRestart = JobRestartPolicyNever + var maxStateAgeToRestoreSeconds = int32(300) // 5 min + + // Restart with savepoint up to date + var savepointTime = time.Now() + var endTime = savepointTime.Add(time.Second * 60) // savepointTime + 1 min + var jobSpec = JobSpec{ + RestartPolicy: &restartOnFailure, + MaxStateAgeToRestoreSeconds: &maxStateAgeToRestoreSeconds, + } + var jobStatus = JobStatus{ + State: JobStateFailed, + SavepointLocation: "gs://my-bucket/savepoint-123", + SavepointTime: tc.ToString(savepointTime), + EndTime: tc.ToString(endTime), + } + var restart = jobStatus.ShouldRestart(&jobSpec) + assert.Equal(t, restart, true) + + // Not restart without savepoint + jobSpec = JobSpec{ + RestartPolicy: &restartOnFailure, + MaxStateAgeToRestoreSeconds: &maxStateAgeToRestoreSeconds, + } + jobStatus = JobStatus{ + State: JobStateFailed, + EndTime: tc.ToString(endTime), + } + restart = jobStatus.ShouldRestart(&jobSpec) + assert.Equal(t, restart, false) + + // Not restart with restartPolicy Never + jobSpec = JobSpec{ + RestartPolicy: &neverRestart, + MaxStateAgeToRestoreSeconds: &maxStateAgeToRestoreSeconds, + } + jobStatus = JobStatus{ + State: JobStateFailed, + SavepointLocation: "gs://my-bucket/savepoint-123", + SavepointTime: tc.ToString(savepointTime), + EndTime: tc.ToString(endTime), + } + restart = jobStatus.ShouldRestart(&jobSpec) + assert.Equal(t, restart, false) + + // Not restart with old savepoint + endTime = savepointTime.Add(time.Second * 300) // savepointTime + 5 min + jobSpec = JobSpec{ + RestartPolicy: &neverRestart, + MaxStateAgeToRestoreSeconds: &maxStateAgeToRestoreSeconds, + } + jobStatus = JobStatus{ + State: JobStateFailed, + SavepointLocation: "gs://my-bucket/savepoint-123", + SavepointTime: tc.ToString(savepointTime), + EndTime: tc.ToString(endTime), + } + restart = jobStatus.ShouldRestart(&jobSpec) + assert.Equal(t, restart, false) +} diff --git a/api/v1beta1/flinkcluster_validate.go b/api/v1beta1/flinkcluster_validate.go index df52d8fa..d7072e58 100644 --- a/api/v1beta1/flinkcluster_validate.go +++ b/api/v1beta1/flinkcluster_validate.go @@ -20,7 +20,9 @@ import ( "encoding/json" "fmt" "reflect" + "strconv" "strings" + "time" "k8s.io/apimachinery/pkg/api/resource" @@ -82,6 +84,11 @@ func (v *Validator) ValidateUpdate(old *FlinkCluster, new *FlinkCluster) error { return err } + // Skip remaining validation if no changes in spec. + if reflect.DeepEqual(new.Spec, old.Spec) { + return nil + } + cancelRequested, err := v.checkCancelRequested(old, new) if err != nil { return err @@ -115,24 +122,24 @@ func (v *Validator) checkControlAnnotations(old *FlinkCluster, new *FlinkCluster oldUserControl, _ := old.Annotations[ControlAnnotation] newUserControl, ok := new.Annotations[ControlAnnotation] if ok { - if oldUserControl != newUserControl && old.Status.Control != nil && old.Status.Control.State == ControlStateProgressing { + if oldUserControl != newUserControl && old.Status.Control != nil && old.Status.Control.State == ControlStateInProgress { return fmt.Errorf(ControlChangeWarnMsg, ControlAnnotation) } switch newUserControl { case ControlNameJobCancel: - var jobStatus = old.Status.Components.Job + var job = old.Status.Components.Job if old.Spec.Job == nil { return fmt.Errorf(SessionClusterWarnMsg, ControlNameJobCancel, ControlAnnotation) - } else if jobStatus == nil || isJobTerminated(old.Spec.Job.RestartPolicy, jobStatus) { + } else if job == nil || job.IsTerminated(old.Spec.Job) { return fmt.Errorf(InvalidJobStateForJobCancelMsg, ControlAnnotation) } case ControlNameSavepoint: - var jobStatus = old.Status.Components.Job + var job = old.Status.Components.Job if old.Spec.Job == nil { return fmt.Errorf(SessionClusterWarnMsg, ControlNameSavepoint, ControlAnnotation) } else if old.Spec.Job.SavepointsDir == nil || *old.Spec.Job.SavepointsDir == "" { return fmt.Errorf(InvalidSavepointDirMsg, ControlAnnotation) - } else if jobStatus == nil || isJobStopped(old.Status.Components.Job) { + } else if job == nil || job.IsStopped() { return fmt.Errorf(InvalidJobStateForSavepointMsg, ControlAnnotation) } default: @@ -211,14 +218,39 @@ func (v *Validator) validateJobUpdate(old *FlinkCluster, new *FlinkCluster) erro case old.Spec.Job == nil && new.Spec.Job == nil: return nil case old.Spec.Job == nil || new.Spec.Job == nil: - oldJob, _ := json.Marshal(old.Spec.Job) - newJob, _ := json.Marshal(new.Spec.Job) - return fmt.Errorf("you cannot change cluster type between session cluster and job cluster, old spec.job: %q, new spec.job: %q", oldJob, newJob) + oldJobSpec, _ := json.Marshal(old.Spec.Job) + newJobSpec, _ := json.Marshal(new.Spec.Job) + return fmt.Errorf("you cannot change cluster type between session cluster and job cluster, old spec.job: %q, new spec.job: %q", oldJobSpec, newJobSpec) case old.Spec.Job.SavepointsDir == nil || *old.Spec.Job.SavepointsDir == "": return fmt.Errorf("updating job is not allowed when spec.job.savepointsDir was not provided") case old.Spec.Job.SavepointsDir != nil && *old.Spec.Job.SavepointsDir != "" && (new.Spec.Job.SavepointsDir == nil || *new.Spec.Job.SavepointsDir == ""): return fmt.Errorf("removing savepointsDir is not allowed") + case !isBlank(new.Spec.Job.FromSavepoint): + return nil + default: + // In the case of taking savepoint is skipped, check if the savepoint is up-to-date. + var oldJob = old.Status.Components.Job + var takeSavepointOnUpdate = new.Spec.Job.TakeSavepointOnUpdate == nil || *new.Spec.Job.TakeSavepointOnUpdate + var skipTakeSavepoint = !takeSavepointOnUpdate || oldJob.IsStopped() + var now = time.Now() + if skipTakeSavepoint && oldJob != nil && !oldJob.UpdateReady(new.Spec.Job, now) { + oldJobJson, _ := json.Marshal(oldJob) + var takeSP, maxStateAge string + if new.Spec.Job.TakeSavepointOnUpdate == nil { + takeSP = "nil" + } else { + takeSP = strconv.FormatBool(*new.Spec.Job.TakeSavepointOnUpdate) + } + if new.Spec.Job.MaxStateAgeToRestoreSeconds == nil { + maxStateAge = "nil" + } else { + maxStateAge = strconv.Itoa(int(*new.Spec.Job.MaxStateAgeToRestoreSeconds)) + } + return fmt.Errorf("cannot update spec: taking savepoint is skipped but no up-to-date savepoint, "+ + "spec.job.takeSavepointOnUpdate: %v, spec.job.maxStateAgeToRestoreSeconds: %v, job status: %q", + takeSP, maxStateAge, oldJobJson) + } } return nil } @@ -413,10 +445,18 @@ func (v *Validator) validateJob(jobSpec *JobSpec) error { switch *jobSpec.RestartPolicy { case JobRestartPolicyNever: case JobRestartPolicyFromSavepointOnFailure: + if jobSpec.MaxStateAgeToRestoreSeconds == nil { + return fmt.Errorf("maxStateAgeToRestoreSeconds must be specified when restartPolicy is set as FromSavepointOnFailure") + } default: return fmt.Errorf("invalid job restartPolicy: %v", *jobSpec.RestartPolicy) } + if jobSpec.TakeSavepointOnUpdate != nil && *jobSpec.TakeSavepointOnUpdate == false && + jobSpec.MaxStateAgeToRestoreSeconds == nil { + return fmt.Errorf("maxStateAgeToRestoreSeconds must be specified when takeSavepointOnUpdate is set as false") + } + if jobSpec.CleanupPolicy == nil { return fmt.Errorf("job cleanupPolicy is unspecified") } @@ -507,26 +547,3 @@ func (v *Validator) validateMemoryOffHeapMin( } return nil } - -// shouldRestartJob returns true if the controller should restart the failed -// job. -func shouldRestartJob( - restartPolicy *JobRestartPolicy, - jobStatus *JobStatus) bool { - return restartPolicy != nil && - *restartPolicy == JobRestartPolicyFromSavepointOnFailure && - jobStatus != nil && - jobStatus.State == JobStateFailed && - len(jobStatus.SavepointLocation) > 0 -} - -func isJobStopped(status *JobStatus) bool { - return status != nil && - (status.State == JobStateSucceeded || - status.State == JobStateFailed || - status.State == JobStateCancelled) -} - -func isJobTerminated(restartPolicy *JobRestartPolicy, jobStatus *JobStatus) bool { - return isJobStopped(jobStatus) && !shouldRestartJob(restartPolicy, jobStatus) -} diff --git a/api/v1beta1/flinkcluster_validate_test.go b/api/v1beta1/flinkcluster_validate_test.go index e1031a5a..9c2f5a7f 100644 --- a/api/v1beta1/flinkcluster_validate_test.go +++ b/api/v1beta1/flinkcluster_validate_test.go @@ -20,6 +20,7 @@ import ( "encoding/json" "fmt" "testing" + "time" "k8s.io/apimachinery/pkg/api/resource" @@ -28,6 +29,8 @@ import ( metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" ) +const MaxStateAgeToRestore = int32(60) + func TestValidateCreate(t *testing.T) { var jmReplicas int32 = 1 var rpcPort int32 = 8001 @@ -36,6 +39,7 @@ func TestValidateCreate(t *testing.T) { var uiPort int32 = 8004 var dataPort int32 = 8005 var parallelism int32 = 2 + var maxStateAgeToRestoreSeconds = int32(60) var restartPolicy = JobRestartPolicyFromSavepointOnFailure var memoryOffHeapRatio int32 = 25 var memoryOffHeapMin = resource.MustParse("600M") @@ -72,9 +76,10 @@ func TestValidateCreate(t *testing.T) { MemoryOffHeapMin: memoryOffHeapMin, }, Job: &JobSpec{ - JarFile: "gs://my-bucket/myjob.jar", - Parallelism: ¶llelism, - RestartPolicy: &restartPolicy, + JarFile: "gs://my-bucket/myjob.jar", + Parallelism: ¶llelism, + MaxStateAgeToRestoreSeconds: &maxStateAgeToRestoreSeconds, + RestartPolicy: &restartPolicy, CleanupPolicy: &CleanupPolicy{ AfterJobSucceeds: CleanupActionKeepCluster, AfterJobFails: CleanupActionDeleteTaskManager, @@ -353,6 +358,7 @@ func TestInvalidJobSpec(t *testing.T) { var queryPort int32 = 8003 var uiPort int32 = 8004 var dataPort int32 = 8005 + var maxStateAgeToRestoreSeconds int32 = 300 var restartPolicy = JobRestartPolicyFromSavepointOnFailure var invalidRestartPolicy JobRestartPolicy = "XXX" var validator = &Validator{} @@ -393,8 +399,9 @@ func TestInvalidJobSpec(t *testing.T) { MemoryOffHeapMin: memoryOffHeapMin, }, Job: &JobSpec{ - JarFile: "", - RestartPolicy: &restartPolicy, + JarFile: "", + RestartPolicy: &restartPolicy, + MaxStateAgeToRestoreSeconds: &maxStateAgeToRestoreSeconds, }, }, } @@ -435,8 +442,9 @@ func TestInvalidJobSpec(t *testing.T) { MemoryOffHeapMin: memoryOffHeapMin, }, Job: &JobSpec{ - JarFile: "gs://my-bucket/myjob.jar", - RestartPolicy: &restartPolicy, + JarFile: "gs://my-bucket/myjob.jar", + RestartPolicy: &restartPolicy, + MaxStateAgeToRestoreSeconds: &maxStateAgeToRestoreSeconds, }, }, } @@ -477,9 +485,10 @@ func TestInvalidJobSpec(t *testing.T) { MemoryOffHeapMin: memoryOffHeapMin, }, Job: &JobSpec{ - JarFile: "gs://my-bucket/myjob.jar", - Parallelism: ¶llelism, - RestartPolicy: &invalidRestartPolicy, + JarFile: "gs://my-bucket/myjob.jar", + Parallelism: ¶llelism, + RestartPolicy: &invalidRestartPolicy, + MaxStateAgeToRestoreSeconds: &maxStateAgeToRestoreSeconds, }, }, } @@ -520,9 +529,10 @@ func TestInvalidJobSpec(t *testing.T) { MemoryOffHeapMin: memoryOffHeapMin, }, Job: &JobSpec{ - JarFile: "gs://my-bucket/myjob.jar", - Parallelism: ¶llelism, - RestartPolicy: &restartPolicy, + JarFile: "gs://my-bucket/myjob.jar", + Parallelism: ¶llelism, + RestartPolicy: &restartPolicy, + MaxStateAgeToRestoreSeconds: &maxStateAgeToRestoreSeconds, CleanupPolicy: &CleanupPolicy{ AfterJobSucceeds: "XXX", AfterJobFails: CleanupActionDeleteCluster, @@ -618,47 +628,19 @@ func TestUpdateSavepointGeneration(t *testing.T) { func TestUpdateJob(t *testing.T) { var validator = &Validator{} - var parallelism int32 = 2 - var restartPolicy = JobRestartPolicyFromSavepointOnFailure - var savepointDir = "/savepoint_dir" + var tc = &TimeConverter{} + var maxStateAge = time.Duration(MaxStateAgeToRestore) - oldCluster := getSimpleFlinkCluster() - oldCluster.Spec.Job = &JobSpec{ - JarFile: "gs://my-bucket/myjob.jar", - Parallelism: ¶llelism, - RestartPolicy: &restartPolicy, - SavepointsDir: &savepointDir, - CleanupPolicy: &CleanupPolicy{ - AfterJobSucceeds: CleanupActionKeepCluster, - AfterJobFails: CleanupActionDeleteTaskManager, - }, - } - newCluster := getSimpleFlinkCluster() - newCluster.Spec.Job = &JobSpec{ - JarFile: "gs://my-bucket/myjob.jar", - Parallelism: ¶llelism, - RestartPolicy: &restartPolicy, - SavepointsDir: nil, - CleanupPolicy: &CleanupPolicy{ - AfterJobSucceeds: CleanupActionKeepCluster, - AfterJobFails: CleanupActionDeleteTaskManager, - }, - } + // cannot remove savepointsDir + var oldCluster = getSimpleFlinkCluster() + var newCluster = getSimpleFlinkCluster() + newCluster.Spec.Job.SavepointsDir = nil err := validator.ValidateUpdate(&oldCluster, &newCluster) expectedErr := "removing savepointsDir is not allowed" assert.Equal(t, err.Error(), expectedErr) + // cannot change cluster type oldCluster = getSimpleFlinkCluster() - oldCluster.Spec.Job = &JobSpec{ - JarFile: "gs://my-bucket/myjob.jar", - Parallelism: ¶llelism, - RestartPolicy: &restartPolicy, - SavepointsDir: &savepointDir, - CleanupPolicy: &CleanupPolicy{ - AfterJobSucceeds: CleanupActionKeepCluster, - AfterJobFails: CleanupActionDeleteTaskManager, - }, - } newCluster = getSimpleFlinkCluster() newCluster.Spec.Job = nil err = validator.ValidateUpdate(&oldCluster, &newCluster) @@ -667,29 +649,111 @@ func TestUpdateJob(t *testing.T) { expectedErr = fmt.Sprintf("you cannot change cluster type between session cluster and job cluster, old spec.job: %q, new spec.job: %q", oldJson, newJson) assert.Equal(t, err.Error(), expectedErr) + // cannot update when savepointDir is not provided oldCluster = getSimpleFlinkCluster() - oldCluster.Spec.Job = &JobSpec{ - JarFile: "gs://my-bucket/myjob-v1.jar", - Parallelism: ¶llelism, - RestartPolicy: &restartPolicy, - CleanupPolicy: &CleanupPolicy{ - AfterJobSucceeds: CleanupActionKeepCluster, - AfterJobFails: CleanupActionDeleteTaskManager, - }, + oldCluster.Spec.Job.SavepointsDir = nil + newCluster = getSimpleFlinkCluster() + newCluster.Spec.Job.SavepointsDir = nil + newCluster.Spec.Job.JarFile = "gs://my-bucket/myjob-v2.jar" + err = validator.ValidateUpdate(&oldCluster, &newCluster) + expectedErr = "updating job is not allowed when spec.job.savepointsDir was not provided" + assert.Equal(t, err.Error(), expectedErr) + + // cannot update when takeSavepointOnUpdate is false and stale savepoint + var takeSavepointOnUpdateFalse = false + var savepointTime = time.Now().Add(-(maxStateAge + 10) * time.Second) // stale savepoint + oldCluster = getSimpleFlinkCluster() + oldCluster.Status.Components.Job = &JobStatus{ + SavepointTime: tc.ToString(savepointTime), + SavepointLocation: "gs://my-bucket/my-sp-123", + State: JobStateRunning, } newCluster = getSimpleFlinkCluster() - newCluster.Spec.Job = &JobSpec{ - JarFile: "gs://my-bucket/myjob-v2.jar", - Parallelism: ¶llelism, - RestartPolicy: &restartPolicy, - CleanupPolicy: &CleanupPolicy{ - AfterJobSucceeds: CleanupActionKeepCluster, - AfterJobFails: CleanupActionDeleteTaskManager, - }, + newCluster.Spec.Job.JarFile = "gs://my-bucket/myjob-v2.jar" + newCluster.Spec.Job.TakeSavepointOnUpdate = &takeSavepointOnUpdateFalse + err = validator.ValidateUpdate(&oldCluster, &newCluster) + jobStatusJson, _ := json.Marshal(oldCluster.Status.Components.Job) + expectedErr = fmt.Sprintf("cannot update spec: taking savepoint is skipped but no up-to-date savepoint, "+ + "spec.job.takeSavepointOnUpdate: false, spec.job.maxStateAgeToRestoreSeconds: 60, job status: %q", jobStatusJson) + assert.Equal(t, err.Error(), expectedErr) + + // update when takeSavepointOnUpdate is false and savepoint is up-to-date + takeSavepointOnUpdateFalse = false + maxStateAge = time.Duration(*getSimpleFlinkCluster().Spec.Job.MaxStateAgeToRestoreSeconds) + savepointTime = time.Now().Add(-(maxStateAge - 10) * time.Second) // up-to-date savepoint + oldCluster = getSimpleFlinkCluster() + oldCluster.Status.Components.Job = &JobStatus{ + SavepointTime: tc.ToString(savepointTime), + SavepointLocation: "gs://my-bucket/my-sp-123", + State: JobStateRunning, } + newCluster = getSimpleFlinkCluster() + newCluster.Spec.Job.JarFile = "gs://my-bucket/myjob-v2.jar" + newCluster.Spec.Job.TakeSavepointOnUpdate = &takeSavepointOnUpdateFalse err = validator.ValidateUpdate(&oldCluster, &newCluster) - expectedErr = "updating job is not allowed when spec.job.savepointsDir was not provided" + assert.Equal(t, err, nil) + + // spec update is allowed when takeSavepointOnUpdate is true and savepoint is not completed yet + oldCluster = getSimpleFlinkCluster() + oldCluster.Status.Components.Job = &JobStatus{ + FinalSavepoint: false, + SavepointLocation: "gs://my-bucket/my-sp-123", + State: JobStateRunning, + } + newCluster = getSimpleFlinkCluster() + newCluster.Spec.Job.JarFile = "gs://my-bucket/myjob-v2.jar" + err = validator.ValidateUpdate(&oldCluster, &newCluster) + assert.Equal(t, err, nil) + + // when job is stopped and no up-to-date savepoint + var jobEndTime = time.Now() + savepointTime = jobEndTime.Add(-(maxStateAge + 10) * time.Second) // stale savepoint + oldCluster = getSimpleFlinkCluster() + oldCluster.Status.Components.Job = &JobStatus{ + SavepointTime: tc.ToString(savepointTime), + SavepointLocation: "gs://my-bucket/my-sp-123", + State: JobStateFailed, + EndTime: tc.ToString(jobEndTime), + } + newCluster = getSimpleFlinkCluster() + newCluster.Spec.Job.JarFile = "gs://my-bucket/myjob-v2.jar" + err = validator.ValidateUpdate(&oldCluster, &newCluster) + jobStatusJson, _ = json.Marshal(oldCluster.Status.Components.Job) + expectedErr = fmt.Sprintf("cannot update spec: taking savepoint is skipped but no up-to-date savepoint, "+ + "spec.job.takeSavepointOnUpdate: nil, spec.job.maxStateAgeToRestoreSeconds: 60, job status: %q", jobStatusJson) assert.Equal(t, err.Error(), expectedErr) + + // when job is stopped and savepoint is up-to-date + jobEndTime = time.Now() + savepointTime = jobEndTime.Add(-(maxStateAge - 10) * time.Second) // up-to-date savepoint + oldCluster = getSimpleFlinkCluster() + oldCluster.Status.Components.Job = &JobStatus{ + SavepointTime: tc.ToString(savepointTime), + SavepointLocation: "gs://my-bucket/my-sp-123", + State: JobStateFailed, + EndTime: tc.ToString(jobEndTime), + } + newCluster = getSimpleFlinkCluster() + newCluster.Spec.Job.JarFile = "gs://my-bucket/myjob-v2.jar" + err = validator.ValidateUpdate(&oldCluster, &newCluster) + assert.Equal(t, err, nil) + + // when job is stopped and savepoint is stale, but fromSavepoint is provided + var fromSavepoint = "gs://my-bucket/sp-123" + jobEndTime = time.Now() + savepointTime = jobEndTime.Add(-(maxStateAge + 10) * time.Second) // stale savepoint + oldCluster = getSimpleFlinkCluster() + oldCluster.Status.Components.Job = &JobStatus{ + SavepointTime: tc.ToString(savepointTime), + SavepointLocation: "gs://my-bucket/my-sp-123", + State: JobStateFailed, + EndTime: tc.ToString(jobEndTime), + } + newCluster = getSimpleFlinkCluster() + newCluster.Spec.Job.JarFile = "gs://my-bucket/myjob-v2.jar" + newCluster.Spec.Job.FromSavepoint = &fromSavepoint + err = validator.ValidateUpdate(&oldCluster, &newCluster) + assert.Equal(t, err, nil) } func TestUpdateCluster(t *testing.T) { @@ -813,7 +877,7 @@ func TestUserControlSavepoint(t *testing.T) { var oldCluster1 = FlinkCluster{ Spec: FlinkClusterSpec{Job: &JobSpec{}}, - Status: FlinkClusterStatus{Control: &FlinkClusterControlStatus{State: ControlStateProgressing}}, + Status: FlinkClusterStatus{Control: &FlinkClusterControlStatus{State: ControlStateInProgress}}, } var err1 = validator.ValidateUpdate(&oldCluster1, &newCluster) var expectedErr1 = "change is not allowed for control in progress, annotation: flinkclusters.flinkoperator.k8s.io/user-control" @@ -852,6 +916,7 @@ func TestUserControlSavepoint(t *testing.T) { } func TestUserControlJobCancel(t *testing.T) { + var tc = TimeConverter{} var validator = &Validator{} var restartPolicy = JobRestartPolicyNever var newCluster = FlinkCluster{ @@ -864,7 +929,7 @@ func TestUserControlJobCancel(t *testing.T) { var oldCluster1 = FlinkCluster{ Spec: FlinkClusterSpec{Job: &JobSpec{}}, - Status: FlinkClusterStatus{Control: &FlinkClusterControlStatus{State: ControlStateProgressing}}, + Status: FlinkClusterStatus{Control: &FlinkClusterControlStatus{State: ControlStateInProgress}}, } var err1 = validator.ValidateUpdate(&oldCluster1, &newCluster) var expectedErr1 = "change is not allowed for control in progress, annotation: flinkclusters.flinkoperator.k8s.io/user-control" @@ -881,16 +946,22 @@ func TestUserControlJobCancel(t *testing.T) { assert.Equal(t, err3.Error(), expectedErr3) var oldCluster4 = FlinkCluster{ - Spec: FlinkClusterSpec{Job: &JobSpec{}}, - Status: FlinkClusterStatus{Components: FlinkClusterComponentsStatus{Job: &JobStatus{State: JobStateSucceeded}}}, + Spec: FlinkClusterSpec{Job: &JobSpec{}}, + Status: FlinkClusterStatus{Components: FlinkClusterComponentsStatus{Job: &JobStatus{ + State: JobStateSucceeded, + EndTime: tc.ToString(time.Now()), + }}}, } var err4 = validator.ValidateUpdate(&oldCluster4, &newCluster) var expectedErr4 = "job-cancel is not allowed because job is not started yet or already terminated, annotation: flinkclusters.flinkoperator.k8s.io/user-control" assert.Equal(t, err4.Error(), expectedErr4) var oldCluster5 = FlinkCluster{ - Spec: FlinkClusterSpec{Job: &JobSpec{RestartPolicy: &restartPolicy}}, - Status: FlinkClusterStatus{Components: FlinkClusterComponentsStatus{Job: &JobStatus{State: JobStateFailed}}}, + Spec: FlinkClusterSpec{Job: &JobSpec{RestartPolicy: &restartPolicy}}, + Status: FlinkClusterStatus{Components: FlinkClusterComponentsStatus{ + Job: &JobStatus{State: JobStateFailed, + EndTime: tc.ToString(time.Now()), + }}}, } var err5 = validator.ValidateUpdate(&oldCluster5, &newCluster) var expectedErr5 = "job-cancel is not allowed because job is not started yet or already terminated, annotation: flinkclusters.flinkoperator.k8s.io/user-control" @@ -965,6 +1036,7 @@ func getSimpleFlinkCluster() FlinkCluster { var memoryOffHeapRatio int32 = 25 var memoryOffHeapMin = resource.MustParse("600M") var parallelism int32 = 2 + var maxStateAge = MaxStateAgeToRestore var restartPolicy = JobRestartPolicyFromSavepointOnFailure var savepointDir = "/savepoint_dir" return FlinkCluster{ @@ -1000,10 +1072,11 @@ func getSimpleFlinkCluster() FlinkCluster { MemoryOffHeapMin: memoryOffHeapMin, }, Job: &JobSpec{ - JarFile: "gs://my-bucket/myjob.jar", - Parallelism: ¶llelism, - RestartPolicy: &restartPolicy, - SavepointsDir: &savepointDir, + JarFile: "gs://my-bucket/myjob.jar", + Parallelism: ¶llelism, + MaxStateAgeToRestoreSeconds: &maxStateAge, + RestartPolicy: &restartPolicy, + SavepointsDir: &savepointDir, CleanupPolicy: &CleanupPolicy{ AfterJobSucceeds: CleanupActionKeepCluster, AfterJobFails: CleanupActionDeleteTaskManager, diff --git a/api/v1beta1/zz_generated.deepcopy.go b/api/v1beta1/zz_generated.deepcopy.go index dc36e1da..25b579e8 100644 --- a/api/v1beta1/zz_generated.deepcopy.go +++ b/api/v1beta1/zz_generated.deepcopy.go @@ -260,11 +260,7 @@ func (in *FlinkClusterStatus) DeepCopyInto(out *FlinkClusterStatus) { *out = new(SavepointStatus) **out = **in } - if in.CollisionCount != nil { - in, out := &in.CollisionCount, &out.CollisionCount - *out = new(int32) - **out = **in - } + in.Revision.DeepCopyInto(&out.Revision) } // DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new FlinkClusterStatus. @@ -494,6 +490,13 @@ func (in *JobManagerSpec) DeepCopyInto(out *JobManagerSpec) { (*in)[i].DeepCopyInto(&(*out)[i]) } } + if in.VolumeClaimTemplates != nil { + in, out := &in.VolumeClaimTemplates, &out.VolumeClaimTemplates + *out = make([]v1.PersistentVolumeClaim, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } if in.InitContainers != nil { in, out := &in.InitContainers, &out.InitContainers *out = make([]v1.Container, len(*in)) @@ -576,16 +579,21 @@ func (in *JobSpec) DeepCopyInto(out *JobSpec) { *out = new(bool) **out = **in } - if in.TakeSavepointOnUpgrade != nil { - in, out := &in.TakeSavepointOnUpgrade, &out.takeSavepointOnUpgrade - *out = new(bool) - **out = **in - } if in.SavepointsDir != nil { in, out := &in.SavepointsDir, &out.SavepointsDir *out = new(string) **out = **in } + if in.TakeSavepointOnUpdate != nil { + in, out := &in.TakeSavepointOnUpdate, &out.TakeSavepointOnUpdate + *out = new(bool) + **out = **in + } + if in.MaxStateAgeToRestoreSeconds != nil { + in, out := &in.MaxStateAgeToRestoreSeconds, &out.MaxStateAgeToRestoreSeconds + *out = new(int32) + **out = **in + } if in.AutoSavepointSeconds != nil { in, out := &in.AutoSavepointSeconds, &out.AutoSavepointSeconds *out = new(int32) @@ -699,6 +707,26 @@ func (in *NamedPort) DeepCopy() *NamedPort { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *RevisionStatus) DeepCopyInto(out *RevisionStatus) { + *out = *in + if in.CollisionCount != nil { + in, out := &in.CollisionCount, &out.CollisionCount + *out = new(int32) + **out = **in + } +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new RevisionStatus. +func (in *RevisionStatus) DeepCopy() *RevisionStatus { + if in == nil { + return nil + } + out := new(RevisionStatus) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *SavepointStatus) DeepCopyInto(out *SavepointStatus) { *out = *in @@ -774,6 +802,13 @@ func (in *TaskManagerSpec) DeepCopyInto(out *TaskManagerSpec) { (*in)[i].DeepCopyInto(&(*out)[i]) } } + if in.VolumeClaimTemplates != nil { + in, out := &in.VolumeClaimTemplates, &out.VolumeClaimTemplates + *out = make([]v1.PersistentVolumeClaim, len(*in)) + for i := range *in { + (*in)[i].DeepCopyInto(&(*out)[i]) + } + } if in.InitContainers != nil { in, out := &in.InitContainers, &out.InitContainers *out = make([]v1.Container, len(*in)) @@ -833,6 +868,21 @@ func (in *TaskManagerSpec) DeepCopy() *TaskManagerSpec { return out } +// DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. +func (in *TimeConverter) DeepCopyInto(out *TimeConverter) { + *out = *in +} + +// DeepCopy is an autogenerated deepcopy function, copying the receiver, creating a new TimeConverter. +func (in *TimeConverter) DeepCopy() *TimeConverter { + if in == nil { + return nil + } + out := new(TimeConverter) + in.DeepCopyInto(out) + return out +} + // DeepCopyInto is an autogenerated deepcopy function, copying the receiver, writing into out. in must be non-nil. func (in *Validator) DeepCopyInto(out *Validator) { *out = *in diff --git a/config/crd/bases/flinkoperator.k8s.io_flinkclusters.yaml b/config/crd/bases/flinkoperator.k8s.io_flinkclusters.yaml index 9a5e5031..f0474f3e 100644 --- a/config/crd/bases/flinkoperator.k8s.io_flinkclusters.yaml +++ b/config/crd/bases/flinkoperator.k8s.io_flinkclusters.yaml @@ -163,8 +163,6 @@ spec: type: integer cancelRequested: type: boolean - takeSavepointOnUpgrade: - type: boolean className: type: string cleanupPolicy: @@ -716,6 +714,10 @@ spec: type: array jarFile: type: string + maxStateAgeToRestoreSeconds: + format: int32 + minimum: 0 + type: integer noLoggingToStdout: type: boolean parallelism: @@ -808,6 +810,8 @@ spec: type: string type: object type: object + takeSavepointOnUpdate: + type: boolean volumeMounts: items: properties: @@ -2661,6 +2665,123 @@ spec: type: string type: object type: array + volumeClaimTemplates: + items: + properties: + apiVersion: + type: string + kind: + type: string + metadata: + type: object + spec: + properties: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + required: + - kind + - name + type: object + resources: + properties: + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + storageClassName: + type: string + volumeMode: + type: string + volumeName: + type: string + type: object + status: + properties: + accessModes: + items: + type: string + type: array + capacity: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + conditions: + items: + properties: + lastProbeTime: + format: date-time + type: string + lastTransitionTime: + format: date-time + type: string + message: + type: string + reason: + type: string + status: + type: string + type: + type: string + required: + - status + - type + type: object + type: array + phase: + type: string + type: object + type: object + type: array volumeMounts: items: properties: @@ -4506,6 +4627,123 @@ spec: type: string type: object type: array + volumeClaimTemplates: + items: + properties: + apiVersion: + type: string + kind: + type: string + metadata: + type: object + spec: + properties: + accessModes: + items: + type: string + type: array + dataSource: + properties: + apiGroup: + type: string + kind: + type: string + name: + type: string + required: + - kind + - name + type: object + resources: + properties: + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + selector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + required: + - key + - operator + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + storageClassName: + type: string + volumeMode: + type: string + volumeName: + type: string + type: object + status: + properties: + accessModes: + items: + type: string + type: array + capacity: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + conditions: + items: + properties: + lastProbeTime: + format: date-time + type: string + lastTransitionTime: + format: date-time + type: string + message: + type: string + reason: + type: string + status: + type: string + type: + type: string + required: + - status + - type + type: object + type: array + phase: + type: string + type: object + type: object + type: array volumeMounts: items: properties: @@ -5129,9 +5367,6 @@ spec: type: object status: properties: - collisionCount: - format: int32 - type: integer components: properties: configMap: @@ -5146,15 +5381,15 @@ spec: type: object job: properties: - fromSavepoint: - type: string - id: + deployTime: type: string - lastSavepointTriggerTime: + endTime: type: string - lastSavepointTime: + finalSavepoint: + type: boolean + fromSavepoint: type: string - lastSavepointTriggerID: + id: type: string name: type: string @@ -5166,19 +5401,13 @@ spec: type: integer savepointLocation: type: string - state: + savepointTime: type: string - required: - - state - type: object - jobManagerStatefulSet: - properties: - name: + startTime: type: string state: type: string required: - - name - state type: object jobManagerIngress: @@ -5208,6 +5437,16 @@ spec: - name - state type: object + jobManagerStatefulSet: + properties: + name: + type: string + state: + type: string + required: + - name + - state + type: object taskManagerStatefulSet: properties: name: @@ -5220,8 +5459,8 @@ spec: type: object required: - configMap - - jobManagerStatefulSet - jobManagerService + - jobManagerStatefulSet - taskManagerStatefulSet type: object control: @@ -5243,12 +5482,18 @@ spec: - state - updateTime type: object - currentRevision: - type: string lastUpdateTime: type: string - nextRevision: - type: string + revision: + properties: + collisionCount: + format: int32 + type: integer + currentRevision: + type: string + nextRevision: + type: string + type: object savepoint: properties: jobID: diff --git a/controllers/batchscheduler/volcano/volcano_test.go b/controllers/batchscheduler/volcano/volcano_test.go index 485f374e..099b7c97 100644 --- a/controllers/batchscheduler/volcano/volcano_test.go +++ b/controllers/batchscheduler/volcano/volcano_test.go @@ -43,7 +43,7 @@ func TestGetClusterResource(t *testing.T) { }, }, Spec: appsv1.StatefulSetSpec{ - Replicas: &jmRep, + Replicas: &jmRep, ServiceName: "flinkjobcluster-sample-jobmanager", Selector: &metav1.LabelSelector{ MatchLabels: map[string]string{ @@ -173,8 +173,8 @@ func TestGetClusterResource(t *testing.T) { }, }, Spec: appsv1.StatefulSetSpec{ - Replicas: &replicas, - ServiceName: "flinkjobcluster-sample-taskmanager", + Replicas: &replicas, + ServiceName: "flinkjobcluster-sample-taskmanager", PodManagementPolicy: "Parallel", Selector: &metav1.LabelSelector{ MatchLabels: map[string]string{ diff --git a/controllers/flinkclient/flink_client.go b/controllers/flinkclient/flink_client.go index 81c0c8b4..5581d5ba 100644 --- a/controllers/flinkclient/flink_client.go +++ b/controllers/flinkclient/flink_client.go @@ -100,17 +100,40 @@ func (c *FlinkClient) StopJob( // TriggerSavepoint triggers an async savepoint operation. func (c *FlinkClient) TriggerSavepoint( - apiBaseURL string, jobID string, dir string) (SavepointTriggerID, error) { + apiBaseURL string, jobID string, dir string, cancel bool) (SavepointTriggerID, error) { var url = fmt.Sprintf("%s/jobs/%s/savepoints", apiBaseURL, jobID) var jsonStr = fmt.Sprintf(`{ "target-directory" : "%s", - "cancel-job" : false - }`, dir) + "cancel-job" : %v + }`, dir, cancel) var triggerID = SavepointTriggerID{} var err = c.HTTPClient.Post(url, []byte(jsonStr), &triggerID) return triggerID, err } +// TakeSavepoint takes savepoint, blocks until it succeeds or fails. +func (c *FlinkClient) TakeSavepoint( + apiBaseURL string, jobID string, dir string) (SavepointStatus, error) { + var triggerID = SavepointTriggerID{} + var status = SavepointStatus{JobID: jobID} + var err error + + triggerID, err = c.TriggerSavepoint(apiBaseURL, jobID, dir, false) + if err != nil { + return SavepointStatus{}, err + } + + for i := 0; i < 12; i++ { + status, err = c.GetSavepointStatus(apiBaseURL, jobID, triggerID.RequestID) + if err == nil && status.Completed { + return status, nil + } + time.Sleep(5 * time.Second) + } + + return status, err +} + // GetSavepointStatus returns savepoint status. // // Flink API response examples: @@ -181,39 +204,3 @@ func (c *FlinkClient) GetSavepointStatus( } return status, err } - -// TakeSavepoint takes savepoint, blocks until it suceeds or fails. -func (c *FlinkClient) TakeSavepoint( - apiBaseURL string, jobID string, dir string) (SavepointStatus, error) { - var triggerID = SavepointTriggerID{} - var status = SavepointStatus{JobID: jobID} - var err error - - triggerID, err = c.TriggerSavepoint(apiBaseURL, jobID, dir) - if err != nil { - return SavepointStatus{}, err - } - - for i := 0; i < 12; i++ { - status, err = c.GetSavepointStatus(apiBaseURL, jobID, triggerID.RequestID) - if err == nil && status.Completed { - return status, nil - } - time.Sleep(5 * time.Second) - } - - return status, err -} - -func (c *FlinkClient) TakeSavepointAsync( - apiBaseURL string, jobID string, dir string) (string, error) { - var triggerID = SavepointTriggerID{} - var err error - - triggerID, err = c.TriggerSavepoint(apiBaseURL, jobID, dir) - if err != nil { - return "", err - } - - return triggerID.RequestID, err -} diff --git a/controllers/flinkcluster_converter.go b/controllers/flinkcluster_converter.go index a79aa403..96cb42de 100644 --- a/controllers/flinkcluster_converter.go +++ b/controllers/flinkcluster_converter.go @@ -68,12 +68,12 @@ func getDesiredClusterState( return model.DesiredClusterState{} } return model.DesiredClusterState{ - ConfigMap: getDesiredConfigMap(cluster), + ConfigMap: getDesiredConfigMap(cluster), JmStatefulSet: getDesiredJobManagerStatefulSet(cluster), - JmService: getDesiredJobManagerService(cluster), - JmIngress: getDesiredJobManagerIngress(cluster), + JmService: getDesiredJobManagerService(cluster), + JmIngress: getDesiredJobManagerIngress(cluster), TmStatefulSet: getDesiredTaskManagerStatefulSet(cluster), - Job: getDesiredJob(observed), + Job: getDesiredJob(observed), } } @@ -102,7 +102,7 @@ func getDesiredJobManagerStatefulSet( var jobManagerStatefulSetName = getJobManagerStatefulSetName(clusterName) var podLabels = getComponentLabels(*flinkCluster, "jobmanager") podLabels = mergeLabels(podLabels, jobManagerSpec.PodLabels) - var statefulSetLabels = mergeLabels(podLabels, getRevisionHashLabels(flinkCluster.Status)) + var statefulSetLabels = mergeLabels(podLabels, getRevisionHashLabels(&flinkCluster.Status.Revision)) var securityContext = jobManagerSpec.SecurityContext // Make Volume, VolumeMount to use configMap data for flink-conf.yaml, if flinkProperties is provided. var volumes []corev1.Volume @@ -217,9 +217,9 @@ func getDesiredJobManagerStatefulSet( Labels: statefulSetLabels, }, Spec: appsv1.StatefulSetSpec{ - Replicas: jobManagerSpec.Replicas, - Selector: &metav1.LabelSelector{MatchLabels: podLabels}, - ServiceName: jobManagerStatefulSetName, + Replicas: jobManagerSpec.Replicas, + Selector: &metav1.LabelSelector{MatchLabels: podLabels}, + ServiceName: jobManagerStatefulSetName, VolumeClaimTemplates: jobManagerSpec.VolumeClaimTemplates, Template: corev1.PodTemplateSpec{ ObjectMeta: metav1.ObjectMeta{ @@ -263,7 +263,7 @@ func getDesiredJobManagerService( var jobManagerServiceName = getJobManagerServiceName(clusterName) var podLabels = getComponentLabels(*flinkCluster, "jobmanager") podLabels = mergeLabels(podLabels, jobManagerSpec.PodLabels) - var serviceLabels = mergeLabels(podLabels, getRevisionHashLabels(flinkCluster.Status)) + var serviceLabels = mergeLabels(podLabels, getRevisionHashLabels(&flinkCluster.Status.Revision)) var jobManagerService = &corev1.Service{ ObjectMeta: metav1.ObjectMeta{ Namespace: clusterNamespace, @@ -325,7 +325,7 @@ func getDesiredJobManagerIngress( var ingressTLS []extensionsv1beta1.IngressTLS var labels = mergeLabels( getComponentLabels(*flinkCluster, "jobmanager"), - getRevisionHashLabels(flinkCluster.Status)) + getRevisionHashLabels(&flinkCluster.Status.Revision)) if jobManagerIngressSpec.HostFormat != nil { ingressHost = getJobManagerIngressHost(*jobManagerIngressSpec.HostFormat, clusterName) } @@ -400,7 +400,7 @@ func getDesiredTaskManagerStatefulSet( var taskManagerStatefulSetName = getTaskManagerStatefulSetName(clusterName) var podLabels = getComponentLabels(*flinkCluster, "taskmanager") podLabels = mergeLabels(podLabels, taskManagerSpec.PodLabels) - var statefulSetLabels = mergeLabels(podLabels, getRevisionHashLabels(flinkCluster.Status)) + var statefulSetLabels = mergeLabels(podLabels, getRevisionHashLabels(&flinkCluster.Status.Revision)) var securityContext = taskManagerSpec.SecurityContext @@ -516,11 +516,11 @@ func getDesiredTaskManagerStatefulSet( Labels: statefulSetLabels, }, Spec: appsv1.StatefulSetSpec{ - Replicas: &taskManagerSpec.Replicas, - Selector: &metav1.LabelSelector{MatchLabels: podLabels}, - ServiceName: taskManagerStatefulSetName, + Replicas: &taskManagerSpec.Replicas, + Selector: &metav1.LabelSelector{MatchLabels: podLabels}, + ServiceName: taskManagerStatefulSetName, VolumeClaimTemplates: taskManagerSpec.VolumeClaimTemplates, - PodManagementPolicy: "Parallel", + PodManagementPolicy: "Parallel", Template: corev1.PodTemplateSpec{ ObjectMeta: metav1.ObjectMeta{ Labels: podLabels, @@ -549,7 +549,7 @@ func getDesiredConfigMap( var configMapName = getConfigMapName(clusterName) var labels = mergeLabels( getClusterLabels(*flinkCluster), - getRevisionHashLabels(flinkCluster.Status)) + getRevisionHashLabels(&flinkCluster.Status.Revision)) var flinkHeapSize = calFlinkHeapSize(flinkCluster) // Properties which should be provided from real deployed environment. var flinkProps = map[string]string{ @@ -594,19 +594,18 @@ func getDesiredConfigMap( // Gets the desired job spec from a cluster spec. func getDesiredJob(observed *ObservedClusterState) *batchv1.Job { var flinkCluster = observed.cluster + var recorded = flinkCluster.Status var jobSpec = flinkCluster.Spec.Job - var jobStatus = flinkCluster.Status.Components.Job + var jobStatus = recorded.Components.Job if jobSpec == nil { return nil } - // Unless update has been triggered or the job needs to be restarted, keep the job to be stopped in that state. - if !(isUpdateTriggered(flinkCluster.Status) || shouldRestartJob(jobSpec.RestartPolicy, jobStatus)) { - // Job cancel requested or stopped already - if isJobCancelRequested(*flinkCluster) || isJobStopped(jobStatus) { - return nil - } + // When the job should be stopped, keep that state unless update is triggered or the job must to be restarted. + if (shouldStopJob(flinkCluster) || jobStatus.IsStopped()) && + !(shouldUpdateJob(observed) || jobStatus.ShouldRestart(jobSpec)) { + return nil } var clusterSpec = flinkCluster.Spec @@ -621,14 +620,14 @@ func getDesiredJob(observed *ObservedClusterState) *batchv1.Job { "%s:%d", jobManagerServiceName, *jobManagerSpec.Ports.UI) var podLabels = getClusterLabels(*flinkCluster) podLabels = mergeLabels(podLabels, jobManagerSpec.PodLabels) - var jobLabels = mergeLabels(podLabels, getRevisionHashLabels(flinkCluster.Status)) + var jobLabels = mergeLabels(podLabels, getRevisionHashLabels(&recorded.Revision)) var jobArgs = []string{"bash", "/opt/flink-operator/submit-job.sh"} jobArgs = append(jobArgs, "--jobmanager", jobManagerAddress) if jobSpec.ClassName != nil { jobArgs = append(jobArgs, "--class", *jobSpec.ClassName) } - var fromSavepoint = convertFromSavepoint(jobSpec, flinkCluster.Status.Components.Job) + var fromSavepoint = convertFromSavepoint(jobSpec, jobStatus, &recorded.Revision) if fromSavepoint != nil { jobArgs = append(jobArgs, "--fromSavepoint", *fromSavepoint) } @@ -771,15 +770,16 @@ func getDesiredJob(observed *ObservedClusterState) *batchv1.Job { // Flink job will be restored from the latest savepoint created by the operator. // // case 3) When latest created savepoint is unavailable, use the savepoint from which current job was restored. -func convertFromSavepoint(jobSpec *v1beta1.JobSpec, jobStatus *v1beta1.JobStatus) *string { +func convertFromSavepoint(jobSpec *v1beta1.JobSpec, jobStatus *v1beta1.JobStatus, revision *v1beta1.RevisionStatus) *string { switch { // Creating for the first time case jobStatus == nil: - if jobSpec.FromSavepoint != nil && *jobSpec.FromSavepoint != "" { + if !isBlank(jobSpec.FromSavepoint) { return jobSpec.FromSavepoint } + return nil // Updating with FromSavepoint provided - case jobStatus.State == v1beta1.JobStateUpdating && jobSpec.FromSavepoint != nil && *jobSpec.FromSavepoint != "": + case revision.IsUpdateTriggered() && !isBlank(jobSpec.FromSavepoint): return jobSpec.FromSavepoint // Latest savepoint case jobStatus.SavepointLocation != "": @@ -872,7 +872,7 @@ func shouldCleanup( return false } - if isUpdateTriggered(cluster.Status) { + if cluster.Status.Revision.IsUpdateTriggered() { return false } @@ -1070,9 +1070,9 @@ func getComponentLabels(cluster v1beta1.FlinkCluster, component string) map[stri }) } -func getRevisionHashLabels(status v1beta1.FlinkClusterStatus) map[string]string { +func getRevisionHashLabels(r *v1beta1.RevisionStatus) map[string]string { return map[string]string{ - RevisionNameLabel: getNextRevisionName(status), + RevisionNameLabel: getNextRevisionName(r), } } diff --git a/controllers/flinkcluster_converter_test.go b/controllers/flinkcluster_converter_test.go index 78923e62..c881d63b 100644 --- a/controllers/flinkcluster_converter_test.go +++ b/controllers/flinkcluster_converter_test.go @@ -273,7 +273,7 @@ func TestGetDesiredClusterState(t *testing.T) { }, }, Status: v1beta1.FlinkClusterStatus{ - NextRevision: "flinkjobcluster-sample-85dc8f749-1", + Revision: v1beta1.RevisionStatus{NextRevision: "flinkjobcluster-sample-85dc8f749-1"}, }, }, } @@ -574,8 +574,8 @@ func TestGetDesiredClusterState(t *testing.T) { }, }, Spec: appsv1.StatefulSetSpec{ - Replicas: &replicas, - ServiceName: "flinkjobcluster-sample-taskmanager", + Replicas: &replicas, + ServiceName: "flinkjobcluster-sample-taskmanager", PodManagementPolicy: "Parallel", Selector: &metav1.LabelSelector{ MatchLabels: map[string]string{ @@ -984,7 +984,7 @@ func TestSecurityContext(t *testing.T) { }, }, Status: v1beta1.FlinkClusterStatus{ - NextRevision: "flinkjobcluster-sample-85dc8f749-1", + Revision: v1beta1.RevisionStatus{NextRevision: "flinkjobcluster-sample-85dc8f749-1"}, }, }, } @@ -1022,7 +1022,7 @@ func TestSecurityContext(t *testing.T) { }, }, Status: v1beta1.FlinkClusterStatus{ - NextRevision: "flinkjobcluster-sample-85dc8f749-1", + Revision: v1beta1.RevisionStatus{NextRevision: "flinkjobcluster-sample-85dc8f749-1"}, }, }, } diff --git a/controllers/flinkcluster_observer.go b/controllers/flinkcluster_observer.go index 31c08fa8..adf84fe0 100644 --- a/controllers/flinkcluster_observer.go +++ b/controllers/flinkcluster_observer.go @@ -18,12 +18,12 @@ package controllers import ( "context" - "errors" "fmt" "github.com/go-logr/logr" v1beta1 "github.com/googlecloudplatform/flink-operator/api/v1beta1" "github.com/googlecloudplatform/flink-operator/controllers/flinkclient" "github.com/googlecloudplatform/flink-operator/controllers/history" + yaml "gopkg.in/yaml.v2" appsv1 "k8s.io/api/apps/v1" batchv1 "k8s.io/api/batch/v1" corev1 "k8s.io/api/core/v1" @@ -51,31 +51,67 @@ type ObservedClusterState struct { cluster *v1beta1.FlinkCluster revisions []*appsv1.ControllerRevision configMap *corev1.ConfigMap - jmStatefulSet *appsv1.StatefulSet + jmStatefulSet *appsv1.StatefulSet jmService *corev1.Service jmIngress *extensionsv1beta1.Ingress - tmStatefulSet *appsv1.StatefulSet - job *batchv1.Job - jobPod *corev1.Pod - flinkJobStatus FlinkJobStatus - flinkJobSubmitLog *FlinkJobSubmitLog - savepoint *flinkclient.SavepointStatus - revisionStatus *RevisionStatus - savepointErr error + tmStatefulSet *appsv1.StatefulSet + flinkJob FlinkJob + flinkJobSubmitter FlinkJobSubmitter + savepoint Savepoint + revision Revision observeTime time.Time + updateState UpdateState } -type FlinkJobStatus struct { - flinkJob *flinkclient.JobStatus - flinkJobList *flinkclient.JobStatusList - flinkJobsUnexpected []string +type FlinkJob struct { + status *flinkclient.JobStatus + list *flinkclient.JobStatusList + unexpected []string } -type FlinkJobSubmitLog struct { +type FlinkJobSubmitter struct { + job *batchv1.Job + pod *corev1.Pod + log *SubmitterLog +} + +type SubmitterLog struct { JobID string `yaml:"jobID,omitempty"` Message string `yaml:"message"` } +type Savepoint struct { + status *flinkclient.SavepointStatus + error error +} + +type Revision struct { + currentRevision *appsv1.ControllerRevision + nextRevision *appsv1.ControllerRevision + collisionCount int32 +} + +func (o *ObservedClusterState) isClusterUpdating() bool { + return o.updateState == UpdateStateInProgress +} + +// Job submitter status. +func (s *FlinkJobSubmitter) getState() JobSubmitState { + switch { + case s.job == nil: + break + case s.job.Status.Succeeded == 0 && s.job.Status.Failed == 0: + return JobDeployStateInProgress + case s.job.Status.Failed > 0: + return JobDeployStateFailed + case s.job.Status.Succeeded > 0: + if s.log != nil && s.log.JobID != "" { + return JobDeployStateSucceeded + } + } + return JobDeployStateUnknown +} + // Observes the state of the cluster and its components. // NOT_FOUND error is ignored because it is normal, other errors are returned. func (observer *ClusterStateObserver) observe( @@ -192,86 +228,124 @@ func (observer *ClusterStateObserver) observe( } // (Optional) Savepoint. - // Savepoint observe error do not affect deploy reconciliation loop. - observer.observeSavepoint(observed) + var observedSavepoint Savepoint + err = observer.observeSavepoint(observed.cluster, &observedSavepoint) + if err != nil { + log.Error(err, "Failed to get Flink job savepoint status") + } else { + log.Info("Observed Flink job savepoint status", "status", observedSavepoint.status) + } + observed.savepoint = observedSavepoint // (Optional) job. err = observer.observeJob(observed) + if err != nil { + log.Error(err, "Failed to get Flink job status") + return err + } observed.observeTime = time.Now() + observed.updateState = getUpdateState(observed) - return err + return nil } func (observer *ClusterStateObserver) observeJob( observed *ObservedClusterState) error { - if observed.cluster == nil { + // Either the cluster has been deleted or it is a session cluster. + if observed.cluster == nil || observed.cluster.Spec.Job == nil { return nil } + var log = observer.log + var recorded = observed.cluster.Status + var err error - // Observe following - var observedJob *batchv1.Job - var observedFlinkJobStatus FlinkJobStatus - var observedFlinkJobSubmitLog *FlinkJobSubmitLog + // Observe the Flink job submitter. + var submitter FlinkJobSubmitter + err = observer.observeSubmitter(&submitter) + if err != nil { + log.Error(err, "Failed to get the status of the job submitter") + } + observed.flinkJobSubmitter = submitter - var recordedJobStatus = observed.cluster.Status.Components.Job - var err error + // Observe the Flink job status. + var flinkJobID string + // Get the ID from the job submitter. + if submitter.log != nil && submitter.log.JobID != "" { + flinkJobID = submitter.log.JobID + } else + // Or get the job ID from the recorded job status which is written in previous iteration. + if recorded.Components.Job != nil { + flinkJobID = recorded.Components.Job.ID + } + var observedFlinkJob FlinkJob + observer.observeFlinkJobStatus(observed, flinkJobID, &observedFlinkJob) + observed.flinkJob = observedFlinkJob + + return nil +} + +func (observer *ClusterStateObserver) observeSubmitter(submitter *FlinkJobSubmitter) error { var log = observer.log + var err error - // Either the cluster has been deleted or it is a session cluster. - if observed.cluster == nil || observed.cluster.Spec.Job == nil { - return nil - } + // Observe following + var job *batchv1.Job + var pod *corev1.Pod + var podLog *SubmitterLog // Job resource. - observedJob = new(batchv1.Job) - err = observer.observeJobResource(observedJob) + job = new(batchv1.Job) + err = observer.observeSubmitterJob(job) if err != nil { if client.IgnoreNotFound(err) != nil { - log.Error(err, "Failed to get job") + log.Error(err, "Failed to get the submitter job") return err } - log.Info("Observed job submitter", "state", "nil") - observedJob = nil + log.Info("Observed submitter job", "state", "nil") + job = nil } else { - log.Info("Observed job submitter", "state", *observedJob) + log.Info("Observed submitter job", "state", *job) } - observed.job = observedJob - - // Get Flink job ID. - // While job state is pending and job submitter is completed, extract the job ID from the pod termination log. - var jobSubmitCompleted = observedJob != nil && (observedJob.Status.Succeeded > 0 || observedJob.Status.Failed > 0) - var jobInPendingState = recordedJobStatus != nil && recordedJobStatus.State == v1beta1.JobStatePending - var flinkJobID string - if jobSubmitCompleted && jobInPendingState { - var observedJobPod *corev1.Pod + submitter.job = job - // Get job submitter pod resource. - observedJobPod = new(corev1.Pod) - err = observer.observeJobPod(observedJobPod) - if err != nil { - log.Error(err, "Failed to get job pod") - } - observed.jobPod = observedJobPod - - // Extract submit result. - observedFlinkJobSubmitLog, err = getFlinkJobSubmitLog(observedJobPod) - if err != nil { - log.Error(err, "Failed to extract job submit result") - } - if observedFlinkJobSubmitLog != nil && observedFlinkJobSubmitLog.JobID != "" { - flinkJobID = observedFlinkJobSubmitLog.JobID - } - observed.flinkJobSubmitLog = observedFlinkJobSubmitLog + // Get the job submission log. + // When the recorded job state is pending or updating, and the actual submission is completed, + // extract the job submission log from the pod termination log. + if submitter.job == nil { + return nil } - // Or get the job ID from the recorded job status which is written previous iteration. - if flinkJobID == "" && recordedJobStatus != nil { - flinkJobID = recordedJobStatus.ID + // Get job submitter pod resource. + pod = new(corev1.Pod) + err = observer.observeJobSubmitterPod(pod) + if err != nil { + log.Error(err, "Failed to get the submitter pod") + return err + } else if pod == nil { + log.Info("Observed submitter job pod", "state", "nil") + return nil + } else { + log.Info("Observed submitter job pod", "state", *pod) } + submitter.pod = pod - // Flink job status. - observer.observeFlinkJobStatus(observed, flinkJobID, &observedFlinkJobStatus) - observed.flinkJobStatus = observedFlinkJobStatus + // Extract submission result. + var jobSubmissionCompleted = job.Status.Succeeded > 0 || job.Status.Failed > 0 + if !jobSubmissionCompleted { + return nil + } + log.Info("Extracting the result of job submission because it is completed") + podLog = new(SubmitterLog) + err = observer.observeFlinkJobSubmitterLog(pod, podLog) + if err != nil { + log.Error(err, "Failed to extract the job submission result") + podLog = nil + } else if podLog == nil { + log.Info("Observed submitter log", "state", "nil") + } else { + log.Info("Observed submitter log", "state", *podLog) + } + submitter.log = podLog return nil } @@ -284,11 +358,11 @@ func (observer *ClusterStateObserver) observeJob( func (observer *ClusterStateObserver) observeFlinkJobStatus( observed *ObservedClusterState, flinkJobID string, - flinkJobStatus *FlinkJobStatus) { + flinkJob *FlinkJob) { var log = observer.log // Observe following - var flinkJob *flinkclient.JobStatus + var flinkJobStatus *flinkclient.JobStatus var flinkJobList *flinkclient.JobStatusList var flinkJobsUnexpected []string @@ -310,71 +384,47 @@ func (observer *ClusterStateObserver) observeFlinkJobStatus( log.Info("Failed to get Flink job status list.", "error", err) return } - log.Info("Observed Flink job status list", "jobs", flinkJobList.Jobs) + flinkJob.list = flinkJobList - // Initialize flinkJobStatus if flink API is available. - flinkJobStatus.flinkJobList = flinkJobList - - // Extract the current job status and unexpected jobs, if submitted job ID is provided. - if flinkJobID == "" { - return - } + // Extract the current job status and unexpected jobs. for _, job := range flinkJobList.Jobs { if flinkJobID == job.ID { - flinkJob = new(flinkclient.JobStatus) - *flinkJob = job + flinkJobStatus = &job } else if getFlinkJobDeploymentState(job.Status) == v1beta1.JobStateRunning { flinkJobsUnexpected = append(flinkJobsUnexpected, job.ID) } } - flinkJobStatus.flinkJob = flinkJob - flinkJobStatus.flinkJobsUnexpected = flinkJobsUnexpected - - // It is okay if there are multiple jobs, but at most one of them is - // expected to be running. This is typically caused by job client - // timed out and exited but the job submission was actually - // successfully. When retrying, it first cancels the existing running - // job which it has lost track of, then submit the job again. - if len(flinkJobsUnexpected) > 1 { - log.Error( - errors.New("more than one unexpected Flink job were found"), - "", "unexpected jobs", flinkJobsUnexpected) - } - if flinkJob != nil { - log.Info("Observed Flink job", "flink job", *flinkJob) + flinkJob.status = flinkJobStatus + flinkJob.unexpected = flinkJobsUnexpected + log.Info("Observed Flink job", + "submitted job status", flinkJob.status, + "all job list", flinkJob.list, + "unexpected job list", flinkJob.unexpected) + if len(flinkJobsUnexpected) > 0 { + log.Info("More than one unexpected Flink job were found!") } return } -func (observer *ClusterStateObserver) observeSavepoint(observed *ObservedClusterState) error { - var log = observer.log - - if observed.cluster == nil { +func (observer *ClusterStateObserver) observeSavepoint(cluster *v1beta1.FlinkCluster, savepoint *Savepoint) error { + if cluster == nil || + cluster.Status.Savepoint == nil || + cluster.Status.Savepoint.State != v1beta1.SavepointStateInProgress { return nil } // Get savepoint status in progress. - var savepointStatus = observed.cluster.Status.Savepoint - if savepointStatus != nil && savepointStatus.State == v1beta1.SavepointStateInProgress { - var flinkAPIBaseURL = getFlinkAPIBaseURL(observed.cluster) - var jobID = savepointStatus.JobID - var triggerID = savepointStatus.TriggerID - var savepoint flinkclient.SavepointStatus - var err error - - savepoint, err = observer.flinkClient.GetSavepointStatus(flinkAPIBaseURL, jobID, triggerID) - observed.savepoint = &savepoint - if err == nil && len(savepoint.FailureCause.StackTrace) > 0 { - err = fmt.Errorf("%s", savepoint.FailureCause.StackTrace) - } - if err != nil { - observed.savepointErr = err - log.Info("Failed to get savepoint.", "error", err, "jobID", jobID, "triggerID", triggerID) - } - return err - } - return nil + var flinkAPIBaseURL = getFlinkAPIBaseURL(cluster) + var recordedSavepoint = cluster.Status.Savepoint + var jobID = recordedSavepoint.JobID + var triggerID = recordedSavepoint.TriggerID + + savepointStatus, err := observer.flinkClient.GetSavepointStatus(flinkAPIBaseURL, jobID, triggerID) + savepoint.status = &savepointStatus + savepoint.error = err + + return err } func (observer *ClusterStateObserver) observeCluster( @@ -479,7 +529,7 @@ func (observer *ClusterStateObserver) observeJobManagerIngress( observedIngress) } -func (observer *ClusterStateObserver) observeJobResource( +func (observer *ClusterStateObserver) observeSubmitterJob( observedJob *batchv1.Job) error { var clusterNamespace = observer.request.Namespace var clusterName = observer.request.Name @@ -493,10 +543,9 @@ func (observer *ClusterStateObserver) observeJobResource( observedJob) } -// observeJobPod observes job submitter pod. -func (observer *ClusterStateObserver) observeJobPod( +// observeJobSubmitterPod observes job submitter pod. +func (observer *ClusterStateObserver) observeJobSubmitterPod( observedPod *corev1.Pod) error { - var log = observer.log var clusterNamespace = observer.request.Namespace var clusterName = observer.request.Name var podSelector = labels.SelectorFromSet(map[string]string{"job-name": getJobName(clusterName)}) @@ -508,25 +557,15 @@ func (observer *ClusterStateObserver) observeJobPod( client.InNamespace(clusterNamespace), client.MatchingLabelsSelector{Selector: podSelector}) if err != nil { - if client.IgnoreNotFound(err) != nil { - log.Error(err, "Failed to get job submitter pod list") - return err - } - log.Info("Observed job submitter pod list", "state", "nil") - } else { - log.Info("Observed job submitter pod list", "state", *podList) + return err } - - if podList != nil && len(podList.Items) > 0 { + if len(podList.Items) == 0 { + observedPod = nil + } else { podList.Items[0].DeepCopyInto(observedPod) } - return nil -} -type RevisionStatus struct { - currentRevision *appsv1.ControllerRevision - nextRevision *appsv1.ControllerRevision - collisionCount int32 + return nil } // syncRevisionStatus synchronizes current FlinkCluster resource and its child ControllerRevision resources. @@ -544,20 +583,19 @@ func (observer *ClusterStateObserver) syncRevisionStatus(observed *ObservedClust return nil } - var revisions = observed.revisions var cluster = observed.cluster - var recordedStatus = cluster.Status + var revisions = observed.revisions + var recorded = cluster.Status var currentRevision, nextRevision *appsv1.ControllerRevision var controllerHistory = observer.history - var revisionStatus = observed.revisionStatus revisionCount := len(revisions) history.SortControllerRevisions(revisions) // Use a local copy of cluster.Status.CollisionCount to avoid modifying cluster.Status directly. var collisionCount int32 - if recordedStatus.CollisionCount != nil { - collisionCount = *recordedStatus.CollisionCount + if recorded.Revision.CollisionCount != nil { + collisionCount = *recorded.Revision.CollisionCount } // create a new revision from the current cluster @@ -590,12 +628,12 @@ func (observer *ClusterStateObserver) syncRevisionStatus(observed *ObservedClust } // if the current revision is nil we initialize the history by setting it to the next revision - if recordedStatus.CurrentRevision == "" { + if recorded.Revision.CurrentRevision == "" { currentRevision = nextRevision // attempt to find the revision that corresponds to the current revision } else { for i := range revisions { - if revisions[i].Name == getCurrentRevisionName(recordedStatus) { + if revisions[i].Name == getCurrentRevisionName(&recorded.Revision) { currentRevision = revisions[i] break } @@ -605,12 +643,12 @@ func (observer *ClusterStateObserver) syncRevisionStatus(observed *ObservedClust return fmt.Errorf("current ControlRevision resoucre not found") } - // update revision status - revisionStatus = new(RevisionStatus) - revisionStatus.currentRevision = currentRevision.DeepCopy() - revisionStatus.nextRevision = nextRevision.DeepCopy() - revisionStatus.collisionCount = collisionCount - observed.revisionStatus = revisionStatus + // Update revision status. + observed.revision = Revision{ + currentRevision: currentRevision.DeepCopy(), + nextRevision: nextRevision.DeepCopy(), + collisionCount: collisionCount, + } // maintain the revision history limit err = observer.truncateHistory(observed) @@ -642,3 +680,25 @@ func (observer *ClusterStateObserver) truncateHistory(observed *ObservedClusterS } return nil } + +// observeFlinkJobSubmit extract submit result from the pod termination log. +func (observer *ClusterStateObserver) observeFlinkJobSubmitterLog(observedPod *corev1.Pod, submitterLog *SubmitterLog) error { + var log = observer.log + var containerStatuses = observedPod.Status.ContainerStatuses + if len(containerStatuses) == 0 || + containerStatuses[0].State.Terminated == nil || + containerStatuses[0].State.Terminated.Message == "" { + submitterLog = nil + log.Info("job pod found, but no termination log") + return nil + } + + // The job submission script writes the submission log to the pod termination log at the end of execution. + // If the job submission is successful, the extracted job ID is also included. + // The job submit script writes the submission result in YAML format, + // so parse it here to get the ID - if available - and log. + // Note: https://kubernetes.io/docs/tasks/debug-application-cluster/determine-reason-pod-failure/ + var rawJobSubmissionResult = containerStatuses[0].State.Terminated.Message + var err = yaml.Unmarshal([]byte(rawJobSubmissionResult), submitterLog) + return err +} diff --git a/controllers/flinkcluster_observer_test.go b/controllers/flinkcluster_observer_test.go new file mode 100644 index 00000000..3a03b5d4 --- /dev/null +++ b/controllers/flinkcluster_observer_test.go @@ -0,0 +1,60 @@ +/* +Copyright 2019 Google LLC. + +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 controllers + +import ( + "gotest.tools/assert" + corev1 "k8s.io/api/core/v1" + "testing" +) + +func TestGetFlinkJobDeploymentState(t *testing.T) { + var pod corev1.Pod + var submitterLog, expected *SubmitterLog + var termMsg string + var observer = ClusterStateObserver{} + + // success + termMsg = ` +jobID: ec74209eb4e3db8ae72db00bd7a830aa +message: | + Successfully submitted! + /opt/flink/bin/flink run --jobmanager flinkjobcluster-sample-jobmanager:8081 --class org.apache.flink.streaming.examples.wordcount.WordCount --parallelism 2 --detached ./examples/streaming/WordCount.jar --input ./README.txt + Starting execution of program + Printing result to stdout. Use --output to specify output path. + Job has been submitted with JobID ec74209eb4e3db8ae72db00bd7a830aa +` + expected = &SubmitterLog{ + JobID: "ec74209eb4e3db8ae72db00bd7a830aa", + Message: `Successfully submitted! +/opt/flink/bin/flink run --jobmanager flinkjobcluster-sample-jobmanager:8081 --class org.apache.flink.streaming.examples.wordcount.WordCount --parallelism 2 --detached ./examples/streaming/WordCount.jar --input ./README.txt +Starting execution of program +Printing result to stdout. Use --output to specify output path. +Job has been submitted with JobID ec74209eb4e3db8ae72db00bd7a830aa +`, + } + pod = corev1.Pod{ + Status: corev1.PodStatus{ + ContainerStatuses: []corev1.ContainerStatus{{ + State: corev1.ContainerState{ + Terminated: &corev1.ContainerStateTerminated{ + Message: termMsg, + }}}}}} + submitterLog = new(SubmitterLog) + _ = observer.observeFlinkJobSubmitterLog(&pod, submitterLog) + assert.DeepEqual(t, *submitterLog, *expected) +} diff --git a/controllers/flinkcluster_reconciler.go b/controllers/flinkcluster_reconciler.go index 7760ca85..06eef321 100644 --- a/controllers/flinkcluster_reconciler.go +++ b/controllers/flinkcluster_reconciler.go @@ -18,13 +18,13 @@ package controllers import ( "context" + "errors" "fmt" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/runtime" "k8s.io/apimachinery/pkg/types" "k8s.io/client-go/tools/record" "k8s.io/client-go/util/retry" - "reflect" "time" "github.com/go-logr/logr" @@ -53,7 +53,9 @@ type ClusterReconciler struct { recorder record.EventRecorder } -var requeueResult = ctrl.Result{RequeueAfter: 10 * time.Second, Requeue: true} +const JobCheckInterval = 10 * time.Second + +var requeueResult = ctrl.Result{RequeueAfter: JobCheckInterval, Requeue: true} // Compares the desired state and the observed state, if there is a difference, // takes actions to drive the observed state towards the desired state. @@ -66,7 +68,7 @@ func (reconciler *ClusterReconciler) reconcile() (ctrl.Result, error) { return ctrl.Result{}, nil } - if getUpdateState(reconciler.observed) == UpdateStateInProgress { + if shouldUpdateCluster(&reconciler.observed) { reconciler.log.Info("The cluster update is in progress") } // If batch-scheduling enabled @@ -138,7 +140,8 @@ func (reconciler *ClusterReconciler) reconcileStatefulSet( } if desiredStatefulSet != nil && observedStatefulSet != nil { - if getUpdateState(reconciler.observed) == UpdateStateInProgress { + var cluster = reconciler.observed.cluster + if shouldUpdateCluster(&reconciler.observed) && !isComponentUpdated(observedStatefulSet, cluster) { updateComponent := fmt.Sprintf("%v StatefulSet", component) var err error if *reconciler.observed.cluster.Spec.RecreateOnUpdate { @@ -151,7 +154,7 @@ func (reconciler *ClusterReconciler) reconcileStatefulSet( } return nil } - log.Info("Statefulset already exists, no action") + log.Info("StatefulSet already exists, no action") return nil } @@ -180,7 +183,7 @@ func (reconciler *ClusterReconciler) createStatefulSet( func (reconciler *ClusterReconciler) deleteOldComponent(desired runtime.Object, observed runtime.Object, component string) error { var log = reconciler.log.WithValues("component", component) - if isComponentUpdated(observed, *reconciler.observed.cluster) { + if isComponentUpdated(observed, reconciler.observed.cluster) { reconciler.log.Info(fmt.Sprintf("%v is already updated, no action", component)) return nil } @@ -254,7 +257,8 @@ func (reconciler *ClusterReconciler) reconcileJobManagerService() error { } if desiredJmService != nil && observedJmService != nil { - if getUpdateState(reconciler.observed) == UpdateStateInProgress { + var cluster = reconciler.observed.cluster + if shouldUpdateCluster(&reconciler.observed) && !isComponentUpdated(observedJmService, cluster) { // v1.Service API does not handle update correctly when below values are empty. desiredJmService.SetResourceVersion(observedJmService.GetResourceVersion()) desiredJmService.Spec.ClusterIP = observedJmService.Spec.ClusterIP @@ -322,7 +326,8 @@ func (reconciler *ClusterReconciler) reconcileJobManagerIngress() error { } if desiredJmIngress != nil && observedJmIngress != nil { - if getUpdateState(reconciler.observed) == UpdateStateInProgress { + var cluster = reconciler.observed.cluster + if shouldUpdateCluster(&reconciler.observed) && !isComponentUpdated(observedJmIngress, cluster) { var err error if *reconciler.observed.cluster.Spec.RecreateOnUpdate { err = reconciler.deleteOldComponent(desiredJmIngress, observedJmIngress, "JobManager ingress") @@ -387,7 +392,8 @@ func (reconciler *ClusterReconciler) reconcileConfigMap() error { } if desiredConfigMap != nil && observedConfigMap != nil { - if getUpdateState(reconciler.observed) == UpdateStateInProgress { + var cluster = reconciler.observed.cluster + if shouldUpdateCluster(&reconciler.observed) && !isComponentUpdated(observedConfigMap, cluster) { var err error if *reconciler.observed.cluster.Spec.RecreateOnUpdate { err = reconciler.deleteOldComponent(desiredConfigMap, observedConfigMap, "ConfigMap") @@ -447,50 +453,47 @@ func (reconciler *ClusterReconciler) reconcileJob() (ctrl.Result, error) { var log = reconciler.log var desiredJob = reconciler.desired.Job var observed = reconciler.observed - var observedJob = observed.job - var recordedJobStatus = observed.cluster.Status.Components.Job - var activeFlinkJob bool + var recorded = observed.cluster.Status + var jobSpec = observed.cluster.Spec.Job + var job = recorded.Components.Job var err error + var jobID = reconciler.getFlinkJobID() // Update status changed via job reconciliation. var newSavepointStatus *v1beta1.SavepointStatus var newControlStatus *v1beta1.FlinkClusterControlStatus defer reconciler.updateStatus(&newSavepointStatus, &newControlStatus) - // Cancel unexpected jobs - if len(observed.flinkJobStatus.flinkJobsUnexpected) > 0 { - log.Info("Cancelling unexpected running job(s)") - err = reconciler.cancelUnexpectedJobs(false /* takeSavepoint */) - return requeueResult, err - } - - // Check if Flink job is active - if isJobActive(recordedJobStatus) { - activeFlinkJob = true - } else { - activeFlinkJob = false - } - - // Create Flink job submitter - if desiredJob != nil && !activeFlinkJob { - // If update triggered, wait until all Flink cluster components are replaced with next revision. - if !isClusterUpdateToDate(observed) { - return requeueResult, nil + // Create new Flink job submitter when starting new job, updating job or restarting job in failure. + if desiredJob != nil && !job.IsActive() { + log.Info("Deploying Flink job") + + // TODO: Record event or introduce Condition in CRD status to notify update state pended. + // https://github.com/kubernetes/apimachinery/blob/57f2a0733447cfd41294477d833cce6580faaca3/pkg/apis/meta/v1/types.go#L1376 + var unexpectedJobs = observed.flinkJob.unexpected + if len(unexpectedJobs) > 0 { + // This is an exceptional situation. + // There should be no jobs because all jobs are terminated in the previous iterations. + // In this case user should identify the problem so that the job is not executed multiple times unintentionally + // cause of Flink error, Flink operator error or other unknown error. + // If user want to proceed, unexpected jobs should be terminated. + log.Error(errors.New("unexpected jobs found"), "Failed to create job submitter", "unexpected jobs", unexpectedJobs) + return ctrl.Result{}, nil } // Create Flink job submitter - log.Info("Updating job status to create new job submitter") - err = reconciler.updateStatusForNewJob() + log.Info("Updating job status to proceed creating new job submitter") + // Job status must be updated before creating a job submitter to ensure the observed job is the job submitted by the operator. + err = reconciler.updateJobDeployStatus() if err != nil { - log.Info("Not proceed to create new job submitter because job status update failed") + log.Info("Failed to update the job status for job submission") return requeueResult, err } - log.Info("Creating new job submitter") - if observedJob != nil { - log.Info("Deleting old job submitter") - err = reconciler.deleteJob(observedJob) + var observedSubmitter = observed.flinkJobSubmitter.job + if observedSubmitter != nil { + log.Info("Found old job submitter") + err = reconciler.deleteJob(observedSubmitter) if err != nil { - log.Info("Failed to delete previous job submitter") return requeueResult, err } } @@ -499,67 +502,55 @@ func (reconciler *ClusterReconciler) reconcileJob() (ctrl.Result, error) { return requeueResult, err } - if desiredJob != nil && activeFlinkJob { - var jobID = reconciler.getFlinkJobID() - var restartPolicy = observed.cluster.Spec.Job.RestartPolicy - var recordedJobStatus = observed.cluster.Status.Components.Job - var jobSpec = reconciler.observed.cluster.Spec.Job + if desiredJob != nil && job.IsActive() { + if job.State == v1beta1.JobStateDeploying { + log.Info("Job submitter is deployed, wait until completed") + return requeueResult, nil + } - // Update or recover Flink job by restart. - if shouldUpdateJob(observed) { - log.Info("Job is about to be restarted to update") - err := reconciler.restartJob(*jobSpec.TakeSavepointOnUpgrade) - return requeueResult, err - } else if shouldRestartJob(restartPolicy, recordedJobStatus) { - log.Info("Job is about to be restarted to recover failure") - err := reconciler.restartJob(false) + // Suspend or stop job to proceed update. + if recorded.Revision.IsUpdateTriggered() { + log.Info("Preparing job update") + var takeSavepoint = jobSpec.TakeSavepointOnUpdate == nil || *jobSpec.TakeSavepointOnUpdate + var shouldSuspend = takeSavepoint && isBlank(jobSpec.FromSavepoint) + if shouldSuspend { + newSavepointStatus, err = reconciler.trySuspendJob() + } else if shouldUpdateJob(&observed) { + err = reconciler.cancelJob() + } return requeueResult, err } // Trigger savepoint if required. if len(jobID) > 0 { - shouldTakeSavepont, savepointTriggerReason := reconciler.shouldTakeSavepoint() - if shouldTakeSavepont { - err = reconciler.updateSavepointTriggerTimeStatus() - if err == nil { - newSavepointStatus, _ = reconciler.takeSavepointAsync(jobID, savepointTriggerReason) - } + var savepointTriggerReason = reconciler.shouldTakeSavepoint() + if savepointTriggerReason != "" { + newSavepointStatus, err = reconciler.triggerSavepoint(jobID, savepointTriggerReason, false) + } + // Get new control status when the savepoint reason matches the requested control. + var userControl = getNewControlRequest(observed.cluster) + if userControl == v1beta1.ControlNameSavepoint && savepointTriggerReason == v1beta1.SavepointTriggerReasonUserRequested { + newControlStatus = getControlStatus(userControl, v1beta1.ControlStateInProgress) } + return requeueResult, err } + log.Info("Job is not finished yet, no action", "jobID", jobID) return requeueResult, nil } - // Stop Flink job - if desiredJob == nil && activeFlinkJob { - // Cancel Flink job if it is live - // case 1) In the case of which savepoint was triggered, after it is completed, proceed to delete step. - // case 2) When savepoint was skipped, continue to delete step immediately. - // - // If savepoint or cancellation was failed, the control state is fallen to the failed in the updater. - var jobID = reconciler.getFlinkJobID() - log.Info("Cancelling job", "jobID", jobID) - - var savepointStatus, err = reconciler.cancelFlinkJobAsync(jobID, true /* takeSavepoint */) - if !reflect.DeepEqual(savepointStatus, observed.cluster.Status.Savepoint) { - newSavepointStatus = savepointStatus - } - if err != nil { - log.Error(err, "Failed to cancel job", "jobID", jobID) - newControlStatus = getFailedCancelStatus(err) - return requeueResult, err + // Job cancel requested. Stop Flink job. + if desiredJob == nil && job.IsActive() { + log.Info("Stopping job", "jobID", jobID) + newSavepointStatus, err = reconciler.trySuspendJob() + var userControl = getNewControlRequest(observed.cluster) + if userControl == v1beta1.ControlNameJobCancel { + newControlStatus = getControlStatus(userControl, v1beta1.ControlStateInProgress) } - // To proceed to delete step: - // case 1) savepoint triggered: savepointStatus state should be SavepointStateSucceeded and there is no error - // case 2) savepoint skipped: savepointStatus is nil and there is no error - if savepointStatus != nil && savepointStatus.State != v1beta1.SavepointStateSucceeded { - return requeueResult, nil - } - - return ctrl.Result{}, err + return requeueResult, err } - if isJobStopped(recordedJobStatus) { + if job.IsStopped() { log.Info("Job has finished, no action") } @@ -571,12 +562,12 @@ func (reconciler *ClusterReconciler) createJob(job *batchv1.Job) error { var log = reconciler.log var k8sClient = reconciler.k8sClient - log.Info("Submitting job", "resource", *job) + log.Info("Creating job submitter", "resource", *job) var err = k8sClient.Create(context, job) if err != nil { - log.Info("Failed to created job", "error", err) + log.Info("Failed to created job submitter", "error", err) } else { - log.Info("Job created") + log.Info("Job submitter created") } return err } @@ -589,13 +580,13 @@ func (reconciler *ClusterReconciler) deleteJob(job *batchv1.Job) error { var deletePolicy = metav1.DeletePropagationBackground var deleteOption = client.DeleteOptions{PropagationPolicy: &deletePolicy} - log.Info("Deleting job", "job", job) + log.Info("Deleting job submitter", "job", job) var err = k8sClient.Delete(context, job, &deleteOption) err = client.IgnoreNotFound(err) if err != nil { - log.Error(err, "Failed to delete job") + log.Error(err, "Failed to delete job submitter") } else { - log.Info("Job deleted") + log.Info("Job submitter deleted") } return err } @@ -608,50 +599,72 @@ func (reconciler *ClusterReconciler) getFlinkJobID() string { return "" } -func (reconciler *ClusterReconciler) restartJob(shouldTakeSavepoint bool) error { +func (reconciler *ClusterReconciler) trySuspendJob() (*v1beta1.SavepointStatus, error) { var log = reconciler.log - var observedJob = reconciler.observed.job - var observedFlinkJob = reconciler.observed.flinkJobStatus.flinkJob + var recorded = reconciler.observed.cluster.Status + var jobID = reconciler.getFlinkJobID() + + log.Info("Checking the conditions for progressing") + var canSuspend = reconciler.canSuspendJob(jobID, recorded.Savepoint) + if canSuspend { + log.Info("Triggering savepoint for suspending job") + var newSavepointStatus, err = reconciler.triggerSavepoint(jobID, v1beta1.SavepointTriggerReasonUpdate, true) + if err != nil { + log.Info("Failed to trigger savepoint", "jobID", jobID, "triggerID", newSavepointStatus.TriggerID, "error", err) + } else { + log.Info("Successfully savepoint triggered", "jobID", jobID, "triggerID", newSavepointStatus.TriggerID) + } + return newSavepointStatus, err + } + + return nil, nil +} - log.Info("Stopping Flink job to restart", "", observedFlinkJob) - shouldTakeSavepoint = shouldTakeSavepoint && canTakeSavepoint(*reconciler.observed.cluster) +func (reconciler *ClusterReconciler) cancelJob() error { + var log = reconciler.log + var observedFlinkJob = reconciler.observed.flinkJob.status - var err = reconciler.cancelRunningJobs(shouldTakeSavepoint /* takeSavepoint */) + log.Info("Stopping Flink job", "", observedFlinkJob) + var err = reconciler.cancelRunningJobs(false /* takeSavepoint */) if err != nil { + log.Info("Failed to stop Flink job") return err } - if observedJob != nil { - var err = reconciler.deleteJob(observedJob) + // TODO: Not to delete the job submitter immediately, and retain the latest ones for inspection. + var observedSubmitter = reconciler.observed.flinkJobSubmitter.job + if observedSubmitter != nil { + var err = reconciler.deleteJob(observedSubmitter) if err != nil { log.Error( - err, "Failed to delete failed job", "job", observedJob) + err, "Failed to delete job submitter", "job", observedSubmitter) return err } } - // Do not create new job immediately, leave it to the next reconciliation, - // because we still need to be able to create the new job if we encounter - // ephemeral error here. It is better to organize the logic in a central place. - return nil } func (reconciler *ClusterReconciler) cancelUnexpectedJobs( takeSavepoint bool) error { - var unexpectedJobs = reconciler.observed.flinkJobStatus.flinkJobsUnexpected + var unexpectedJobs = reconciler.observed.flinkJob.unexpected return reconciler.cancelJobs(takeSavepoint, unexpectedJobs) } // Cancel running jobs. func (reconciler *ClusterReconciler) cancelRunningJobs( takeSavepoint bool) error { - var runningJobs = reconciler.observed.flinkJobStatus.flinkJobsUnexpected - var flinkJob = reconciler.observed.flinkJobStatus.flinkJob + var log = reconciler.log + var runningJobs = reconciler.observed.flinkJob.unexpected + var flinkJob = reconciler.observed.flinkJob.status if flinkJob != nil && flinkJob.ID != "" && getFlinkJobDeploymentState(flinkJob.Status) == v1beta1.JobStateRunning { runningJobs = append(runningJobs, flinkJob.ID) } + if len(runningJobs) == 0 { + log.Info("No running Flink jobs to stop.") + return nil + } return reconciler.cancelJobs(takeSavepoint, runningJobs) } @@ -675,12 +688,11 @@ func (reconciler *ClusterReconciler) cancelJobs( func (reconciler *ClusterReconciler) cancelFlinkJob(jobID string, takeSavepoint bool) error { var log = reconciler.log if takeSavepoint && canTakeSavepoint(*reconciler.observed.cluster) { + log.Info("Taking savepoint before stopping job", "jobID", jobID) var err = reconciler.takeSavepoint(jobID) if err != nil { return err } - } else { - log.Info("Skip taking savepoint before stopping job", "jobID", jobID) } var apiBaseURL = getFlinkAPIBaseURL(reconciler.observed.cluster) @@ -688,152 +700,114 @@ func (reconciler *ClusterReconciler) cancelFlinkJob(jobID string, takeSavepoint return reconciler.flinkClient.StopJob(apiBaseURL, jobID) } -// Trigger savepoint if it is possible, then return the savepoint status to update. -// When savepoint was already triggered, return the current observed status. -// If triggering savepoint is impossible or skipped or triggered savepoint was created, proceed to stop the job. -func (reconciler *ClusterReconciler) cancelFlinkJobAsync(jobID string, takeSavepoint bool) (*v1beta1.SavepointStatus, error) { +// canSuspendJob +func (reconciler *ClusterReconciler) canSuspendJob(jobID string, s *v1beta1.SavepointStatus) bool { var log = reconciler.log - var cluster = reconciler.observed.cluster - var observedSavepoint = reconciler.observed.cluster.Status.Savepoint - var savepointStatus *v1beta1.SavepointStatus - var err error + var firstTry = !finalSavepointRequested(jobID, s) + if firstTry { + return true + } - switch observedSavepoint.State { - case v1beta1.SavepointStateNotTriggered: - if takeSavepoint && canTakeSavepoint(*reconciler.observed.cluster) { - savepointStatus, err = reconciler.takeSavepointAsync(jobID, v1beta1.SavepointTriggerReasonJobCancel) - if err != nil { - log.Info("Failed to trigger savepoint.") - return savepointStatus, fmt.Errorf("failed to trigger savepoint: %v", err) - } - log.Info("Triggered savepoint and wait it is completed.") - return savepointStatus, nil - } else { - savepointStatus = nil - if takeSavepoint { - log.Info("Savepoint was desired but couldn't be triggered. Skip taking savepoint before stopping job", "jobID", jobID) - } else { - log.Info("Skip taking savepoint before stopping job", "jobID", jobID) - } - } - case v1beta1.SavepointStateInProgress: - log.Info("Triggered savepoint already and wait until it is completed.") - return observedSavepoint, nil + switch s.State { case v1beta1.SavepointStateSucceeded: - savepointStatus = observedSavepoint - log.Info("Successfully savepoint created. Proceed to stop job.") - // Cannot be reached here with these states, because job-cancel control should be finished with failed savepoint states by updater. + log.Info("Successfully savepoint completed, wait until the job stops") + return false + case v1beta1.SavepointStateInProgress: + log.Info("Savepoint is in progress, wait until it is completed") + return false case v1beta1.SavepointStateTriggerFailed: - fallthrough + log.Info("Savepoint trigger failed in previous request") case v1beta1.SavepointStateFailed: - fallthrough - default: - return nil, fmt.Errorf("unexpected savepoint status: %v", *observedSavepoint) + log.Info("Savepoint failed on previous request") } - var apiBaseURL = getFlinkAPIBaseURL(cluster) - log.Info("Stopping job", "jobID", jobID) - err = reconciler.flinkClient.StopJob(apiBaseURL, jobID) - if err != nil { - return savepointStatus, fmt.Errorf("failed to stop job: %v", err) + var retryTimeArrived = hasTimeElapsed(s.UpdateTime, time.Now(), SavepointRetryIntervalSeconds) + if !retryTimeArrived { + log.Info("Wait until next retry time arrived") } - return savepointStatus, nil + return retryTimeArrived } -func (reconciler *ClusterReconciler) shouldTakeSavepoint() (bool, string) { - var log = reconciler.log - var jobSpec = reconciler.observed.cluster.Spec.Job - var jobStatus = reconciler.observed.cluster.Status.Components.Job - var savepointStatus = reconciler.observed.cluster.Status.Savepoint +func (reconciler *ClusterReconciler) shouldTakeSavepoint() string { + var observed = reconciler.observed + var cluster = observed.cluster + var jobSpec = observed.cluster.Spec.Job + var job = observed.cluster.Status.Components.Job + var savepoint = observed.cluster.Status.Savepoint + var newRequestedControl = getNewControlRequest(cluster) if !canTakeSavepoint(*reconciler.observed.cluster) { - return false, "" - } - - // User requested. - - // In the case of which savepoint status is in finished state, - // savepoint trigger by spec.job.savepointGeneration is not possible - // because the field cannot be increased more when savepoint is failed. - // - // Savepoint retry by annotation is possible because the annotations would be cleared - // when the last savepoint was finished and user can attach the annotation again. - - // Savepoint can be triggered in updater for user request, job-cancel and job update - if savepointStatus != nil && savepointStatus.State == v1beta1.SavepointStateNotTriggered { - return true, savepointStatus.TriggerReason + return "" } + // Savepoint trigger priority is user request including update and job stop. + switch { + // TODO: spec.job.cancelRequested will be deprecated + // Should stop job with savepoint by user requested control + case newRequestedControl == v1beta1.ControlNameJobCancel || (jobSpec.CancelRequested != nil && *jobSpec.CancelRequested): + return v1beta1.SavepointTriggerReasonJobCancel + // Take savepoint by user request + case newRequestedControl == v1beta1.ControlNameSavepoint: + fallthrough // TODO: spec.job.savepointGeneration will be deprecated - if jobSpec.SavepointGeneration > jobStatus.SavepointGeneration && - (savepointStatus != nil && savepointStatus.State != v1beta1.SavepointStateFailed && savepointStatus.State != v1beta1.SavepointStateTriggerFailed) { - log.Info( - "Savepoint is requested", - "statusGen", jobStatus.SavepointGeneration, - "specGen", jobSpec.SavepointGeneration) - return true, v1beta1.SavepointTriggerReasonUserRequested - } - - if jobSpec.AutoSavepointSeconds == nil { - return false, "" - } - - var nextOkTriggerTime = getTimeAfterAddedSeconds(jobStatus.LastSavepointTriggerTime, SavepointTimeoutSec) - if time.Now().Before(nextOkTriggerTime) { - return false, "" - } - - // First savepoint. - if len(jobStatus.LastSavepointTime) == 0 { - return true, v1beta1.SavepointTriggerReasonScheduledInitial - } - - // Scheduled, check if next trigger time arrived. - var nextTime = getTimeAfterAddedSeconds(jobStatus.LastSavepointTime, int64(*jobSpec.AutoSavepointSeconds)) - return time.Now().After(nextTime), v1beta1.SavepointTriggerReasonScheduled -} - -// Convert raw time to object and add `addedSeconds` to it, -// getting a time object for the parsed `rawTime` with `addedSeconds` added to it. -func getTimeAfterAddedSeconds(rawTime string, addedSeconds int64) time.Time { - var tc = &TimeConverter{} - var lastTriggerTime = time.Time{} - if len(rawTime) != 0 { - lastTriggerTime = tc.FromString(rawTime) + case jobSpec.SavepointGeneration > job.SavepointGeneration: + // Triggered by savepointGeneration increased. + // When previous savepoint is failed, savepoint trigger by spec.job.savepointGeneration is not possible + // because the field cannot be increased more. + // Note: checkSavepointGeneration in flinkcluster_validate.go + return v1beta1.SavepointTriggerReasonUserRequested + // Scheduled auto savepoint + case jobSpec.AutoSavepointSeconds != nil: + // When previous try was failed, check retry interval. + if savepoint.IsFailed() && savepoint.TriggerReason == v1beta1.SavepointTriggerReasonScheduled { + var nextRetryTime = getTime(savepoint.UpdateTime).Add(SavepointRetryIntervalSeconds * time.Second) + if time.Now().After(nextRetryTime) { + return v1beta1.SavepointTriggerReasonScheduled + } else { + return "" + } + } + // Check if next trigger time arrived. + var compareTime string + if len(job.SavepointTime) == 0 { + compareTime = job.StartTime + } else { + compareTime = job.SavepointTime + } + var nextTime = getTimeAfterAddedSeconds(compareTime, int64(*jobSpec.AutoSavepointSeconds)) + if time.Now().After(nextTime) { + return v1beta1.SavepointTriggerReasonScheduled + } } - return lastTriggerTime.Add(time.Duration(addedSeconds * int64(time.Second))) + return "" } // Trigger savepoint for a job then return savepoint status to update. -func (reconciler *ClusterReconciler) takeSavepointAsync(jobID string, triggerReason string) (*v1beta1.SavepointStatus, error) { +func (reconciler *ClusterReconciler) triggerSavepoint(jobID string, triggerReason string, cancel bool) (*v1beta1.SavepointStatus, error) { var log = reconciler.log var cluster = reconciler.observed.cluster var apiBaseURL = getFlinkAPIBaseURL(reconciler.observed.cluster) var triggerSuccess bool - var triggerID string + var triggerID flinkclient.SavepointTriggerID var message string var err error - log.Info("Trigger savepoint.", "jobID", jobID) - triggerID, err = reconciler.flinkClient.TakeSavepointAsync(apiBaseURL, jobID, *cluster.Spec.Job.SavepointsDir) + log.Info(fmt.Sprintf("Trigger savepoint for %s", triggerReason), "jobID", jobID) + triggerID, err = reconciler.flinkClient.TriggerSavepoint(apiBaseURL, jobID, *cluster.Spec.Job.SavepointsDir, cancel) if err != nil { // limit message size to 1KiB if message = err.Error(); len(message) > 1024 { message = message[:1024] + "..." } triggerSuccess = false - log.Info("Savepoint trigger is failed.", "jobID", jobID, "triggerID", triggerID, "error", err) + log.Info("Failed to trigger savepoint", "jobID", jobID, "triggerID", triggerID, "error", err) } else { triggerSuccess = true - log.Info("Savepoint is triggered successfully.", "jobID", jobID, "triggerID", triggerID) - } - newSavepointStatus := getTriggeredSavepointStatus(jobID, triggerID, triggerReason, message, triggerSuccess) - requestedSavepoint := reconciler.observed.cluster.Status.Savepoint - // When savepoint was requested, maintain the requested time - if requestedSavepoint != nil && requestedSavepoint.State == v1beta1.SavepointStateNotTriggered { - newSavepointStatus.RequestTime = requestedSavepoint.RequestTime + log.Info("Successfully savepoint triggered", "jobID", jobID, "triggerID", triggerID) } - return &newSavepointStatus, err + newSavepointStatus := reconciler.getNewSavepointStatus(triggerID.RequestID, triggerReason, message, triggerSuccess) + + return newSavepointStatus, err } // Takes savepoint for a job then update job status with the info. @@ -857,78 +831,11 @@ func (reconciler *ClusterReconciler) takeSavepoint( log.Info("Failed to take savepoint.", "jobID", jobID) } - statusUpdateErr := reconciler.updateSavepointStatus(status) - if statusUpdateErr != nil { - log.Error( - statusUpdateErr, "Failed to update savepoint status.", "error", statusUpdateErr) - } - return err } -func (reconciler *ClusterReconciler) updateSavepointTriggerTimeStatus() error { - var cluster = v1beta1.FlinkCluster{} - reconciler.observed.cluster.DeepCopyInto(&cluster) - var jobStatus = cluster.Status.Components.Job - setTimestamp(&jobStatus.LastSavepointTriggerTime) - return reconciler.k8sClient.Status().Update(reconciler.context, &cluster) -} - -func (reconciler *ClusterReconciler) updateSavepointStatus( - savepointStatus flinkclient.SavepointStatus) error { - var cluster = v1beta1.FlinkCluster{} - reconciler.observed.cluster.DeepCopyInto(&cluster) - if savepointStatus.IsSuccessful() { - var jobStatus = cluster.Status.Components.Job - jobStatus.SavepointGeneration++ - jobStatus.LastSavepointTriggerID = savepointStatus.TriggerID - jobStatus.SavepointLocation = savepointStatus.Location - setTimestamp(&jobStatus.LastSavepointTime) - setTimestamp(&cluster.Status.LastUpdateTime) - } - // case in which savepointing is triggered by control annotation - var controlStatus = cluster.Status.Control - if controlStatus != nil && controlStatus.Name == v1beta1.ControlNameSavepoint && - controlStatus.State == v1beta1.ControlStateProgressing { - if controlStatus.Details == nil { - controlStatus.Details = make(map[string]string) - } - var retries, err = getRetryCount(controlStatus.Details) - if err == nil { - if savepointStatus.IsFailed() || retries != "1" { - controlStatus.Details[ControlRetries] = retries - } - } else { - reconciler.log.Error(err, "failed to get retries from control status", "control status", controlStatus) - } - controlStatus.Details[ControlSavepointTriggerID] = savepointStatus.TriggerID - controlStatus.Details[ControlJobID] = savepointStatus.JobID - setTimestamp(&controlStatus.UpdateTime) - } - return reconciler.k8sClient.Status().Update(reconciler.context, &cluster) -} - -// If job cancellation is failed, fill the status message with error message. -// Then, the state will be transited to the failed by the updater. -func getFailedCancelStatus(cancelErr error) *v1beta1.FlinkClusterControlStatus { - var state string - var message string - var now string - setTimestamp(&now) - state = v1beta1.ControlStateProgressing - // limit message size to 1KiB - if message = cancelErr.Error(); len(message) > 1024 { - message = message[:1024] + "..." - } - return &v1beta1.FlinkClusterControlStatus{ - Name: v1beta1.ControlNameJobCancel, - State: state, - UpdateTime: now, - Message: message, - } -} - -func (reconciler *ClusterReconciler) updateStatus(ss **v1beta1.SavepointStatus, cs **v1beta1.FlinkClusterControlStatus) { +func (reconciler *ClusterReconciler) updateStatus( + ss **v1beta1.SavepointStatus, cs **v1beta1.FlinkClusterControlStatus) { var log = reconciler.log var savepointStatus = *ss @@ -977,41 +884,73 @@ func (reconciler *ClusterReconciler) updateStatus(ss **v1beta1.SavepointStatus, } } -func (reconciler *ClusterReconciler) updateStatusForNewJob() error { +func (reconciler *ClusterReconciler) updateJobDeployStatus() error { var log = reconciler.log - var newJobStatus *v1beta1.JobStatus - var desiredJob = reconciler.desired.Job - var clusterClone = reconciler.observed.cluster.DeepCopy() + var observedCluster = reconciler.observed.cluster + var desiredJobSubmitter = reconciler.desired.Job var err error - if clusterClone.Status.Components.Job != nil { - newJobStatus = clusterClone.Status.Components.Job - switch previousJobState := newJobStatus.State; previousJobState { - case v1beta1.JobStateFailed: - newJobStatus.RestartCount++ - case v1beta1.JobStateUpdating: - newJobStatus.RestartCount = 0 - } - } else { - newJobStatus = &v1beta1.JobStatus{} - clusterClone.Status.Components.Job = newJobStatus - } - var fromSavepoint = getFromSavepoint(desiredJob.Spec) - newJobStatus.ID = "" - newJobStatus.State = v1beta1.JobStatePending - newJobStatus.FromSavepoint = fromSavepoint - if newJobStatus.SavepointLocation != "" { - // Latest savepoint should be "fromSavepoint" - newJobStatus.SavepointLocation = fromSavepoint - } + var clusterClone = observedCluster.DeepCopy() + var newJob = clusterClone.Status.Components.Job + + // Reset running job information. + newJob.ID = "" + newJob.StartTime = "" + newJob.EndTime = "" + + // Mark as job submitter is deployed. + setTimestamp(&newJob.DeployTime) setTimestamp(&clusterClone.Status.LastUpdateTime) - err = reconciler.k8sClient.Status().Update(reconciler.context, clusterClone) + // Latest savepoint location should be fromSavepoint. + var fromSavepoint = getFromSavepoint(desiredJobSubmitter.Spec) + newJob.FromSavepoint = fromSavepoint + if newJob.SavepointLocation != "" { + newJob.SavepointLocation = fromSavepoint + } + + // Update job status. + err = reconciler.k8sClient.Status().Update(reconciler.context, clusterClone) if err != nil { log.Error( - err, "Failed to update job status for new job submission", "error", err) + err, "Failed to update job status for new job submitter", "error", err) } else { - log.Info("Succeeded to update job status for new job submission.", "job status", newJobStatus) + log.Info("Succeeded to update job status for new job submitter.", "job status", newJob) } return err } + +// getNewSavepointStatus returns newly triggered savepoint status. +func (reconciler *ClusterReconciler) getNewSavepointStatus(triggerID string, triggerReason string, message string, triggerSuccess bool) *v1beta1.SavepointStatus { + var jobID = reconciler.getFlinkJobID() + var savepointState string + var now string + setTimestamp(&now) + + if triggerSuccess { + savepointState = v1beta1.SavepointStateInProgress + } else { + savepointState = v1beta1.SavepointStateTriggerFailed + } + var savepointStatus = &v1beta1.SavepointStatus{ + JobID: jobID, + TriggerID: triggerID, + TriggerReason: triggerReason, + TriggerTime: now, + UpdateTime: now, + Message: message, + State: savepointState, + } + return savepointStatus +} + +// Convert raw time to object and add `addedSeconds` to it, +// getting a time object for the parsed `rawTime` with `addedSeconds` added to it. +func getTimeAfterAddedSeconds(rawTime string, addedSeconds int64) time.Time { + var tc = &TimeConverter{} + var lastTriggerTime = time.Time{} + if len(rawTime) != 0 { + lastTriggerTime = tc.FromString(rawTime) + } + return lastTriggerTime.Add(time.Duration(addedSeconds * int64(time.Second))) +} diff --git a/controllers/flinkcluster_submit_job_script.go b/controllers/flinkcluster_submit_job_script.go index 59ee9780..466d2c5c 100644 --- a/controllers/flinkcluster_submit_job_script.go +++ b/controllers/flinkcluster_submit_job_script.go @@ -52,7 +52,7 @@ function check_jm_ready() { # Waiting for 5 mins. local -r MAX_RETRY=60 local -r RETRY_INTERVAL=5s - local -r REQUIRED_SUCCESS_NUMBER=2 + local -r REQUIRED_SUCCESS_NUMBER=3 local success_count=0 echo_log "Checking job manager to be ready. Will check success of ${REQUIRED_SUCCESS_NUMBER} API calls for stable job submission." "job_check_log" diff --git a/controllers/flinkcluster_updater.go b/controllers/flinkcluster_updater.go index 49706f7a..18aedba3 100644 --- a/controllers/flinkcluster_updater.go +++ b/controllers/flinkcluster_updater.go @@ -22,7 +22,6 @@ package controllers import ( "context" "encoding/json" - "errors" "fmt" "k8s.io/apimachinery/pkg/types" "reflect" @@ -185,6 +184,7 @@ func (updater *ClusterStatusUpdater) createStatusChangeEvent( } } +// TODO: Need to organize func (updater *ClusterStatusUpdater) deriveClusterStatus( recorded *v1beta1.FlinkClusterStatus, observed *ObservedClusterState) v1beta1.FlinkClusterStatus { @@ -192,13 +192,10 @@ func (updater *ClusterStatusUpdater) deriveClusterStatus( var runningComponents = 0 // jmStatefulSet, jmService, tmStatefulSet. var totalComponents = 3 - var updateState = getUpdateState(*observed) - var isClusterUpdating = !isClusterUpdateToDate(*observed) && updateState == UpdateStateInProgress - var isJobUpdating = recorded.Components.Job != nil && recorded.Components.Job.State == v1beta1.JobStateUpdating // ConfigMap. var observedConfigMap = observed.configMap - if !isComponentUpdated(observedConfigMap, *observed.cluster) && isJobUpdating { + if !isComponentUpdated(observedConfigMap, observed.cluster) && shouldUpdateCluster(observed) { recorded.Components.ConfigMap.DeepCopyInto(&status.Components.ConfigMap) status.Components.ConfigMap.State = v1beta1.ComponentStateUpdating } else if observedConfigMap != nil { @@ -214,7 +211,7 @@ func (updater *ClusterStatusUpdater) deriveClusterStatus( // JobManager StatefulSet. var observedJmStatefulSet = observed.jmStatefulSet - if !isComponentUpdated(observedJmStatefulSet, *observed.cluster) && isJobUpdating { + if !isComponentUpdated(observedJmStatefulSet, observed.cluster) && shouldUpdateCluster(observed) { recorded.Components.JobManagerStatefulSet.DeepCopyInto(&status.Components.JobManagerStatefulSet) status.Components.JobManagerStatefulSet.State = v1beta1.ComponentStateUpdating } else if observedJmStatefulSet != nil { @@ -233,7 +230,7 @@ func (updater *ClusterStatusUpdater) deriveClusterStatus( // JobManager service. var observedJmService = observed.jmService - if !isComponentUpdated(observedJmService, *observed.cluster) && isJobUpdating { + if !isComponentUpdated(observedJmService, observed.cluster) && shouldUpdateCluster(observed) { recorded.Components.JobManagerService.DeepCopyInto(&status.Components.JobManagerService) status.Components.JobManagerService.State = v1beta1.ComponentStateUpdating } else if observedJmService != nil { @@ -284,7 +281,7 @@ func (updater *ClusterStatusUpdater) deriveClusterStatus( // (Optional) JobManager ingress. var observedJmIngress = observed.jmIngress - if !isComponentUpdated(observedJmIngress, *observed.cluster) && isJobUpdating { + if !isComponentUpdated(observedJmIngress, observed.cluster) && shouldUpdateCluster(observed) { status.Components.JobManagerIngress = &v1beta1.JobManagerIngressStatus{} recorded.Components.JobManagerIngress.DeepCopyInto(status.Components.JobManagerIngress) status.Components.JobManagerIngress.State = v1beta1.ComponentStateUpdating @@ -367,7 +364,7 @@ func (updater *ClusterStatusUpdater) deriveClusterStatus( // TaskManager StatefulSet. var observedTmStatefulSet = observed.tmStatefulSet - if !isComponentUpdated(observedTmStatefulSet, *observed.cluster) && isJobUpdating { + if !isComponentUpdated(observedTmStatefulSet, observed.cluster) && shouldUpdateCluster(observed) { recorded.Components.TaskManagerStatefulSet.DeepCopyInto(&status.Components.TaskManagerStatefulSet) status.Components.TaskManagerStatefulSet.State = v1beta1.ComponentStateUpdating } else if observedTmStatefulSet != nil { @@ -387,18 +384,6 @@ func (updater *ClusterStatusUpdater) deriveClusterStatus( } } - // (Optional) Job. - var jobStopped = false - var jobStatus = updater.getJobStatus() - status.Components.Job = jobStatus - if jobStatus != nil && - (jobStatus.State == v1beta1.JobStateSucceeded || - jobStatus.State == v1beta1.JobStateFailed || - jobStatus.State == v1beta1.JobStateCancelled || - jobStatus.State == v1beta1.JobStateSuspended) { - jobStopped = true - } - // Derive the new cluster state. switch recorded.State { case "", v1beta1.ClusterStateCreating: @@ -408,10 +393,10 @@ func (updater *ClusterStatusUpdater) deriveClusterStatus( status.State = v1beta1.ClusterStateRunning } case v1beta1.ClusterStateUpdating: - if isClusterUpdating { + if shouldUpdateCluster(observed) { status.State = v1beta1.ClusterStateUpdating } else if runningComponents < totalComponents { - if isUpdateTriggered(*recorded) { + if recorded.Revision.IsUpdateTriggered() { status.State = v1beta1.ClusterStateUpdating } else { status.State = v1beta1.ClusterStateReconciling @@ -421,9 +406,10 @@ func (updater *ClusterStatusUpdater) deriveClusterStatus( } case v1beta1.ClusterStateRunning, v1beta1.ClusterStateReconciling: - if isClusterUpdating { + var jobStatus = recorded.Components.Job + if shouldUpdateCluster(observed) { status.State = v1beta1.ClusterStateUpdating - } else if jobStopped { + } else if !recorded.Revision.IsUpdateTriggered() && jobStatus.IsStopped() { var policy = observed.cluster.Spec.Job.CleanupPolicy if jobStatus.State == v1beta1.JobStateSucceeded && policy.AfterJobSucceeds != v1beta1.CleanupActionKeepCluster { @@ -444,7 +430,7 @@ func (updater *ClusterStatusUpdater) deriveClusterStatus( } case v1beta1.ClusterStateStopping, v1beta1.ClusterStatePartiallyStopped: - if isClusterUpdating { + if shouldUpdateCluster(observed) { status.State = v1beta1.ClusterStateUpdating } else if runningComponents == 0 { status.State = v1beta1.ClusterStateStopped @@ -454,7 +440,7 @@ func (updater *ClusterStatusUpdater) deriveClusterStatus( status.State = v1beta1.ClusterStateStopping } case v1beta1.ClusterStateStopped: - if isUpdateTriggered(*recorded) { + if recorded.Revision.IsUpdateTriggered() { status.State = v1beta1.ClusterStateUpdating } else { status.State = v1beta1.ClusterStateStopped @@ -463,202 +449,33 @@ func (updater *ClusterStatusUpdater) deriveClusterStatus( panic(fmt.Sprintf("Unknown cluster state: %v", recorded.State)) } - // Savepoint status - // update savepoint status if it is in progress - if recorded.Savepoint != nil { - var newSavepointStatus = recorded.Savepoint.DeepCopy() - if recorded.Savepoint.State == v1beta1.SavepointStateInProgress && observed.savepoint != nil { - switch { - case observed.savepoint.IsSuccessful(): - newSavepointStatus.State = v1beta1.SavepointStateSucceeded - case observed.savepoint.IsFailed(): - var msg string - newSavepointStatus.State = v1beta1.SavepointStateFailed - if observed.savepoint.FailureCause.StackTrace != "" { - msg = fmt.Sprintf("Savepoint error: %v", observed.savepoint.FailureCause.StackTrace) - } else if observed.savepointErr != nil { - msg = fmt.Sprintf("Failed to get triggered savepoint status: %v", observed.savepointErr) - } else { - msg = "Failed to get triggered savepoint status" - } - if len(msg) > 1024 { - msg = msg[:1024] + "..." - } - newSavepointStatus.Message = msg - // TODO: organize more making savepoint status - if newSavepointStatus.TriggerReason == v1beta1.SavepointTriggerReasonUpdate { - newSavepointStatus.Message = - "Failed to take savepoint for update. " + - "The update process is being postponed until a savepoint is available. " + newSavepointStatus.Message - } - } - } - if newSavepointStatus.State == v1beta1.SavepointStateNotTriggered || newSavepointStatus.State == v1beta1.SavepointStateInProgress { - var flinkJobID = updater.getFlinkJobID() - switch { - case savepointTimeout(newSavepointStatus): - newSavepointStatus.State = v1beta1.SavepointStateFailed - newSavepointStatus.Message = "Timed out taking savepoint." - case isJobStopped(recorded.Components.Job): - newSavepointStatus.Message = "Flink job is stopped." - newSavepointStatus.State = v1beta1.SavepointStateFailed - case !isFlinkAPIReady(*observed): - newSavepointStatus.Message = "Flink API is not available." - newSavepointStatus.State = v1beta1.SavepointStateFailed - case flinkJobID == nil: - newSavepointStatus.Message = "Flink job is not submitted or identified." - newSavepointStatus.State = v1beta1.SavepointStateFailed - case flinkJobID != nil && (recorded.Savepoint.TriggerID != "" && *flinkJobID != recorded.Savepoint.JobID): - newSavepointStatus.Message = "Savepoint triggered Flink job is lost." - newSavepointStatus.State = v1beta1.SavepointStateFailed - } - // TODO: organize more making savepoint status - if newSavepointStatus.State == v1beta1.SavepointStateFailed && - newSavepointStatus.TriggerReason == v1beta1.SavepointTriggerReasonUpdate { - newSavepointStatus.Message = - "Failed to take savepoint for update. " + - "The update process is being postponed until a savepoint is available. " + newSavepointStatus.Message - } - } - status.Savepoint = newSavepointStatus - } - - // User requested control - var userControl = observed.cluster.Annotations[v1beta1.ControlAnnotation] - - // Update job control status in progress - var controlStatus *v1beta1.FlinkClusterControlStatus - if recorded.Control != nil && userControl == recorded.Control.Name && - recorded.Control.State == v1beta1.ControlStateProgressing { - controlStatus = recorded.Control.DeepCopy() - var savepointStatus = status.Savepoint - switch recorded.Control.Name { - case v1beta1.ControlNameJobCancel: - if status.Components.Job.State == v1beta1.JobStateCancelled { - controlStatus.State = v1beta1.ControlStateSucceeded - setTimestamp(&controlStatus.UpdateTime) - } else if isJobTerminated(observed.cluster.Spec.Job.RestartPolicy, recorded.Components.Job) { - controlStatus.Message = "Aborted job cancellation: Job is terminated." - controlStatus.State = v1beta1.ControlStateFailed - setTimestamp(&controlStatus.UpdateTime) - } else if savepointStatus != nil && savepointStatus.State == v1beta1.SavepointStateFailed { - controlStatus.Message = "Aborted job cancellation: failed to create savepoint." - controlStatus.State = v1beta1.ControlStateFailed - setTimestamp(&controlStatus.UpdateTime) - } else if recorded.Control.Message != "" { - controlStatus.State = v1beta1.ControlStateFailed - setTimestamp(&controlStatus.UpdateTime) - } - case v1beta1.ControlNameSavepoint: - if savepointStatus != nil { - if savepointStatus.State == v1beta1.SavepointStateSucceeded { - controlStatus.State = v1beta1.ControlStateSucceeded - setTimestamp(&controlStatus.UpdateTime) - } else if savepointStatus.State == v1beta1.SavepointStateFailed || savepointStatus.State == v1beta1.SavepointStateTriggerFailed { - controlStatus.State = v1beta1.ControlStateFailed - setTimestamp(&controlStatus.UpdateTime) - } - } - } - // aborted by max retry reach - var retries = controlStatus.Details[ControlRetries] - if retries == ControlMaxRetries { - controlStatus.Message = fmt.Sprintf("Aborted control %v. The maximum number of retries has been reached.", controlStatus.Name) - controlStatus.State = v1beta1.ControlStateFailed - setTimestamp(&controlStatus.UpdateTime) - } - } else if userControl != "" { - // Handle new user control. - updater.log.Info("New user control requested: " + userControl) - if userControl != v1beta1.ControlNameJobCancel && userControl != v1beta1.ControlNameSavepoint { - if userControl != "" { - updater.log.Info(fmt.Sprintf(v1beta1.InvalidControlAnnMsg, v1beta1.ControlAnnotation, userControl)) - } - } else if recorded.Control != nil && recorded.Control.State == v1beta1.ControlStateProgressing { - updater.log.Info(fmt.Sprintf(v1beta1.ControlChangeWarnMsg, v1beta1.ControlAnnotation), "current control", recorded.Control.Name, "new control", userControl) - } else { - switch userControl { - case v1beta1.ControlNameSavepoint: - if observed.cluster.Spec.Job.SavepointsDir == nil || *observed.cluster.Spec.Job.SavepointsDir == "" { - updater.log.Info(fmt.Sprintf(v1beta1.InvalidSavepointDirMsg, v1beta1.ControlAnnotation)) - break - } else if isJobStopped(observed.cluster.Status.Components.Job) { - updater.log.Info(fmt.Sprintf(v1beta1.InvalidJobStateForSavepointMsg, v1beta1.ControlAnnotation)) - break - } - // Clear status for new savepoint - status.Savepoint = getRequestedSavepointStatus(v1beta1.SavepointTriggerReasonUserRequested) - controlStatus = getNewUserControlStatus(userControl) - case v1beta1.ControlNameJobCancel: - if isJobTerminated(observed.cluster.Spec.Job.RestartPolicy, recorded.Components.Job) { - updater.log.Info(fmt.Sprintf(v1beta1.InvalidJobStateForJobCancelMsg, v1beta1.ControlAnnotation)) - break - } - // Savepoint for job-cancel - var observedSavepoint = observed.cluster.Status.Savepoint - if observedSavepoint == nil || - (observedSavepoint.State != v1beta1.SavepointStateInProgress && observedSavepoint.State != v1beta1.SavepointStateNotTriggered) { - updater.log.Info("There is no savepoint in progress. Trigger savepoint in reconciler.") - status.Savepoint = getRequestedSavepointStatus(v1beta1.SavepointTriggerReasonJobCancel) - } else { - updater.log.Info("There is a savepoint in progress. Skip new savepoint.") - } - controlStatus = getNewUserControlStatus(userControl) - } - } - } - // Maintain control status if there is no change. - if recorded.Control != nil && controlStatus == nil { - controlStatus = recorded.Control.DeepCopy() - } - status.Control = controlStatus - - // Handle update. - var savepointForJobUpdate *v1beta1.SavepointStatus - switch updateState { - case UpdateStatePreparing: - // Even if savepoint has been created for update already, we check the age of savepoint continually. - // If created savepoint is old and savepoint can be triggered, we should take savepoint again. - // (e.g., for the case update is not progressed by accidents like network partition) - if !isSavepointUpToDate(observed.observeTime, *jobStatus) && - canTakeSavepoint(*observed.cluster) && - (recorded.Savepoint == nil || recorded.Savepoint.State != v1beta1.SavepointStateNotTriggered) { - // If failed to take savepoint, retry after SavepointRequestRetryIntervalSec. - if recorded.Savepoint != nil && - !hasTimeElapsed(recorded.Savepoint.RequestTime, time.Now(), SavepointRequestRetryIntervalSec) { - updater.log.Info(fmt.Sprintf("Will retry to trigger savepoint for update, in %v seconds because previous request was failed", SavepointRequestRetryIntervalSec)) - } else { - status.Savepoint = getRequestedSavepointStatus(v1beta1.SavepointTriggerReasonUpdate) - updater.log.Info("Savepoint will be triggered for update") - } - } else if recorded.Savepoint != nil && recorded.Savepoint.State == v1beta1.SavepointStateInProgress { - updater.log.Info("Savepoint for update is in progress") - } else { - updater.log.Info("Stopping job for update") - } - case UpdateStateInProgress: - updater.log.Info("Updating cluster") - case UpdateStateFinished: - status.CurrentRevision = observed.cluster.Status.NextRevision - updater.log.Info("Finished update") - } - if savepointForJobUpdate != nil { - status.Savepoint = savepointForJobUpdate - } - - // Update revision status - status.NextRevision = getRevisionWithNameNumber(observed.revisionStatus.nextRevision) - if status.CurrentRevision == "" { - if recorded.CurrentRevision == "" { - status.CurrentRevision = getRevisionWithNameNumber(observed.revisionStatus.currentRevision) - } else { - status.CurrentRevision = recorded.CurrentRevision - } - } - if observed.revisionStatus.collisionCount != 0 { - status.CollisionCount = new(int32) - *status.CollisionCount = observed.revisionStatus.collisionCount - } + // (Optional) Job. + // Update job status. + status.Components.Job = updater.deriveJobStatus() + + // (Optional) Savepoint. + // Update savepoint status if it is in progress or requested. + var newJobStatus = status.Components.Job + status.Savepoint = updater.deriveSavepointStatus( + &observed.savepoint, + recorded.Savepoint, + newJobStatus, + updater.getFlinkJobID()) + + // (Optional) Control. + // Update user requested control status. + status.Control = deriveControlStatus( + observed.cluster, + status.Savepoint, + status.Components.Job, + recorded.Control) + + // Update revision status. + // When update completed, finish the process by marking CurrentRevision to NextRevision. + status.Revision = deriveRevisionStatus( + observed.updateState, + &observed.revision, + &recorded.Revision) return status } @@ -669,16 +486,16 @@ func (updater *ClusterStatusUpdater) deriveClusterStatus( // to transient error or being skiped as an optimization. // If this returned nil, it is the state that job is not submitted or not identified yet. func (updater *ClusterStatusUpdater) getFlinkJobID() *string { - // Observed from active job manager - var observedFlinkJob = updater.observed.flinkJobStatus.flinkJob + // Observed from active job manager. + var observedFlinkJob = updater.observed.flinkJob.status if observedFlinkJob != nil && len(observedFlinkJob.ID) > 0 { return &observedFlinkJob.ID } - // Observed from job submitter (when job manager is not ready yet) - var observedJobSubmitLog = updater.observed.flinkJobSubmitLog - if observedJobSubmitLog != nil && observedJobSubmitLog.JobID != "" { - return &observedJobSubmitLog.JobID + // Observed from job submitter (when Flink API is not ready). + var observedJobSubmitterLog = updater.observed.flinkJobSubmitter.log + if observedJobSubmitterLog != nil && observedJobSubmitterLog.JobID != "" { + return &observedJobSubmitterLog.JobID } // Recorded. @@ -690,88 +507,124 @@ func (updater *ClusterStatusUpdater) getFlinkJobID() *string { return nil } -func (updater *ClusterStatusUpdater) getJobStatus() *v1beta1.JobStatus { +func (updater *ClusterStatusUpdater) deriveJobStatus() *v1beta1.JobStatus { var observed = updater.observed - var observedJob = updater.observed.job - var observedFlinkJob = updater.observed.flinkJobStatus.flinkJob - var observedCluster = updater.observed.cluster - var observedSavepoint = updater.observed.savepoint - var recordedJobStatus = updater.observed.cluster.Status.Components.Job - var newJobStatus *v1beta1.JobStatus - - if recordedJobStatus == nil { + var observedCluster = observed.cluster + var jobSpec = observedCluster.Spec.Job + if jobSpec == nil { return nil } - newJobStatus = recordedJobStatus.DeepCopy() - // Determine job state - var jobState string + var observedSubmitter = observed.flinkJobSubmitter + var observedFlinkJob = observed.flinkJob.status + var observedSavepoint = observed.savepoint + var recorded = observedCluster.Status + var savepoint = recorded.Savepoint + var oldJob = recorded.Components.Job + var newJob *v1beta1.JobStatus + + // Derive new job state. + if oldJob != nil { + newJob = oldJob.DeepCopy() + } else { + newJob = new(v1beta1.JobStatus) + } + var newJobState string + var newJobID string switch { - // Updating state - case isUpdateTriggered(observedCluster.Status) && - (isJobStopped(recordedJobStatus) || observedCluster.Status.State == v1beta1.ClusterStateStopped): - jobState = v1beta1.JobStateUpdating - // Already terminated state - case isJobTerminated(observedCluster.Spec.Job.RestartPolicy, recordedJobStatus): - jobState = recordedJobStatus.State - // Derive state from the observed Flink job + case oldJob == nil: + newJobState = v1beta1.JobStatePending + case shouldUpdateJob(&observed): + newJobState = v1beta1.JobStateUpdating + case oldJob.ShouldRestart(jobSpec): + newJobState = v1beta1.JobStateRestarting + case oldJob.IsPending() && oldJob.DeployTime != "": + newJobState = v1beta1.JobStateDeploying + case oldJob.IsStopped(): + newJobState = oldJob.State + // Derive the job state from the observed Flink job, if it exists. case observedFlinkJob != nil: - jobState = getFlinkJobDeploymentState(observedFlinkJob.Status) - if jobState == "" { - updater.log.Error(errors.New("failed to determine Flink job deployment state"), "observed flink job status", observedFlinkJob.Status) - jobState = recordedJobStatus.State - } - // When Flink job not found - case isFlinkAPIReady(observed): - switch recordedJobStatus.State { - case v1beta1.JobStateRunning: - jobState = v1beta1.JobStateLost - case v1beta1.JobStatePending: - // Flink job is submitted but not confirmed via job manager yet - var jobSubmitSucceeded = updater.getFlinkJobID() != nil - // Flink job submit is in progress - var jobSubmitInProgress = observedJob != nil && - observedJob.Status.Succeeded == 0 && - observedJob.Status.Failed == 0 - if jobSubmitSucceeded || jobSubmitInProgress { - jobState = v1beta1.JobStatePending - break - } - jobState = v1beta1.JobStateFailed - default: - jobState = recordedJobStatus.State + newJobState = getFlinkJobDeploymentState(observedFlinkJob.Status) + // Unexpected Flink job state + if newJobState == "" { + panic(fmt.Sprintf("Unknown Flink job status: %s", observedFlinkJob.Status)) } - // When Flink API unavailable + newJobID = observedFlinkJob.ID + // When Flink job not found in JobManager or JobManager is unavailable + case isFlinkAPIReady(observed.flinkJob.list): + if oldJob.State == v1beta1.JobStateRunning { + newJobState = v1beta1.JobStateLost + break + } + fallthrough default: - if recordedJobStatus.State == v1beta1.JobStatePending { - var jobSubmitFailed = observedJob != nil && observedJob.Status.Failed > 0 - if jobSubmitFailed { - jobState = v1beta1.JobStateFailed - break + if oldJob.State != v1beta1.JobStateDeploying { + newJobState = oldJob.State + break + } + // Job submitter is deployed but tracking failed. + var submitterState = observedSubmitter.getState() + if submitterState == JobDeployStateUnknown { + newJobState = v1beta1.JobStateLost + break + } + // Case in which the job submission clearly fails even if it is not confirmed by JobManager + // Job submitter is deployed but failed. + if submitterState == JobDeployStateFailed { + newJobState = v1beta1.JobStateDeployFailed + break + } + newJobState = oldJob.State + } + newJob.State = newJobState + if newJobID != "" { + newJob.ID = newJobID + } + + // Derived new job status if the state is changed. + if oldJob == nil || oldJob.State != newJob.State { + // TODO: It would be ideal to set the times with the timestamp retrieved from the Flink API like /jobs/{job-id}. + switch { + case newJob.IsPending(): + newJob.DeployTime = "" + if newJob.State == v1beta1.JobStateUpdating { + newJob.RestartCount = 0 + } else if newJob.State == v1beta1.JobStateRestarting { + newJob.RestartCount++ + } + case newJob.State == v1beta1.JobStateRunning: + setTimestamp(&newJob.StartTime) + newJob.EndTime = "" + // When job started, the savepoint is not the final state of the job any more. + if oldJob.FinalSavepoint { + newJob.FinalSavepoint = false + } + case newJob.IsStopped(): + if newJob.EndTime == "" { + setTimestamp(&newJob.EndTime) + } + // When tracking failed, we cannot guarantee if the savepoint is the final job state. + if newJob.State == v1beta1.JobStateLost && oldJob.FinalSavepoint { + newJob.FinalSavepoint = false } } - jobState = recordedJobStatus.State } - // Flink Job ID - if jobState == v1beta1.JobStateUpdating { - newJobStatus.ID = "" - } else if observedFlinkJob != nil { - newJobStatus.ID = observedFlinkJob.ID - } - - // State - newJobStatus.State = jobState - // Savepoint - if newJobStatus != nil && observedSavepoint != nil && observedSavepoint.IsSuccessful() { - newJobStatus.SavepointGeneration++ - newJobStatus.LastSavepointTriggerID = observedSavepoint.TriggerID - newJobStatus.SavepointLocation = observedSavepoint.Location - setTimestamp(&newJobStatus.LastSavepointTime) + if observedSavepoint.status != nil && observedSavepoint.status.IsSuccessful() { + newJob.SavepointGeneration++ + newJob.SavepointLocation = observedSavepoint.status.Location + if finalSavepointRequested(newJob.ID, savepoint) { + newJob.FinalSavepoint = true + } + // TODO: SavepointTime should be set with the timestamp generated in job manager. + // Currently savepoint complete timestamp is not included in savepoints API response. + // Whereas checkpoint API returns the timestamp latest_ack_timestamp. + // Note: https://ci.apache.org/projects/flink/flink-docs-stable/ops/rest_api.html#jobs-jobid-checkpoints-details-checkpointid + setTimestamp(&newJob.SavepointTime) } - return newJobStatus + return newJob } func (updater *ClusterStatusUpdater) isStatusChanged( @@ -888,15 +741,17 @@ func (updater *ClusterStatusUpdater) isStatusChanged( newStatus.Savepoint) changed = true } - if newStatus.CurrentRevision != currentStatus.CurrentRevision || - newStatus.NextRevision != currentStatus.NextRevision || - (newStatus.CollisionCount != nil && currentStatus.CollisionCount == nil) || - (currentStatus.CollisionCount != nil && *newStatus.CollisionCount != *currentStatus.CollisionCount) { + var nr = newStatus.Revision // New revision status + var cr = currentStatus.Revision // Current revision status + if nr.CurrentRevision != cr.CurrentRevision || + nr.NextRevision != cr.NextRevision || + (nr.CollisionCount != nil && cr.CollisionCount == nil) || + (cr.CollisionCount != nil && *nr.CollisionCount != *cr.CollisionCount) { updater.log.Info( "FlinkCluster revision status changed", "current", - fmt.Sprintf("currentRevision: %v, nextRevision: %v, collisionCount: %v", currentStatus.CurrentRevision, currentStatus.NextRevision, currentStatus.CollisionCount), + fmt.Sprintf("currentRevision: %v, nextRevision: %v, collisionCount: %v", cr.CurrentRevision, cr.NextRevision, cr.CollisionCount), "new", - fmt.Sprintf("currentRevision: %v, nextRevision: %v, collisionCount: %v", newStatus.CurrentRevision, newStatus.NextRevision, newStatus.CollisionCount)) + fmt.Sprintf("currentRevision: %v, nextRevision: %v, collisionCount: %v", nr.CurrentRevision, nr.NextRevision, nr.CollisionCount)) changed = true } return changed @@ -939,6 +794,147 @@ func (updater *ClusterStatusUpdater) clearControlAnnotation(newControlStatus *v1 return nil } +func (updater *ClusterStatusUpdater) deriveSavepointStatus( + observedSavepoint *Savepoint, + recordedSavepointStatus *v1beta1.SavepointStatus, + newJobStatus *v1beta1.JobStatus, + flinkJobID *string) *v1beta1.SavepointStatus { + if recordedSavepointStatus == nil { + return nil + } + + // Derived savepoint status to return + var s = recordedSavepointStatus.DeepCopy() + var errMsg string + + // Update the savepoint status when observed savepoint is found. + if s.State == v1beta1.SavepointStateInProgress && observedSavepoint != nil { + switch { + case observedSavepoint.status.IsSuccessful(): + s.State = v1beta1.SavepointStateSucceeded + case observedSavepoint.status.IsFailed(): + s.State = v1beta1.SavepointStateFailed + errMsg = fmt.Sprintf("Savepoint error: %v", observedSavepoint.status.FailureCause.StackTrace) + case observedSavepoint.error != nil: + s.State = v1beta1.SavepointStateFailed + errMsg = fmt.Sprintf("Failed to get savepoint status: %v", observedSavepoint.error) + } + } + + // Check failure conditions of savepoint in progress. + if s.State == v1beta1.SavepointStateInProgress { + switch { + case newJobStatus.IsStopped(): + errMsg = "Flink job is stopped." + s.State = v1beta1.SavepointStateFailed + case flinkJobID == nil: + errMsg = "Flink job is not identified." + s.State = v1beta1.SavepointStateFailed + case flinkJobID != nil && (recordedSavepointStatus.TriggerID != "" && *flinkJobID != recordedSavepointStatus.JobID): + errMsg = "Savepoint triggered Flink job is lost." + s.State = v1beta1.SavepointStateFailed + } + } + // TODO: Record event or introduce Condition in CRD status to notify update state pended. + // https://github.com/kubernetes/apimachinery/blob/57f2a0733447cfd41294477d833cce6580faaca3/pkg/apis/meta/v1/types.go#L1376 + // Make up message. + if errMsg != "" { + if s.TriggerReason == v1beta1.SavepointTriggerReasonUpdate { + errMsg = + "Failed to take savepoint for update. " + + "The update process is being postponed until a savepoint is available. " + errMsg + } + if len(errMsg) > 1024 { + errMsg = errMsg[:1024] + } + s.Message = errMsg + } + + return s +} + +func deriveControlStatus( + cluster *v1beta1.FlinkCluster, + newSavepoint *v1beta1.SavepointStatus, + newJob *v1beta1.JobStatus, + recordedControl *v1beta1.FlinkClusterControlStatus) *v1beta1.FlinkClusterControlStatus { + var controlRequest = getNewControlRequest(cluster) + + // Derived control status to return + var c *v1beta1.FlinkClusterControlStatus + + // New control status + if controlRequest != "" { + c = getControlStatus(controlRequest, v1beta1.ControlStateRequested) + return c + } + + // Update control status in progress. + if recordedControl != nil && recordedControl.State == v1beta1.ControlStateInProgress { + c = recordedControl.DeepCopy() + switch recordedControl.Name { + case v1beta1.ControlNameJobCancel: + if newSavepoint.State == v1beta1.SavepointStateSucceeded && newJob.State == v1beta1.JobStateCancelled { + c.State = v1beta1.ControlStateSucceeded + } else if newJob.IsStopped() { + c.Message = "Aborted job cancellation: savepoint is not completed, but job is stopped already." + c.State = v1beta1.ControlStateFailed + } else if newSavepoint.IsFailed() && newSavepoint.TriggerReason == v1beta1.SavepointTriggerReasonJobCancel { + c.Message = "Aborted job cancellation: failed to take savepoint." + c.State = v1beta1.ControlStateFailed + } + case v1beta1.ControlNameSavepoint: + if newSavepoint.State == v1beta1.SavepointStateSucceeded { + c.State = v1beta1.ControlStateSucceeded + } else if newSavepoint.IsFailed() && newSavepoint.TriggerReason == v1beta1.SavepointTriggerReasonUserRequested { + c.State = v1beta1.ControlStateFailed + } + } + // Update time when state changed. + if c.State != v1beta1.ControlStateInProgress { + setTimestamp(&c.UpdateTime) + } + return c + } + // Maintain control status if there is no change. + if recordedControl != nil && c == nil { + c = recordedControl.DeepCopy() + return c + } + + return nil +} + +func deriveRevisionStatus( + updateState UpdateState, + observedRevision *Revision, + recordedRevision *v1beta1.RevisionStatus, +) v1beta1.RevisionStatus { + // Derived revision status + var r = v1beta1.RevisionStatus{} + + // Finalize update process. + if updateState == UpdateStateFinished { + r.CurrentRevision = recordedRevision.NextRevision + } + + // Update revision status. + r.NextRevision = getRevisionWithNameNumber(observedRevision.nextRevision) + if r.CurrentRevision == "" { + if recordedRevision.CurrentRevision == "" { + r.CurrentRevision = getRevisionWithNameNumber(observedRevision.currentRevision) + } else { + r.CurrentRevision = recordedRevision.CurrentRevision + } + } + if observedRevision.collisionCount != 0 { + r.CollisionCount = new(int32) + *r.CollisionCount = observedRevision.collisionCount + } + + return r +} + func getStatefulSetState(statefulSet *appsv1.StatefulSet) string { if statefulSet.Status.ReadyReplicas >= *statefulSet.Spec.Replicas { return v1beta1.ComponentStateReady diff --git a/controllers/flinkcluster_util.go b/controllers/flinkcluster_util.go index 28607aef..bb140084 100644 --- a/controllers/flinkcluster_util.go +++ b/controllers/flinkcluster_util.go @@ -20,9 +20,9 @@ import ( "bytes" "encoding/json" "fmt" - v1beta1 "github.com/googlecloudplatform/flink-operator/api/v1beta1" + "github.com/googlecloudplatform/flink-operator/api/v1beta1" + "github.com/googlecloudplatform/flink-operator/controllers/flinkclient" "github.com/googlecloudplatform/flink-operator/controllers/history" - "gopkg.in/yaml.v2" appsv1 "k8s.io/api/apps/v1" batchv1 "k8s.io/api/batch/v1" corev1 "k8s.io/api/core/v1" @@ -37,26 +37,26 @@ import ( ) const ( - ControlSavepointTriggerID = "SavepointTriggerID" - ControlJobID = "jobID" - ControlRetries = "retries" - ControlMaxRetries = "3" - - SavepointTimeoutSec = 900 // 15 mins + ControlRetries = "retries" + ControlMaxRetries = "3" RevisionNameLabel = "flinkoperator.k8s.io/revision-name" - // TODO: need to be user configurable - SavepointAgeForJobUpdateSec = 300 - SavepointRequestRetryIntervalSec = 10 + SavepointRetryIntervalSeconds = 10 ) type UpdateState string +type JobSubmitState string const ( UpdateStatePreparing UpdateState = "Preparing" UpdateStateInProgress UpdateState = "InProgress" UpdateStateFinished UpdateState = "Finished" + + JobDeployStateInProgress = "InProgress" + JobDeployStateSucceeded = "Succeeded" + JobDeployStateFailed = "Failed" + JobDeployStateUnknown = "Unknown" ) type objectForPatch struct { @@ -137,31 +137,30 @@ func setTimestamp(target *string) { *target = tc.ToString(now) } +func getTime(timeStr string) time.Time { + var tc TimeConverter + return tc.FromString(timeStr) +} + +func isBlank(s *string) bool { + return s == nil || strings.TrimSpace(*s) == "" +} + // Checks whether it is possible to take savepoint. func canTakeSavepoint(cluster v1beta1.FlinkCluster) bool { var jobSpec = cluster.Spec.Job var savepointStatus = cluster.Status.Savepoint - var jobStatus = cluster.Status.Components.Job + var job = cluster.Status.Components.Job return jobSpec != nil && jobSpec.SavepointsDir != nil && - !isJobStopped(jobStatus) && + !job.IsStopped() && (savepointStatus == nil || savepointStatus.State != v1beta1.SavepointStateInProgress) } -// shouldRestartJob returns true if the controller should restart failed or lost job. -func shouldRestartJob( - restartPolicy *v1beta1.JobRestartPolicy, - jobStatus *v1beta1.JobStatus) bool { - return restartPolicy != nil && - *restartPolicy == v1beta1.JobRestartPolicyFromSavepointOnFailure && - jobStatus != nil && - (jobStatus.State == v1beta1.JobStateFailed || jobStatus.State == v1beta1.JobStateLost) && - len(jobStatus.SavepointLocation) > 0 -} - -func shouldUpdateJob(observed ObservedClusterState) bool { - var jobStatus = observed.cluster.Status.Components.Job - var readyToUpdate = jobStatus == nil || isJobStopped(jobStatus) || isSavepointUpToDate(observed.observeTime, *jobStatus) - return isUpdateTriggered(observed.cluster.Status) && readyToUpdate +func shouldStopJob(cluster *v1beta1.FlinkCluster) bool { + var userControl = cluster.Annotations[v1beta1.ControlAnnotation] + var cancelRequested = cluster.Spec.Job.CancelRequested + return userControl == v1beta1.ControlNameJobCancel || + (cancelRequested != nil && *cancelRequested) } func getFromSavepoint(jobSpec batchv1.JobSpec) string { @@ -238,12 +237,12 @@ func getNextRevisionNumber(revisions []*appsv1.ControllerRevision) int64 { return revisions[count-1].Revision + 1 } -func getCurrentRevisionName(status v1beta1.FlinkClusterStatus) string { - return status.CurrentRevision[:strings.LastIndex(status.CurrentRevision, "-")] +func getCurrentRevisionName(r *v1beta1.RevisionStatus) string { + return r.CurrentRevision[:strings.LastIndex(r.CurrentRevision, "-")] } -func getNextRevisionName(status v1beta1.FlinkClusterStatus) string { - return status.NextRevision[:strings.LastIndex(status.NextRevision, "-")] +func getNextRevisionName(r *v1beta1.RevisionStatus) string { + return r.NextRevision[:strings.LastIndex(r.NextRevision, "-")] } // Compose revision in FlinkClusterStatus with name and number of ControllerRevision @@ -266,59 +265,31 @@ func getRetryCount(data map[string]string) (string, error) { return retries, err } -func getNewUserControlStatus(controlName string) *v1beta1.FlinkClusterControlStatus { +// getNewControlRequest returns new requested control that is not in progress now. +func getNewControlRequest(cluster *v1beta1.FlinkCluster) string { + var userControl = cluster.Annotations[v1beta1.ControlAnnotation] + var recorded = cluster.Status + if recorded.Control == nil || recorded.Control.State != v1beta1.ControlStateInProgress { + return userControl + } + return "" +} + +func getControlStatus(controlName string, state string) *v1beta1.FlinkClusterControlStatus { var controlStatus = new(v1beta1.FlinkClusterControlStatus) controlStatus.Name = controlName - controlStatus.State = v1beta1.ControlStateProgressing + controlStatus.State = state setTimestamp(&controlStatus.UpdateTime) return controlStatus } -func getTriggeredSavepointStatus(jobID string, triggerID string, triggerReason string, message string, triggerSuccess bool) v1beta1.SavepointStatus { - var savepointStatus = v1beta1.SavepointStatus{} - var now string - setTimestamp(&now) - savepointStatus.JobID = jobID - savepointStatus.TriggerID = triggerID - savepointStatus.TriggerReason = triggerReason - savepointStatus.TriggerTime = now - savepointStatus.RequestTime = now - savepointStatus.Message = message - if triggerSuccess { - savepointStatus.State = v1beta1.SavepointStateInProgress - } else { - savepointStatus.State = v1beta1.SavepointStateTriggerFailed - } - return savepointStatus -} - -func getRequestedSavepointStatus(triggerReason string) *v1beta1.SavepointStatus { - var now string - setTimestamp(&now) - return &v1beta1.SavepointStatus{ - State: v1beta1.SavepointStateNotTriggered, - TriggerReason: triggerReason, - RequestTime: now, - } -} - -func savepointTimeout(s *v1beta1.SavepointStatus) bool { - if s.TriggerTime == "" { - return false - } - tc := &TimeConverter{} - triggerTime := tc.FromString(s.TriggerTime) - validTime := triggerTime.Add(time.Duration(int64(SavepointTimeoutSec) * int64(time.Second))) - return time.Now().After(validTime) -} - func getControlEvent(status v1beta1.FlinkClusterControlStatus) (eventType string, eventReason string, eventMessage string) { var msg = status.Message if len(msg) > 100 { msg = msg[:100] + "..." } switch status.State { - case v1beta1.ControlStateProgressing: + case v1beta1.ControlStateInProgress: eventType = corev1.EventTypeNormal eventReason = "ControlRequested" eventMessage = fmt.Sprintf("Requested new user control %v", status.Name) @@ -368,50 +339,11 @@ func getSavepointEvent(status v1beta1.SavepointStatus) (eventType string, eventR return } -func isJobActive(status *v1beta1.JobStatus) bool { - return status != nil && - (status.State == v1beta1.JobStateRunning || status.State == v1beta1.JobStatePending) -} - -func isJobStopped(status *v1beta1.JobStatus) bool { - return status != nil && - (status.State == v1beta1.JobStateSucceeded || - status.State == v1beta1.JobStateFailed || - status.State == v1beta1.JobStateCancelled || - status.State == v1beta1.JobStateSuspended || - status.State == v1beta1.JobStateLost) -} - -func isJobCancelRequested(cluster v1beta1.FlinkCluster) bool { - var userControl = cluster.Annotations[v1beta1.ControlAnnotation] - var cancelRequested = cluster.Spec.Job.CancelRequested - return userControl == v1beta1.ControlNameJobCancel || - (cancelRequested != nil && *cancelRequested) -} - -func isJobTerminated(restartPolicy *v1beta1.JobRestartPolicy, jobStatus *v1beta1.JobStatus) bool { - return isJobStopped(jobStatus) && !shouldRestartJob(restartPolicy, jobStatus) -} - -func isUpdateTriggered(status v1beta1.FlinkClusterStatus) bool { - return status.CurrentRevision != status.NextRevision -} - func isUserControlFinished(controlStatus *v1beta1.FlinkClusterControlStatus) bool { return controlStatus.State == v1beta1.ControlStateSucceeded || controlStatus.State == v1beta1.ControlStateFailed } -// Check if the savepoint has been created recently. -func isSavepointUpToDate(now time.Time, jobStatus v1beta1.JobStatus) bool { - if jobStatus.SavepointLocation != "" && jobStatus.LastSavepointTime != "" { - if !hasTimeElapsed(jobStatus.LastSavepointTime, now, SavepointAgeForJobUpdateSec) { - return true - } - } - return false -} - // Check time has passed func hasTimeElapsed(timeToCheckStr string, now time.Time, intervalSec int) bool { tc := &TimeConverter{} @@ -427,8 +359,8 @@ func hasTimeElapsed(timeToCheckStr string, now time.Time, intervalSec int) bool // If the component is observed as well as the next revision name in status.nextRevision and component's label `flinkoperator.k8s.io/hash` are equal, then it is updated already. // If the component is not observed and it is required, then it is not updated yet. // If the component is not observed and it is optional, but it is specified in the spec, then it is not updated yet. -func isComponentUpdated(component runtime.Object, cluster v1beta1.FlinkCluster) bool { - if !isUpdateTriggered(cluster.Status) { +func isComponentUpdated(component runtime.Object, cluster *v1beta1.FlinkCluster) bool { + if !cluster.Status.Revision.IsUpdateTriggered() { return true } switch o := component.(type) { @@ -465,14 +397,14 @@ func isComponentUpdated(component runtime.Object, cluster v1beta1.FlinkCluster) } var labels, err = meta.NewAccessor().Labels(component) - var nextRevisionName = getNextRevisionName(cluster.Status) + var nextRevisionName = getNextRevisionName(&cluster.Status.Revision) if err != nil { return false } return labels[RevisionNameLabel] == nextRevisionName } -func areComponentsUpdated(components []runtime.Object, cluster v1beta1.FlinkCluster) bool { +func areComponentsUpdated(components []runtime.Object, cluster *v1beta1.FlinkCluster) bool { for _, c := range components { if !isComponentUpdated(c, cluster) { return false @@ -488,14 +420,14 @@ func isUpdatedAll(observed ObservedClusterState) bool { observed.tmStatefulSet, observed.jmService, observed.jmIngress, - observed.job, + observed.flinkJobSubmitter.job, } - return areComponentsUpdated(components, *observed.cluster) + return areComponentsUpdated(components, observed.cluster) } // isClusterUpdateToDate checks whether all cluster components are replaced to next revision. -func isClusterUpdateToDate(observed ObservedClusterState) bool { - if !isUpdateTriggered(observed.cluster.Status) { +func isClusterUpdateToDate(observed *ObservedClusterState) bool { + if !observed.cluster.Status.Revision.IsUpdateTriggered() { return true } components := []runtime.Object{ @@ -504,29 +436,52 @@ func isClusterUpdateToDate(observed ObservedClusterState) bool { observed.tmStatefulSet, observed.jmService, } - return areComponentsUpdated(components, *observed.cluster) + return areComponentsUpdated(components, observed.cluster) } // isFlinkAPIReady checks whether cluster is ready to submit job. -func isFlinkAPIReady(observed ObservedClusterState) bool { +func isFlinkAPIReady(list *flinkclient.JobStatusList) bool { // If the observed Flink job status list is not nil (e.g., emtpy list), // it means Flink REST API server is up and running. It is the source of // truth of whether we can submit a job. - return observed.flinkJobStatus.flinkJobList != nil + return list != nil +} + +// jobStateFinalized returns true, if job state is saved so that it can be resumed later. +func finalSavepointRequested(jobID string, s *v1beta1.SavepointStatus) bool { + return s != nil && s.JobID == jobID && + (s.TriggerReason == v1beta1.SavepointTriggerReasonUpdate || + s.TriggerReason == v1beta1.SavepointTriggerReasonJobCancel) } -func getUpdateState(observed ObservedClusterState) UpdateState { - var recordedJobStatus = observed.cluster.Status.Components.Job - if !isUpdateTriggered(observed.cluster.Status) { +func getUpdateState(observed *ObservedClusterState) UpdateState { + if observed.cluster == nil { return "" } - if isJobActive(recordedJobStatus) { - return UpdateStatePreparing + var recorded = observed.cluster.Status + var revision = recorded.Revision + var job = recorded.Components.Job + var jobSpec = observed.cluster.Spec.Job + + if !revision.IsUpdateTriggered() { + return "" } - if isClusterUpdateToDate(observed) { - return UpdateStateFinished + switch { + case !job.UpdateReady(jobSpec, observed.observeTime): + return UpdateStatePreparing + case !isClusterUpdateToDate(observed): + return UpdateStateInProgress } - return UpdateStateInProgress + return UpdateStateFinished +} + +func shouldUpdateJob(observed *ObservedClusterState) bool { + return observed.updateState == UpdateStateInProgress +} + +func shouldUpdateCluster(observed *ObservedClusterState) bool { + var job = observed.cluster.Status.Components.Job + return !job.IsActive() && observed.updateState == UpdateStateInProgress } func getNonLiveHistory(revisions []*appsv1.ControllerRevision, historyLimit int) []*appsv1.ControllerRevision { @@ -545,7 +500,7 @@ func getNonLiveHistory(revisions []*appsv1.ControllerRevision, historyLimit int) func getFlinkJobDeploymentState(flinkJobState string) string { switch flinkJobState { - case "INITIALIZING", "CREATED", "RUNNING", "FAILING", "CANCELLING", "RESTARTING", "RECONCILING": + case "INITIALIZING", "CREATED", "RUNNING", "FAILING", "CANCELLING", "RESTARTING", "RECONCILING", "SUSPENDED": return v1beta1.JobStateRunning case "FINISHED": return v1beta1.JobStateSucceeded @@ -553,36 +508,7 @@ func getFlinkJobDeploymentState(flinkJobState string) string { return v1beta1.JobStateCancelled case "FAILED": return v1beta1.JobStateFailed - case "SUSPENDED": - return v1beta1.JobStateSuspended default: return "" } } - -// getFlinkJobSubmitLog extract submit result from the pod termination log. -func getFlinkJobSubmitLog(observedPod *corev1.Pod) (*FlinkJobSubmitLog, error) { - if observedPod == nil { - return nil, fmt.Errorf("no job pod found, even though submission completed") - } - var containerStatuses = observedPod.Status.ContainerStatuses - if len(containerStatuses) == 0 || - containerStatuses[0].State.Terminated == nil || - containerStatuses[0].State.Terminated.Message == "" { - return nil, fmt.Errorf("job pod found, but no termination log found even though submission completed") - } - - // The job submission script writes the submission log to the pod termination log at the end of execution. - // If the job submission is successful, the extracted job ID is also included. - // The job submit script writes the submission result in YAML format, - // so parse it here to get the ID - if available - and log. - // Note: https://kubernetes.io/docs/tasks/debug-application-cluster/determine-reason-pod-failure/ - var rawJobSubmitResult = containerStatuses[0].State.Terminated.Message - var result = new(FlinkJobSubmitLog) - var err = yaml.Unmarshal([]byte(rawJobSubmitResult), result) - if err != nil { - return nil, err - } - - return result, nil -} diff --git a/controllers/flinkcluster_util_test.go b/controllers/flinkcluster_util_test.go index e90671c3..26089648 100644 --- a/controllers/flinkcluster_util_test.go +++ b/controllers/flinkcluster_util_test.go @@ -17,7 +17,8 @@ limitations under the License. package controllers import ( - "github.com/googlecloudplatform/flink-operator/controllers/flinkclient" + v1beta1 "github.com/googlecloudplatform/flink-operator/api/v1beta1" + "gotest.tools/assert" appsv1 "k8s.io/api/apps/v1" batchv1 "k8s.io/api/batch/v1" corev1 "k8s.io/api/core/v1" @@ -27,10 +28,6 @@ import ( "k8s.io/apimachinery/pkg/runtime" "os" "testing" - "time" - - v1beta1 "github.com/googlecloudplatform/flink-operator/api/v1beta1" - "gotest.tools/assert" ) func TestTimeConverter(t *testing.T) { @@ -47,30 +44,6 @@ func TestTimeConverter(t *testing.T) { assert.Assert(t, str3 == str4) } -func TestShouldRestartJob(t *testing.T) { - var restartOnFailure = v1beta1.JobRestartPolicyFromSavepointOnFailure - var jobStatus1 = v1beta1.JobStatus{ - State: v1beta1.JobStateFailed, - SavepointLocation: "gs://my-bucket/savepoint-123", - } - var restart1 = shouldRestartJob(&restartOnFailure, &jobStatus1) - assert.Equal(t, restart1, true) - - var jobStatus2 = v1beta1.JobStatus{ - State: v1beta1.JobStateFailed, - } - var restart2 = shouldRestartJob(&restartOnFailure, &jobStatus2) - assert.Equal(t, restart2, false) - - var neverRestart = v1beta1.JobRestartPolicyNever - var jobStatus3 = v1beta1.JobStatus{ - State: v1beta1.JobStateFailed, - SavepointLocation: "gs://my-bucket/savepoint-123", - } - var restart3 = shouldRestartJob(&neverRestart, &jobStatus3) - assert.Equal(t, restart3, false) -} - func TestGetRetryCount(t *testing.T) { var data1 = map[string]string{} var result1, _ = getRetryCount(data1) @@ -220,82 +193,6 @@ func TestCanTakeSavepoint(t *testing.T) { assert.Equal(t, take, false) } -func TestShouldUpdateJob(t *testing.T) { - // should update - var tc = &TimeConverter{} - var savepointTime = time.Now() - var observeTime = savepointTime.Add(time.Second * 100) - var observed = ObservedClusterState{ - observeTime: observeTime, - cluster: &v1beta1.FlinkCluster{ - Status: v1beta1.FlinkClusterStatus{ - Components: v1beta1.FlinkClusterComponentsStatus{Job: &v1beta1.JobStatus{ - State: v1beta1.JobStateRunning, - LastSavepointTime: tc.ToString(savepointTime), - LastSavepointTriggerTime: tc.ToString(savepointTime), - SavepointLocation: "gs://my-bucket/savepoint-123", - }}, - CurrentRevision: "1", NextRevision: "2", - }, - }, - } - var update = shouldUpdateJob(observed) - assert.Equal(t, update, true) - - // should update when update triggered and job failed. - observed = ObservedClusterState{ - cluster: &v1beta1.FlinkCluster{ - Status: v1beta1.FlinkClusterStatus{ - Components: v1beta1.FlinkClusterComponentsStatus{Job: &v1beta1.JobStatus{ - State: v1beta1.JobStateFailed, - }}, - CurrentRevision: "1", NextRevision: "2", - }, - }, - } - update = shouldUpdateJob(observed) - assert.Equal(t, update, true) - - // cannot update with old savepoint - tc = &TimeConverter{} - savepointTime = time.Now() - observeTime = savepointTime.Add(time.Second * 500) - observed = ObservedClusterState{ - observeTime: observeTime, - cluster: &v1beta1.FlinkCluster{ - Status: v1beta1.FlinkClusterStatus{ - Components: v1beta1.FlinkClusterComponentsStatus{Job: &v1beta1.JobStatus{ - State: v1beta1.JobStateRunning, - LastSavepointTime: tc.ToString(savepointTime), - LastSavepointTriggerTime: tc.ToString(savepointTime), - SavepointLocation: "gs://my-bucket/savepoint-123", - }}, - CurrentRevision: "1", NextRevision: "2", - }, - }, - } - update = shouldUpdateJob(observed) - assert.Equal(t, update, false) - - // cannot update without savepointLocation - tc = &TimeConverter{} - savepointTime = time.Now() - observeTime = savepointTime.Add(time.Second * 500) - observed = ObservedClusterState{ - observeTime: observeTime, - cluster: &v1beta1.FlinkCluster{ - Status: v1beta1.FlinkClusterStatus{ - Components: v1beta1.FlinkClusterComponentsStatus{Job: &v1beta1.JobStatus{ - State: v1beta1.JobStateUpdating, - }}, - CurrentRevision: "1", NextRevision: "2", - }, - }, - } - update = shouldUpdateJob(observed) - assert.Equal(t, update, false) -} - func TestGetNextRevisionNumber(t *testing.T) { var revisions []*appsv1.ControllerRevision var nextRevision = getNextRevisionNumber(revisions) @@ -306,172 +203,50 @@ func TestGetNextRevisionNumber(t *testing.T) { assert.Equal(t, nextRevision, int64(3)) } -func TestIsJobTerminated(t *testing.T) { - var jobStatus = v1beta1.JobStatus{ - State: v1beta1.JobStateSucceeded, - } - var terminated = isJobTerminated(nil, &jobStatus) - assert.Equal(t, terminated, true) - - var restartOnFailure = v1beta1.JobRestartPolicyFromSavepointOnFailure - jobStatus = v1beta1.JobStatus{ - State: v1beta1.JobStateFailed, - SavepointLocation: "gs://my-bucket/savepoint-123", - } - terminated = isJobTerminated(&restartOnFailure, &jobStatus) - assert.Equal(t, terminated, false) -} - -func TestIsSavepointUpToDate(t *testing.T) { - var tc = &TimeConverter{} - var savepointTime = time.Now() - var observeTime = savepointTime.Add(time.Second * 100) - var jobStatus = v1beta1.JobStatus{ - State: v1beta1.JobStateFailed, - LastSavepointTime: tc.ToString(savepointTime), - LastSavepointTriggerTime: tc.ToString(savepointTime), - SavepointLocation: "gs://my-bucket/savepoint-123", - } - var update = isSavepointUpToDate(observeTime, jobStatus) - assert.Equal(t, update, true) - - // old - savepointTime = time.Now() - observeTime = savepointTime.Add(time.Second * 500) - jobStatus = v1beta1.JobStatus{ - State: v1beta1.JobStateFailed, - LastSavepointTime: tc.ToString(savepointTime), - LastSavepointTriggerTime: tc.ToString(savepointTime), - SavepointLocation: "gs://my-bucket/savepoint-123", - } - update = isSavepointUpToDate(observeTime, jobStatus) - assert.Equal(t, update, false) - - // Fails without savepointLocation - savepointTime = time.Now() - observeTime = savepointTime.Add(time.Second * 500) - jobStatus = v1beta1.JobStatus{ - State: v1beta1.JobStateFailed, - LastSavepointTime: tc.ToString(savepointTime), - LastSavepointTriggerTime: tc.ToString(savepointTime), - } - update = isSavepointUpToDate(observeTime, jobStatus) - assert.Equal(t, update, false) -} - func TestIsComponentUpdated(t *testing.T) { var cluster = v1beta1.FlinkCluster{ - Status: v1beta1.FlinkClusterStatus{NextRevision: "cluster-85dc8f749-2"}, + Status: v1beta1.FlinkClusterStatus{Revision: v1beta1.RevisionStatus{NextRevision: "cluster-85dc8f749-2"}}, } var cluster2 = v1beta1.FlinkCluster{ Spec: v1beta1.FlinkClusterSpec{ JobManager: v1beta1.JobManagerSpec{Ingress: &v1beta1.JobManagerIngressSpec{}}, Job: &v1beta1.JobSpec{}, }, - Status: v1beta1.FlinkClusterStatus{NextRevision: "cluster-85dc8f749-2"}, + Status: v1beta1.FlinkClusterStatus{Revision: v1beta1.RevisionStatus{NextRevision: "cluster-85dc8f749-2"}}, } var deploy = &appsv1.Deployment{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{ RevisionNameLabel: "cluster-85dc8f749", }}} - var update = isComponentUpdated(deploy, cluster) + var update = isComponentUpdated(deploy, &cluster) assert.Equal(t, update, true) deploy = &appsv1.Deployment{} - update = isComponentUpdated(deploy, cluster) + update = isComponentUpdated(deploy, &cluster) assert.Equal(t, update, false) deploy = nil - update = isComponentUpdated(deploy, cluster) + update = isComponentUpdated(deploy, &cluster) assert.Equal(t, update, false) var job = &batchv1.Job{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{ RevisionNameLabel: "cluster-85dc8f749", }}} - update = isComponentUpdated(job, cluster2) + update = isComponentUpdated(job, &cluster2) assert.Equal(t, update, true) job = &batchv1.Job{} - update = isComponentUpdated(job, cluster2) + update = isComponentUpdated(job, &cluster2) assert.Equal(t, update, false) job = nil - update = isComponentUpdated(job, cluster2) + update = isComponentUpdated(job, &cluster2) assert.Equal(t, update, false) job = nil - update = isComponentUpdated(job, cluster) + update = isComponentUpdated(job, &cluster) assert.Equal(t, update, true) } -func TestIsFlinkAPIReady(t *testing.T) { - var observed = ObservedClusterState{ - cluster: &v1beta1.FlinkCluster{ - Spec: v1beta1.FlinkClusterSpec{ - JobManager: v1beta1.JobManagerSpec{Ingress: &v1beta1.JobManagerIngressSpec{}}, - Job: &v1beta1.JobSpec{}, - }, - Status: v1beta1.FlinkClusterStatus{NextRevision: "cluster-85dc8f749-2"}, - }, - configMap: &corev1.ConfigMap{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - jmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - tmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - jmService: &corev1.Service{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - flinkJobStatus: FlinkJobStatus{flinkJobList: &flinkclient.JobStatusList{}}, - } - var ready = isFlinkAPIReady(observed) - assert.Equal(t, ready, true) - - // flinkJobList is nil - observed = ObservedClusterState{ - cluster: &v1beta1.FlinkCluster{ - Spec: v1beta1.FlinkClusterSpec{ - JobManager: v1beta1.JobManagerSpec{Ingress: &v1beta1.JobManagerIngressSpec{}}, - Job: &v1beta1.JobSpec{}, - }, - Status: v1beta1.FlinkClusterStatus{NextRevision: "cluster-85dc8f749-2"}, - }, - configMap: &corev1.ConfigMap{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - jmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - tmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - jmService: &corev1.Service{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - } - ready = isFlinkAPIReady(observed) - assert.Equal(t, ready, false) - - // jmStatefulSet is not observed - observed = ObservedClusterState{ - cluster: &v1beta1.FlinkCluster{ - Spec: v1beta1.FlinkClusterSpec{ - JobManager: v1beta1.JobManagerSpec{Ingress: &v1beta1.JobManagerIngressSpec{}}, - Job: &v1beta1.JobSpec{}, - }, - Status: v1beta1.FlinkClusterStatus{NextRevision: "cluster-85dc8f749-2"}, - }, - configMap: &corev1.ConfigMap{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - tmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - jmService: &corev1.Service{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - } - ready = isFlinkAPIReady(observed) - assert.Equal(t, ready, false) - - // jmStatefulSet is not updated - observed = ObservedClusterState{ - cluster: &v1beta1.FlinkCluster{ - Spec: v1beta1.FlinkClusterSpec{ - JobManager: v1beta1.JobManagerSpec{Ingress: &v1beta1.JobManagerIngressSpec{}}, - Job: &v1beta1.JobSpec{}, - }, - Status: v1beta1.FlinkClusterStatus{NextRevision: "cluster-85dc8f749-2"}, - }, - configMap: &corev1.ConfigMap{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - jmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-aa5e3a87z"}}}, - tmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - jmService: &corev1.Service{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - } - ready = isFlinkAPIReady(observed) - assert.Equal(t, ready, false) -} - func TestGetUpdateState(t *testing.T) { var observed = ObservedClusterState{ cluster: &v1beta1.FlinkCluster{ @@ -480,16 +255,16 @@ func TestGetUpdateState(t *testing.T) { Job: &v1beta1.JobSpec{}, }, Status: v1beta1.FlinkClusterStatus{ - Components: v1beta1.FlinkClusterComponentsStatus{Job: &v1beta1.JobStatus{State: v1beta1.JobStateRunning}}, - CurrentRevision: "cluster-85dc8f749-2", NextRevision: "cluster-aa5e3a87z-3"}, + Components: v1beta1.FlinkClusterComponentsStatus{Job: &v1beta1.JobStatus{State: v1beta1.JobStateRunning}}, + Revision: v1beta1.RevisionStatus{CurrentRevision: "cluster-85dc8f749-2", NextRevision: "cluster-aa5e3a87z-3"}}, }, - job: &batchv1.Job{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - configMap: &corev1.ConfigMap{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - jmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - tmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, - jmService: &corev1.Service{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, + flinkJobSubmitter: FlinkJobSubmitter{job: &batchv1.Job{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}}, + configMap: &corev1.ConfigMap{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, + jmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, + tmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, + jmService: &corev1.Service{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, } - var state = getUpdateState(observed) + var state = getUpdateState(&observed) assert.Equal(t, state, UpdateStatePreparing) observed = ObservedClusterState{ @@ -498,13 +273,15 @@ func TestGetUpdateState(t *testing.T) { JobManager: v1beta1.JobManagerSpec{Ingress: &v1beta1.JobManagerIngressSpec{}}, Job: &v1beta1.JobSpec{}, }, - Status: v1beta1.FlinkClusterStatus{CurrentRevision: "cluster-85dc8f749-2", NextRevision: "cluster-aa5e3a87z-3"}, + Status: v1beta1.FlinkClusterStatus{ + Revision: v1beta1.RevisionStatus{CurrentRevision: "cluster-85dc8f749-2", NextRevision: "cluster-aa5e3a87z-3"}, + }, }, jmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-aa5e3a87z"}}}, tmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, jmService: &corev1.Service{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-85dc8f749"}}}, } - state = getUpdateState(observed) + state = getUpdateState(&observed) assert.Equal(t, state, UpdateStateInProgress) observed = ObservedClusterState{ @@ -513,16 +290,16 @@ func TestGetUpdateState(t *testing.T) { JobManager: v1beta1.JobManagerSpec{Ingress: &v1beta1.JobManagerIngressSpec{}}, Job: &v1beta1.JobSpec{}, }, - Status: v1beta1.FlinkClusterStatus{CurrentRevision: "cluster-85dc8f749-2", NextRevision: "cluster-aa5e3a87z-3"}, + Status: v1beta1.FlinkClusterStatus{Revision: v1beta1.RevisionStatus{CurrentRevision: "cluster-85dc8f749-2", NextRevision: "cluster-aa5e3a87z-3"}}, }, - job: &batchv1.Job{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-aa5e3a87z"}}}, - configMap: &corev1.ConfigMap{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-aa5e3a87z"}}}, - jmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-aa5e3a87z"}}}, - tmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-aa5e3a87z"}}}, - jmService: &corev1.Service{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-aa5e3a87z"}}}, - jmIngress: &extensionsv1beta1.Ingress{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-aa5e3a87z"}}}, + flinkJobSubmitter: FlinkJobSubmitter{job: &batchv1.Job{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-aa5e3a87z"}}}}, + configMap: &corev1.ConfigMap{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-aa5e3a87z"}}}, + jmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-aa5e3a87z"}}}, + tmStatefulSet: &appsv1.StatefulSet{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-aa5e3a87z"}}}, + jmService: &corev1.Service{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-aa5e3a87z"}}}, + jmIngress: &extensionsv1beta1.Ingress{ObjectMeta: metav1.ObjectMeta{Labels: map[string]string{RevisionNameLabel: "cluster-aa5e3a87z"}}}, } - state = getUpdateState(observed) + state = getUpdateState(&observed) assert.Equal(t, state, UpdateStateFinished) } @@ -575,54 +352,3 @@ func TestGetNonLiveHistory(t *testing.T) { nonLiveHistory = getNonLiveHistory(revisions, historyLimit) assert.Equal(t, len(nonLiveHistory), 0) } - -func TestGetFlinkJobDeploymentState(t *testing.T) { - var pod corev1.Pod - var submit, expected *FlinkJobSubmitLog - var err error - var termMsg string - - // success - termMsg = ` -jobID: ec74209eb4e3db8ae72db00bd7a830aa -message: | - Successfully submitted! - /opt/flink/bin/flink run --jobmanager flinkjobcluster-sample-jobmanager:8081 --class org.apache.flink.streaming.examples.wordcount.WordCount --parallelism 2 --detached ./examples/streaming/WordCount.jar --input ./README.txt - Starting execution of program - Printing result to stdout. Use --output to specify output path. - Job has been submitted with JobID ec74209eb4e3db8ae72db00bd7a830aa -` - expected = &FlinkJobSubmitLog{ - JobID: "ec74209eb4e3db8ae72db00bd7a830aa", - Message: `Successfully submitted! -/opt/flink/bin/flink run --jobmanager flinkjobcluster-sample-jobmanager:8081 --class org.apache.flink.streaming.examples.wordcount.WordCount --parallelism 2 --detached ./examples/streaming/WordCount.jar --input ./README.txt -Starting execution of program -Printing result to stdout. Use --output to specify output path. -Job has been submitted with JobID ec74209eb4e3db8ae72db00bd7a830aa -`, - } - pod = corev1.Pod{ - Status: corev1.PodStatus{ - ContainerStatuses: []corev1.ContainerStatus{{ - State: corev1.ContainerState{ - Terminated: &corev1.ContainerStateTerminated{ - Message: termMsg, - }}}}}} - submit, _ = getFlinkJobSubmitLog(&pod) - assert.DeepEqual(t, *submit, *expected) - - // failed: pod not found - submit, err = getFlinkJobSubmitLog(nil) - assert.Error(t, err, "no job pod found, even though submission completed") - - // failed: message not found - pod = corev1.Pod{ - Status: corev1.PodStatus{ - ContainerStatuses: []corev1.ContainerStatus{{ - State: corev1.ContainerState{ - Terminated: &corev1.ContainerStateTerminated{ - Message: "", - }}}}}} - submit, err = getFlinkJobSubmitLog(&pod) - assert.Error(t, err, "job pod found, but no termination log found even though submission completed") -} diff --git a/controllers/model/model.go b/controllers/model/model.go index 9faf6d39..968b6df3 100644 --- a/controllers/model/model.go +++ b/controllers/model/model.go @@ -23,9 +23,9 @@ import ( // DesiredClusterState holds desired state of a cluster. type DesiredClusterState struct { JmStatefulSet *appsv1.StatefulSet - JmService *corev1.Service - JmIngress *extensionsv1beta1.Ingress + JmService *corev1.Service + JmIngress *extensionsv1beta1.Ingress TmStatefulSet *appsv1.StatefulSet - ConfigMap *corev1.ConfigMap - Job *batchv1.Job + ConfigMap *corev1.ConfigMap + Job *batchv1.Job } diff --git a/docs/crd.md b/docs/crd.md index 1eef7a35..ae20ff4b 100644 --- a/docs/crd.md +++ b/docs/crd.md @@ -69,7 +69,7 @@ FlinkCluster |__ args |__ fromSavepoint |__ allowNonRestoredState - |__ takeSavepointOnUpgrade + |__ takeSavepointOnUpdate |__ autoSavepointSeconds |__ savepointsDir |__ savepointGeneration @@ -263,7 +263,7 @@ FlinkCluster * **autoSavepointSeconds** (optional): Automatically take a savepoint to the `savepointsDir` every n seconds. * **savepointsDir** (optional): Savepoints dir where to store automatically taken savepoints. * **allowNonRestoredState** (optional): Allow non-restored state, default: false. - * **takeSavepointOnUpgrade** (optional): Should take savepoint before upgrading the job, default: false. + * **takeSavepointOnUpdate** (optional): Should take savepoint before updating the job, default: false. * **savepointGeneration** (optional): Update this field to `jobStatus.savepointGeneration + 1` for a running job cluster to trigger a new savepoint to `savepointsDir` on demand. * **parallelism** (optional): Parallelism of the job, default: 1. diff --git a/helm-chart/flink-operator/templates/flink-cluster-crd.yaml b/helm-chart/flink-operator/templates/flink-cluster-crd.yaml index 066cdaa6..7c2cee3d 100644 --- a/helm-chart/flink-operator/templates/flink-cluster-crd.yaml +++ b/helm-chart/flink-operator/templates/flink-cluster-crd.yaml @@ -164,7 +164,7 @@ spec: type: integer cancelRequested: type: boolean - takeSavepointOnUpgrade: + takeSavepointOnUpdate: type: boolean className: type: string