29
29
import org .apache .commons .lang3 .tuple .Pair ;
30
30
import org .testng .Assert ;
31
31
import org .testng .annotations .AfterSuite ;
32
+ import org .testng .annotations .BeforeClass ;
32
33
import org .testng .annotations .BeforeSuite ;
33
34
import org .testng .annotations .Test ;
34
35
35
36
import com .google .common .io .Closer ;
36
37
import com .typesafe .config .Config ;
37
38
39
+ import lombok .extern .slf4j .Slf4j ;
40
+
38
41
import org .apache .gobblin .configuration .ConfigurationKeys ;
39
42
import org .apache .gobblin .kafka .KafkaTestBase ;
40
43
import org .apache .gobblin .kafka .client .Kafka09ConsumerClient ;
41
44
import org .apache .gobblin .kafka .writer .KafkaWriterConfigurationKeys ;
42
45
import org .apache .gobblin .runtime .api .JobSpec ;
43
46
import org .apache .gobblin .runtime .api .Spec ;
47
+ import org .apache .gobblin .runtime .api .SpecExecutor ;
44
48
import org .apache .gobblin .runtime .job_catalog .NonObservingFSJobCatalog ;
45
49
import org .apache .gobblin .runtime .job_monitor .KafkaJobMonitor ;
46
50
import org .apache .gobblin .util .ConfigUtils ;
47
51
import org .apache .gobblin .writer .WriteResponse ;
48
- import org .apache .gobblin .runtime .api .SpecExecutor ;
49
-
50
- import lombok .extern .slf4j .Slf4j ;
51
52
52
53
53
54
@ Slf4j
@@ -63,9 +64,12 @@ public class StreamingKafkaSpecExecutorTest extends KafkaTestBase {
63
64
private String _kafkaBrokers ;
64
65
private static final String _TEST_DIR_PATH = "/tmp/StreamingKafkaSpecExecutorTest" ;
65
66
private static final String _JOBS_DIR_PATH = _TEST_DIR_PATH + "/jobs" ;
67
+ String flowSpecUriString = "/flowgroup/flowname/spec" ;
68
+ Spec flowSpec = initJobSpecWithFlowExecutionId (flowSpecUriString , "12345" );
66
69
String specUriString = "/foo/bar/spec" ;
67
70
Spec spec = initJobSpec (specUriString );
68
71
72
+
69
73
@ BeforeSuite
70
74
public void beforeSuite () {
71
75
log .info ("Process id = " + ManagementFactory .getRuntimeMXBean ().getName ());
@@ -92,9 +96,8 @@ private void cleanupTestDir() {
92
96
}
93
97
}
94
98
}
95
-
96
- @ Test
97
- public void testAddSpec () throws Exception {
99
+ @ BeforeClass
100
+ public void setup () throws Exception {
98
101
_closer = Closer .create ();
99
102
_properties = new Properties ();
100
103
@@ -116,16 +119,20 @@ public void testAddSpec() throws Exception {
116
119
// SEI Producer
117
120
_seip = _closer .register (new SimpleKafkaSpecProducer (config ));
118
121
119
- WriteResponse writeResponse = (WriteResponse ) _seip .addSpec (spec ).get ();
120
- log .info ("WriteResponse: " + writeResponse );
121
-
122
122
_jobCatalog = new NonObservingFSJobCatalog (config .getConfig ("gobblin.cluster" ));
123
123
_jobCatalog .startAsync ().awaitRunning ();
124
124
125
125
// SEI Consumer
126
126
_seic = _closer .register (new StreamingKafkaSpecConsumer (config , _jobCatalog ));
127
127
_seic .startAsync ().awaitRunning ();
128
128
129
+ }
130
+
131
+ @ Test
132
+ public void testAddSpec () throws Exception {
133
+ WriteResponse writeResponse = (WriteResponse ) _seip .addSpec (spec ).get ();
134
+ log .info ("WriteResponse: " + writeResponse );
135
+
129
136
List <Pair <SpecExecutor .Verb , Spec >> consumedEvent = _seic .changedSpecs ().get ();
130
137
Assert .assertTrue (consumedEvent .size () == 1 , "Consumption did not match production" );
131
138
@@ -165,6 +172,78 @@ public void testDeleteSpec() throws Exception {
165
172
Assert .assertTrue (consumedSpecAction .getValue () instanceof JobSpec , "Expected JobSpec" );
166
173
}
167
174
175
+ @ Test (dependsOnMethods = "testDeleteSpec" )
176
+ public void testCancelSpec () throws Exception {
177
+ // Cancel an existing spec that was added
178
+ _seip .addSpec (spec ).get ();
179
+ WriteResponse writeResponse = (WriteResponse ) _seip .cancelJob (new URI (specUriString ), new Properties ()).get ();
180
+ log .info ("WriteResponse: " + writeResponse );
181
+
182
+ // Wait for the cancellation to be processed
183
+ Thread .sleep (5000 );
184
+ List <Pair <SpecExecutor .Verb , Spec >> consumedEvent = _seic .changedSpecs ().get ();
185
+ Assert .assertTrue (consumedEvent .size () == 3 , "Consumption did not match production" );
186
+
187
+ Map .Entry <SpecExecutor .Verb , Spec > consumedSpecAction = consumedEvent .get (2 );
188
+ log .info (consumedSpecAction .getKey ().toString ());
189
+ Assert .assertTrue (consumedEvent .get (0 ).getKey ().equals (SpecExecutor .Verb .ADD ), "Verb did not match" );
190
+ Assert .assertTrue (consumedEvent .get (1 ).getKey ().equals (SpecExecutor .Verb .DELETE ), "Verb did not match" );
191
+ Assert .assertTrue (consumedSpecAction .getKey ().equals (SpecExecutor .Verb .CANCEL ), "Verb did not match" );
192
+ Assert .assertTrue (consumedSpecAction .getValue ().getUri ().toString ().equals (specUriString ), "Expected URI did not match" );
193
+ Assert .assertTrue (consumedSpecAction .getValue () instanceof JobSpec , "Expected JobSpec" );
194
+ }
195
+
196
+ @ Test (dependsOnMethods = "testCancelSpec" )
197
+ public void testCancelSpecNoopDefault () throws Exception {
198
+ _seip .addSpec (flowSpec ).get ();
199
+ Properties props = new Properties ();
200
+ props .setProperty (ConfigurationKeys .FLOW_EXECUTION_ID_KEY , "54321" ); // Does not match with added jobspec, so should not cancel job
201
+ WriteResponse writeResponse = (WriteResponse ) _seip .cancelJob (new URI (flowSpecUriString ), props ).get ();
202
+ log .info ("WriteResponse: " + writeResponse );
203
+ // Wait for the cancellation to be processed, but it should ignore the spec as flow execution IDs do not match
204
+ Thread .sleep (5000 );
205
+ List <Pair <SpecExecutor .Verb , Spec >> consumedEvent = _seic .changedSpecs ().get ();
206
+ Assert .assertTrue (consumedEvent .size () == 1 , "Consumption did not match production" );
207
+
208
+ Map .Entry <SpecExecutor .Verb , Spec > consumedSpecAction = consumedEvent .get (0 );
209
+ Assert .assertTrue (consumedSpecAction .getKey ().equals (SpecExecutor .Verb .ADD ), "Verb did not match" );
210
+ Assert .assertTrue (consumedSpecAction .getValue ().getUri ().toString ().equals (flowSpecUriString ), "Expected URI did not match" );
211
+ Assert .assertTrue (consumedSpecAction .getValue () instanceof JobSpec , "Expected JobSpec" );
212
+
213
+ _seip .cancelJob (new URI (flowSpecUriString ), new Properties ()).get ();
214
+ Thread .sleep (5000 );
215
+ consumedEvent = _seic .changedSpecs ().get ();
216
+ Assert .assertTrue (consumedEvent .size () == 2 , "Should emit cancellation event if no flow ID provided" );
217
+ consumedSpecAction = consumedEvent .get (1 );
218
+ Assert .assertTrue (consumedEvent .get (0 ).getKey ().equals (SpecExecutor .Verb .DELETE ), "Verb did not match" );
219
+ Assert .assertTrue (consumedSpecAction .getKey ().equals (SpecExecutor .Verb .CANCEL ), "Verb did not match" );
220
+ Assert .assertTrue (consumedSpecAction .getValue ().getUri ().toString ().equals (flowSpecUriString ), "Expected URI did not match" );
221
+ Assert .assertTrue (consumedSpecAction .getValue () instanceof JobSpec , "Expected JobSpec" );
222
+ }
223
+
224
+ @ Test (dependsOnMethods = "testCancelSpecNoopDefault" )
225
+ public void testCancelSpecWithFlowExecutionId () throws Exception {
226
+ _seip .addSpec (flowSpec ).get ();
227
+ Properties props = new Properties ();
228
+ props .setProperty (ConfigurationKeys .FLOW_EXECUTION_ID_KEY , "12345" );
229
+ WriteResponse writeResponse = (WriteResponse ) _seip .cancelJob (new URI (flowSpecUriString ), props ).get ();
230
+ log .info ("WriteResponse: " + writeResponse );
231
+
232
+ // Wait for the cancellation to be processed
233
+ Thread .sleep (5000 );
234
+ List <Pair <SpecExecutor .Verb , Spec >> consumedEvent = _seic .changedSpecs ().get ();
235
+ Assert .assertTrue (consumedEvent .size () == 3 , "Consumption did not match production" );
236
+
237
+ Map .Entry <SpecExecutor .Verb , Spec > consumedSpecAction = consumedEvent .get (2 );
238
+ log .info (consumedSpecAction .getKey ().toString ());
239
+ Assert .assertTrue (consumedEvent .get (0 ).getKey ().equals (SpecExecutor .Verb .ADD ), "Verb did not match" );
240
+ Assert .assertTrue (consumedEvent .get (1 ).getKey ().equals (SpecExecutor .Verb .DELETE ), "Verb did not match" );
241
+ Assert .assertTrue (consumedSpecAction .getKey ().equals (SpecExecutor .Verb .CANCEL ), "Verb did not match" );
242
+ Assert .assertTrue (consumedSpecAction .getValue ().getUri ().toString ().equals (flowSpecUriString ), "Expected URI did not match" );
243
+ Assert .assertTrue (consumedSpecAction .getValue () instanceof JobSpec , "Expected JobSpec" );
244
+ }
245
+
246
+
168
247
private static JobSpec initJobSpec (String specUri ) {
169
248
Properties properties = new Properties ();
170
249
return JobSpec .builder (specUri )
@@ -174,6 +253,16 @@ private static JobSpec initJobSpec(String specUri) {
174
253
.build ();
175
254
}
176
255
256
+ private static JobSpec initJobSpecWithFlowExecutionId (String specUri , String flowExecutionId ) {
257
+ Properties properties = new Properties ();
258
+ properties .setProperty (ConfigurationKeys .FLOW_EXECUTION_ID_KEY , flowExecutionId );
259
+ return JobSpec .builder (specUri )
260
+ .withConfig (ConfigUtils .propertiesToConfig (properties ))
261
+ .withVersion ("1" )
262
+ .withDescription ("Spec Description" )
263
+ .build ();
264
+ }
265
+
177
266
@ AfterSuite
178
267
public void after () {
179
268
try {
0 commit comments