17
17
18
18
package org .apache .flink .kubernetes .operator .observer ;
19
19
20
+ import org .apache .flink .api .common .JobID ;
20
21
import org .apache .flink .kubernetes .operator .api .AbstractFlinkResource ;
21
22
import org .apache .flink .kubernetes .operator .api .spec .JobState ;
22
23
import org .apache .flink .kubernetes .operator .api .status .JobStatus ;
23
24
import org .apache .flink .kubernetes .operator .controller .FlinkResourceContext ;
24
25
import org .apache .flink .kubernetes .operator .reconciler .ReconciliationUtils ;
25
26
import org .apache .flink .kubernetes .operator .utils .EventRecorder ;
26
27
import org .apache .flink .runtime .client .JobStatusMessage ;
28
+ import org .apache .flink .runtime .rest .NotFoundException ;
27
29
28
30
import org .slf4j .Logger ;
29
31
import org .slf4j .LoggerFactory ;
30
32
31
- import java .util .ArrayList ;
32
- import java .util .List ;
33
- import java .util .Optional ;
34
33
import java .util .concurrent .TimeoutException ;
35
34
36
35
import static org .apache .flink .kubernetes .operator .utils .FlinkResourceExceptionUtils .updateFlinkResourceException ;
37
36
38
37
/** An observer to observe the job status. */
39
- public abstract class JobStatusObserver <R extends AbstractFlinkResource <?, ?>> {
38
+ public class JobStatusObserver <R extends AbstractFlinkResource <?, ?>> {
40
39
41
40
private static final Logger LOG = LoggerFactory .getLogger (JobStatusObserver .class );
42
41
43
- public static final String MISSING_SESSION_JOB_ERR = "Missing Session Job " ;
42
+ public static final String JOB_NOT_FOUND_ERR = "Job Not Found " ;
44
43
45
44
protected final EventRecorder eventRecorder ;
46
45
@@ -68,58 +67,51 @@ public boolean observe(FlinkResourceContext<R> ctx) {
68
67
LOG .debug ("Observing job status" );
69
68
var previousJobStatus = jobStatus .getState ();
70
69
71
- List <JobStatusMessage > clusterJobStatuses ;
72
70
try {
73
- // Query job list from the cluster
74
- clusterJobStatuses =
75
- new ArrayList <>(ctx .getFlinkService ().listJobs (ctx .getObserveConfig ()));
71
+ var newJobStatusOpt =
72
+ ctx .getFlinkService ()
73
+ .getJobStatus (
74
+ ctx .getObserveConfig (),
75
+ JobID .fromHexString (jobStatus .getJobId ()));
76
+
77
+ if (newJobStatusOpt .isPresent ()) {
78
+ updateJobStatus (ctx , newJobStatusOpt .get ());
79
+ ReconciliationUtils .checkAndUpdateStableSpec (resource .getStatus ());
80
+ return true ;
81
+ } else {
82
+ onTargetJobNotFound (ctx );
83
+ }
76
84
} catch (Exception e ) {
77
85
// Error while accessing the rest api, will try again later...
78
- LOG .warn ("Exception while listing jobs " , e );
86
+ LOG .warn ("Exception while getting job status " , e );
79
87
ifRunningMoveToReconciling (jobStatus , previousJobStatus );
80
88
if (e instanceof TimeoutException ) {
81
89
onTimeout (ctx );
82
90
}
83
- return false ;
84
- }
85
-
86
- if (!clusterJobStatuses .isEmpty ()) {
87
- // There are jobs on the cluster, we filter the ones for this resource
88
- Optional <JobStatusMessage > targetJobStatusMessage =
89
- filterTargetJob (jobStatus , clusterJobStatuses );
90
-
91
- if (targetJobStatusMessage .isEmpty ()) {
92
- LOG .warn ("No matching jobs found on the cluster" );
93
- ifRunningMoveToReconciling (jobStatus , previousJobStatus );
94
- onTargetJobNotFound (ctx );
95
- return false ;
96
- } else {
97
- updateJobStatus (ctx , targetJobStatusMessage .get ());
98
- }
99
- ReconciliationUtils .checkAndUpdateStableSpec (resource .getStatus ());
100
- return true ;
101
- } else {
102
- LOG .debug ("No jobs found on the cluster" );
103
- // No jobs found on the cluster, it is possible that the jobmanager is still starting up
104
- ifRunningMoveToReconciling (jobStatus , previousJobStatus );
105
- onNoJobsFound (ctx );
106
- return false ;
107
91
}
92
+ return false ;
108
93
}
109
94
110
95
/**
111
96
* Callback when no matching target job was found on a cluster where jobs were found.
112
97
*
113
98
* @param ctx The Flink resource context.
114
99
*/
115
- protected abstract void onTargetJobNotFound (FlinkResourceContext <R > ctx );
116
-
117
- /**
118
- * Callback when no jobs were found on the cluster.
119
- *
120
- * @param ctx The Flink resource context.
121
- */
122
- protected void onNoJobsFound (FlinkResourceContext <R > ctx ) {}
100
+ protected void onTargetJobNotFound (FlinkResourceContext <R > ctx ) {
101
+ ctx .getResource ()
102
+ .getStatus ()
103
+ .getJobStatus ()
104
+ .setState (org .apache .flink .api .common .JobStatus .RECONCILING .name ());
105
+ ReconciliationUtils .updateForReconciliationError (
106
+ ctx , new NotFoundException (JOB_NOT_FOUND_ERR ));
107
+ eventRecorder .triggerEvent (
108
+ ctx .getResource (),
109
+ EventRecorder .Type .Warning ,
110
+ EventRecorder .Reason .Missing ,
111
+ EventRecorder .Component .Job ,
112
+ JOB_NOT_FOUND_ERR ,
113
+ ctx .getKubernetesClient ());
114
+ }
123
115
124
116
/**
125
117
* If we observed the job previously in RUNNING state we move to RECONCILING instead as we are
@@ -139,18 +131,7 @@ private void ifRunningMoveToReconciling(JobStatus jobStatus, String previousJobS
139
131
*
140
132
* @param ctx Resource context.
141
133
*/
142
- protected abstract void onTimeout (FlinkResourceContext <R > ctx );
143
-
144
- /**
145
- * Filter the target job status message by the job list from the cluster.
146
- *
147
- * @param status the target job status.
148
- * @param clusterJobStatuses the candidate cluster jobs.
149
- * @return The target job status message. If no matched job found, {@code Optional.empty()} will
150
- * be returned.
151
- */
152
- protected abstract Optional <JobStatusMessage > filterTargetJob (
153
- JobStatus status , List <JobStatusMessage > clusterJobStatuses );
134
+ protected void onTimeout (FlinkResourceContext <R > ctx ) {}
154
135
155
136
/**
156
137
* Update the status in CR according to the cluster job status.
@@ -161,16 +142,13 @@ protected abstract Optional<JobStatusMessage> filterTargetJob(
161
142
private void updateJobStatus (FlinkResourceContext <R > ctx , JobStatusMessage clusterJobStatus ) {
162
143
var resource = ctx .getResource ();
163
144
var jobStatus = resource .getStatus ().getJobStatus ();
164
- var previousJobId = jobStatus .getJobId ();
165
145
var previousJobStatus = jobStatus .getState ();
166
146
167
147
jobStatus .setState (clusterJobStatus .getJobState ().name ());
168
148
jobStatus .setJobName (clusterJobStatus .getJobName ());
169
- jobStatus .setJobId (clusterJobStatus .getJobId ().toHexString ());
170
149
jobStatus .setStartTime (String .valueOf (clusterJobStatus .getStartTime ()));
171
150
172
- if (jobStatus .getJobId ().equals (previousJobId )
173
- && jobStatus .getState ().equals (previousJobStatus )) {
151
+ if (jobStatus .getState ().equals (previousJobStatus )) {
174
152
LOG .debug ("Job status ({}) unchanged" , previousJobStatus );
175
153
} else {
176
154
jobStatus .setUpdateTime (String .valueOf (System .currentTimeMillis ()));
0 commit comments