|
17 | 17 |
|
18 | 18 | package org.apache.flink.kubernetes.operator.reconciler.deployment;
|
19 | 19 |
|
| 20 | +import org.apache.flink.api.common.JobID; |
| 21 | +import org.apache.flink.api.common.JobStatus; |
| 22 | +import org.apache.flink.api.java.tuple.Tuple3; |
20 | 23 | import org.apache.flink.configuration.Configuration;
|
21 | 24 | import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions;
|
22 | 25 | import org.apache.flink.kubernetes.operator.OperatorTestBase;
|
23 | 26 | import org.apache.flink.kubernetes.operator.TestUtils;
|
24 | 27 | import org.apache.flink.kubernetes.operator.TestingFlinkService;
|
25 | 28 | import org.apache.flink.kubernetes.operator.api.FlinkDeployment;
|
| 29 | +import org.apache.flink.kubernetes.operator.api.FlinkSessionJob; |
26 | 30 | import org.apache.flink.kubernetes.operator.api.spec.FlinkDeploymentSpec;
|
27 | 31 | import org.apache.flink.kubernetes.operator.api.status.FlinkDeploymentStatus;
|
28 | 32 | import org.apache.flink.kubernetes.operator.api.status.JobManagerDeploymentStatus;
|
29 | 33 | import org.apache.flink.kubernetes.operator.api.status.ReconciliationState;
|
| 34 | +import org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions; |
30 | 35 | import org.apache.flink.kubernetes.operator.reconciler.TestReconcilerAdapter;
|
| 36 | +import org.apache.flink.runtime.client.JobStatusMessage; |
31 | 37 |
|
32 | 38 | import io.fabric8.kubernetes.api.model.ObjectMeta;
|
33 | 39 | import io.fabric8.kubernetes.client.KubernetesClient;
|
|
36 | 42 | import org.junit.jupiter.api.Assertions;
|
37 | 43 | import org.junit.jupiter.api.Test;
|
38 | 44 |
|
| 45 | +import java.util.HashSet; |
39 | 46 | import java.util.List;
|
40 | 47 | import java.util.Map;
|
41 | 48 | import java.util.Optional;
|
42 | 49 | import java.util.Set;
|
43 | 50 | import java.util.concurrent.atomic.AtomicInteger;
|
44 | 51 |
|
45 | 52 | import static org.junit.jupiter.api.Assertions.assertEquals;
|
| 53 | +import static org.junit.jupiter.api.Assertions.assertFalse; |
46 | 54 | import static org.junit.jupiter.api.Assertions.assertTrue;
|
47 | 55 | import static org.junit.jupiter.api.Assertions.fail;
|
48 | 56 |
|
@@ -139,4 +147,161 @@ public void testSetOwnerReference() throws Exception {
|
139 | 147 | deployConfig.get(KubernetesConfigOptions.JOB_MANAGER_OWNER_REFERENCE);
|
140 | 148 | Assertions.assertEquals(expectedOwnerReferences, or);
|
141 | 149 | }
|
| 150 | + |
| 151 | + @Test |
| 152 | + public void testGetNonTerminalJobs() throws Exception { |
| 153 | + FlinkDeployment deployment = TestUtils.buildSessionCluster(); |
| 154 | + deployment |
| 155 | + .getSpec() |
| 156 | + .getFlinkConfiguration() |
| 157 | + .put(KubernetesOperatorConfigOptions.BLOCK_ON_UNMANAGED_JOBS.key(), "true"); |
| 158 | + |
| 159 | + assertEquals( |
| 160 | + "true", |
| 161 | + deployment |
| 162 | + .getSpec() |
| 163 | + .getFlinkConfiguration() |
| 164 | + .get(KubernetesOperatorConfigOptions.BLOCK_ON_UNMANAGED_JOBS.key())); |
| 165 | + |
| 166 | + reconciler.reconcile(deployment, flinkService.getContext()); |
| 167 | + |
| 168 | + // Verify deployment is in DEPLOYED state |
| 169 | + assertEquals( |
| 170 | + ReconciliationState.DEPLOYED, |
| 171 | + deployment.getStatus().getReconciliationStatus().getState()); |
| 172 | + |
| 173 | + // Create different types of jobs |
| 174 | + JobID managedJobId1 = new JobID(); |
| 175 | + JobID managedJobId2 = new JobID(); |
| 176 | + JobID unmanagedRunningJobId1 = new JobID(); |
| 177 | + JobID unmanagedTerminatedJobId = new JobID(); |
| 178 | + JobID unmanagedRunningJobId2 = new JobID(); |
| 179 | + |
| 180 | + // Add jobs to the testing service |
| 181 | + flinkService |
| 182 | + .listJobs() |
| 183 | + .add( |
| 184 | + Tuple3.of( |
| 185 | + null, |
| 186 | + new JobStatusMessage( |
| 187 | + managedJobId1, |
| 188 | + "managed-job-1", |
| 189 | + JobStatus.RUNNING, |
| 190 | + System.currentTimeMillis()), |
| 191 | + new Configuration())); |
| 192 | + flinkService |
| 193 | + .listJobs() |
| 194 | + .add( |
| 195 | + Tuple3.of( |
| 196 | + null, |
| 197 | + new JobStatusMessage( |
| 198 | + managedJobId2, |
| 199 | + "managed-job-2", |
| 200 | + JobStatus.RUNNING, |
| 201 | + System.currentTimeMillis()), |
| 202 | + new Configuration())); |
| 203 | + flinkService |
| 204 | + .listJobs() |
| 205 | + .add( |
| 206 | + Tuple3.of( |
| 207 | + null, |
| 208 | + new JobStatusMessage( |
| 209 | + unmanagedRunningJobId1, |
| 210 | + "unmanaged-running-job-1", |
| 211 | + JobStatus.RUNNING, |
| 212 | + System.currentTimeMillis()), |
| 213 | + new Configuration())); |
| 214 | + flinkService |
| 215 | + .listJobs() |
| 216 | + .add( |
| 217 | + Tuple3.of( |
| 218 | + null, |
| 219 | + new JobStatusMessage( |
| 220 | + unmanagedTerminatedJobId, |
| 221 | + "unmanaged-terminated-job", |
| 222 | + JobStatus.CANCELED, |
| 223 | + System.currentTimeMillis()), |
| 224 | + new Configuration())); |
| 225 | + flinkService |
| 226 | + .listJobs() |
| 227 | + .add( |
| 228 | + Tuple3.of( |
| 229 | + null, |
| 230 | + new JobStatusMessage( |
| 231 | + unmanagedRunningJobId2, |
| 232 | + "unmanaged-running-job-2", |
| 233 | + JobStatus.RUNNING, |
| 234 | + System.currentTimeMillis()), |
| 235 | + new Configuration())); |
| 236 | + |
| 237 | + // Create FlinkSessionJob resources for the managed jobs |
| 238 | + FlinkSessionJob managedSessionJob1 = TestUtils.buildSessionJob(); |
| 239 | + managedSessionJob1.getMetadata().setName("managed-session-job-1"); |
| 240 | + managedSessionJob1.getStatus().getJobStatus().setJobId(managedJobId1.toHexString()); |
| 241 | + kubernetesClient.resource(managedSessionJob1).createOrReplace(); |
| 242 | + |
| 243 | + FlinkSessionJob managedSessionJob2 = TestUtils.buildSessionJob(); |
| 244 | + managedSessionJob2.getMetadata().setName("managed-session-job-2"); |
| 245 | + managedSessionJob2.getStatus().getJobStatus().setJobId(managedJobId2.toHexString()); |
| 246 | + kubernetesClient.resource(managedSessionJob2).createOrReplace(); |
| 247 | + |
| 248 | + Set<FlinkSessionJob> sessionJobs = new HashSet<>(); |
| 249 | + sessionJobs.add(managedSessionJob1); |
| 250 | + sessionJobs.add(managedSessionJob2); |
| 251 | + |
| 252 | + // Test with blocking enabled - should identify all non-terminal jobs |
| 253 | + var context = TestUtils.createContextWithReadyFlinkDeployment(kubernetesClient); |
| 254 | + var resourceContext = getResourceContext(deployment, context); |
| 255 | + |
| 256 | + var sessionReconciler = (SessionReconciler) reconciler.getReconciler(); |
| 257 | + Set<JobID> nonTerminalJobs = sessionReconciler.getNonTerminalJobs(resourceContext); |
| 258 | + |
| 259 | + // Verify all non-terminal jobs are identified - should be 4 (2 managed + 2 unmanaged |
| 260 | + // running) |
| 261 | + assertEquals(4, nonTerminalJobs.size(), "Should identify exactly 4 non-terminal jobs"); |
| 262 | + |
| 263 | + assertTrue( |
| 264 | + nonTerminalJobs.contains(unmanagedRunningJobId1), |
| 265 | + "Should contain unmanagedRunningJobId1"); |
| 266 | + assertTrue( |
| 267 | + nonTerminalJobs.contains(unmanagedRunningJobId2), |
| 268 | + "Should contain unmanagedRunningJobId2"); |
| 269 | + |
| 270 | + // Verify terminated job is not included |
| 271 | + assertFalse( |
| 272 | + nonTerminalJobs.contains(unmanagedTerminatedJobId), |
| 273 | + "Should not contain terminated job"); |
| 274 | + |
| 275 | + // Test scenario with only unmanaged jobs |
| 276 | + flinkService |
| 277 | + .listJobs() |
| 278 | + .removeIf( |
| 279 | + job -> |
| 280 | + job.f1.getJobId().equals(managedJobId1) |
| 281 | + || job.f1.getJobId().equals(managedJobId2)); |
| 282 | + |
| 283 | + Set<JobID> nonTerminalJobsAfterSessionJobsRemoval = |
| 284 | + sessionReconciler.getNonTerminalJobs(resourceContext); |
| 285 | + |
| 286 | + assertEquals( |
| 287 | + 2, |
| 288 | + nonTerminalJobsAfterSessionJobsRemoval.size(), |
| 289 | + "Should have 2 non-terminal jobs when sessionjobs are deleted"); |
| 290 | + |
| 291 | + // Test scenario with no running jobs |
| 292 | + flinkService |
| 293 | + .listJobs() |
| 294 | + .removeIf( |
| 295 | + job -> |
| 296 | + job.f1.getJobId().equals(unmanagedRunningJobId1) |
| 297 | + || job.f1.getJobId().equals(unmanagedRunningJobId2)); |
| 298 | + |
| 299 | + Set<JobID> nonTerminalJobsAfterRemoval = |
| 300 | + sessionReconciler.getNonTerminalJobs(resourceContext); |
| 301 | + |
| 302 | + assertEquals( |
| 303 | + 0, |
| 304 | + nonTerminalJobsAfterRemoval.size(), |
| 305 | + "Should have no non-terminal jobs when only terminated jobs exist"); |
| 306 | + } |
142 | 307 | }
|
0 commit comments