@@ -306,20 +306,34 @@ func (r *Reconciler) handleSparkApplicationDeletion(ctx context.Context, req ctr
306306func (r * Reconciler ) reconcileNewSparkApplication (ctx context.Context , req ctrl.Request ) (ctrl.Result , error ) {
307307 logger := log .FromContext (ctx )
308308 key := req .NamespacedName
309+
310+ // Fetch and validate state outside the retry loop.
311+ old , err := r .getSparkApplication (ctx , key )
312+ if err != nil {
313+ return ctrl.Result {Requeue : true }, err
314+ }
315+ if old .Status .AppState .State != v1beta2 .ApplicationStateNew {
316+ return ctrl.Result {}, nil
317+ }
318+
319+ // Submit once, outside the retry loop. submitSparkApplication is idempotent —
320+ // if a driver pod already exists it will recover state instead of re-submitting.
321+ app := old .DeepCopy ()
322+ r .submitSparkApplication (ctx , app )
323+
324+ // Use RetryOnConflict only for the status update.
309325 retryErr := retry .RetryOnConflict (
310326 retry .DefaultRetry ,
311327 func () error {
312- old , err := r .getSparkApplication (ctx , key )
328+ fresh , err := r .getSparkApplication (ctx , key )
313329 if err != nil {
314330 return err
315331 }
316- if old .Status .AppState .State != v1beta2 .ApplicationStateNew {
332+ if fresh .Status .AppState .State != v1beta2 .ApplicationStateNew {
317333 return nil
318334 }
319- app := old .DeepCopy ()
320-
321- r .submitSparkApplication (ctx , app )
322- if err := r .updateSparkApplicationStatus (ctx , app ); err != nil {
335+ fresh .Status = app .Status
336+ if err := r .updateSparkApplicationStatus (ctx , fresh ); err != nil {
323337 return err
324338 }
325339 return nil
@@ -831,22 +845,34 @@ func (r *Reconciler) reconcileSuspendedSparkApplication(ctx context.Context, req
831845func (r * Reconciler ) reconcileResumingSparkApplication (ctx context.Context , req ctrl.Request ) (ctrl.Result , error ) {
832846 logger := log .FromContext (ctx )
833847 key := req .NamespacedName
848+
849+ // Fetch and validate state outside the retry loop.
850+ old , err := r .getSparkApplication (ctx , key )
851+ if err != nil {
852+ return ctrl.Result {Requeue : true }, err
853+ }
854+ if old .Status .AppState .State != v1beta2 .ApplicationStateResuming {
855+ return ctrl.Result {}, nil
856+ }
857+
858+ // Submit once, outside the retry loop. submitSparkApplication is idempotent.
859+ app := old .DeepCopy ()
860+ r .recordSparkApplicationEvent (app )
861+ r .submitSparkApplication (ctx , app )
862+
863+ // Use RetryOnConflict only for the status update.
834864 retryErr := retry .RetryOnConflict (
835865 retry .DefaultRetry ,
836866 func () error {
837- old , err := r .getSparkApplication (ctx , key )
867+ fresh , err := r .getSparkApplication (ctx , key )
838868 if err != nil {
839869 return err
840870 }
841- if old .Status .AppState .State != v1beta2 .ApplicationStateResuming {
871+ if fresh .Status .AppState .State != v1beta2 .ApplicationStateResuming {
842872 return nil
843873 }
844- app := old .DeepCopy ()
845-
846- r .recordSparkApplicationEvent (app )
847-
848- r .submitSparkApplication (ctx , app )
849- if err := r .updateSparkApplicationStatus (ctx , app ); err != nil {
874+ fresh .Status = app .Status
875+ if err := r .updateSparkApplicationStatus (ctx , fresh ); err != nil {
850876 return err
851877 }
852878 return nil
@@ -897,13 +923,39 @@ func (r *Reconciler) getSparkApplication(ctx context.Context, key types.Namespac
897923
898924// submitSparkApplication creates a new submission for the given SparkApplication and submits it using spark-submit.
899925// The submission result are recorded in app.Status.{AppState,ExecutionAttempts}.
926+ // This method is idempotent: if a driver pod already exists for this application (e.g. from a previous
927+ // submission attempt whose status update failed), the submission state is recovered from the existing pod
928+ // instead of re-running spark-submit.
900929func (r * Reconciler ) submitSparkApplication (ctx context.Context , app * v1beta2.SparkApplication ) {
901930 logger := log .FromContext (ctx )
902931 logger .Info ("Submitting SparkApplication" , "state" , app .Status .AppState .State )
903932
933+ // Check if a driver pod from a previous submission attempt already exists.
934+ driverPodName := util .GetDriverPodName (app )
935+ existingPod := & corev1.Pod {}
936+ podKey := types.NamespacedName {Name : driverPodName , Namespace : app .Namespace }
937+ if err := r .client .Get (ctx , podKey , existingPod ); err == nil {
938+ // Pod exists — verify it belongs to this application and was launched by the operator.
939+ if existingPod .Labels [common .LabelSparkAppName ] == app .Name &&
940+ existingPod .Labels [common .LabelLaunchedBySparkOperator ] == "true" {
941+ logger .Info ("Found existing driver pod from previous submission attempt, recovering state" ,
942+ "pod" , driverPodName , "submissionID" , existingPod .Labels [common .LabelSubmissionID ])
943+ app .Status .SubmissionID = existingPod .Labels [common .LabelSubmissionID ]
944+ app .Status .DriverInfo .PodName = driverPodName
945+ app .Status .LastSubmissionAttemptTime = metav1 .Now ()
946+ app .Status .SubmissionAttempts = app .Status .SubmissionAttempts + 1
947+ app .Status .AppState = v1beta2.ApplicationState {
948+ State : v1beta2 .ApplicationStateSubmitted ,
949+ }
950+ app .Status .ExecutionAttempts = app .Status .ExecutionAttempts + 1
951+ r .recordSparkApplicationEvent (app )
952+ return
953+ }
954+ }
955+
904956 // SubmissionID must be set before creating any resources to ensure all the resources are labeled.
905957 app .Status .SubmissionID = uuid .New ().String ()
906- app .Status .DriverInfo .PodName = util . GetDriverPodName ( app )
958+ app .Status .DriverInfo .PodName = driverPodName
907959 app .Status .LastSubmissionAttemptTime = metav1 .Now ()
908960 app .Status .SubmissionAttempts = app .Status .SubmissionAttempts + 1
909961
0 commit comments