19
19
package org .apache .flink .hdfstests ;
20
20
21
21
import org .apache .flink .api .common .functions .RichMapFunction ;
22
+ import org .apache .flink .client .program .rest .RestClusterClient ;
22
23
import org .apache .flink .core .fs .FileStatus ;
23
24
import org .apache .flink .core .fs .FileSystem ;
24
25
import org .apache .flink .core .fs .Path ;
26
+ import org .apache .flink .runtime .jobgraph .JobGraph ;
27
+ import org .apache .flink .runtime .jobmaster .JobResult ;
25
28
import org .apache .flink .runtime .testutils .MiniClusterResourceConfiguration ;
26
29
import org .apache .flink .streaming .api .environment .StreamExecutionEnvironment ;
27
30
import org .apache .flink .streaming .api .functions .sink .DiscardingSink ;
@@ -120,8 +123,36 @@ public static void teardown() {
120
123
}
121
124
122
125
@ Test
123
- public void testDistributeFileViaDFS () throws Exception {
126
+ public void testDistributedFileViaDFS () throws Exception {
127
+ createJobWithRegisteredCachedFiles ().execute ("Distributed Cache Via Blob Test Program" );
128
+ }
129
+
130
+ /**
131
+ * All the Flink Standalone, Yarn, Mesos, Kubernetes sessions are using {@link RestClusterClient#submitJob(JobGraph)}
132
+ * to submit a job to an existing session. This test will cover this cases.
133
+ */
134
+ @ Test (timeout = 30000 )
135
+ public void testSubmittingJobViaRestClusterClient () throws Exception {
136
+ RestClusterClient <String > restClusterClient = new RestClusterClient <>(
137
+ MINI_CLUSTER_RESOURCE .getClientConfiguration (),
138
+ "testSubmittingJobViaRestClusterClient" );
139
+
140
+ final JobGraph jobGraph = createJobWithRegisteredCachedFiles ()
141
+ .getStreamGraph ()
142
+ .getJobGraph ();
143
+
144
+ final JobResult jobResult = restClusterClient
145
+ .submitJob (jobGraph )
146
+ .thenCompose (restClusterClient ::requestJobResult )
147
+ .get ();
148
+
149
+ final String messageInCaseOfFailure = jobResult .getSerializedThrowable ().isPresent () ?
150
+ jobResult .getSerializedThrowable ().get ().getFullStringifiedStackTrace ()
151
+ : "Job failed." ;
152
+ assertTrue (messageInCaseOfFailure , jobResult .isSuccess ());
153
+ }
124
154
155
+ private StreamExecutionEnvironment createJobWithRegisteredCachedFiles () {
125
156
final StreamExecutionEnvironment env = StreamExecutionEnvironment .getExecutionEnvironment ();
126
157
env .setParallelism (1 );
127
158
@@ -131,8 +162,7 @@ public void testDistributeFileViaDFS() throws Exception {
131
162
env .fromElements (1 )
132
163
.map (new TestMapFunction ())
133
164
.addSink (new DiscardingSink <>());
134
-
135
- env .execute ("Distributed Cache Via Blob Test Program" );
165
+ return env ;
136
166
}
137
167
138
168
private static class TestMapFunction extends RichMapFunction <Integer , String > {
0 commit comments