19
19
package org .apache .flink .runtime .webmonitor .handlers .utils ;
20
20
21
21
import org .apache .flink .api .common .JobID ;
22
+ import org .apache .flink .client .program .PackagedProgram ;
23
+ import org .apache .flink .configuration .Configuration ;
22
24
import org .apache .flink .configuration .CoreOptions ;
25
+ import org .apache .flink .configuration .PipelineOptions ;
23
26
import org .apache .flink .runtime .rest .handler .HandlerRequest ;
24
27
import org .apache .flink .runtime .rest .handler .RestHandlerException ;
25
28
import org .apache .flink .runtime .webmonitor .handlers .JarPlanRequestBody ;
31
34
import org .slf4j .Logger ;
32
35
import org .slf4j .LoggerFactory ;
33
36
37
+ import javax .annotation .Nullable ;
38
+
39
+ import java .net .URL ;
34
40
import java .nio .file .Path ;
35
41
import java .util .Arrays ;
36
42
import java .util .Collections ;
37
43
import java .util .List ;
44
+ import java .util .Map ;
45
+ import java .util .stream .Collectors ;
38
46
47
+ import static org .apache .flink .core .testutils .FlinkAssertions .anyCauseMatches ;
39
48
import static org .assertj .core .api .Assertions .assertThat ;
49
+ import static org .assertj .core .api .Assertions .assertThatThrownBy ;
40
50
41
51
/** Tests for {@link JarHandlerUtils}. */
42
52
class JarHandlerUtilsTest {
43
53
44
54
private static final Logger LOG = LoggerFactory .getLogger (JarHandlerUtilsTest .class );
45
55
56
+ @ TempDir private Path tempDir ;
57
+
46
58
@ Test
47
59
void testTokenizeNonQuoted () {
48
60
final List <String > arguments = JarHandlerUtils .tokenizeArguments ("--foo bar" );
@@ -65,17 +77,14 @@ void testTokenizeDoubleQuoted() {
65
77
}
66
78
67
79
@ Test
68
- void testFromRequestDefaults (@ TempDir Path tmp ) throws RestHandlerException {
69
- final JarRunMessageParameters parameters =
70
- JarRunHeaders .getInstance ().getUnresolvedMessageParameters ();
71
-
72
- parameters .jarIdPathParameter .resolve ("someJar" );
80
+ void testFromRequestDefaults () throws Exception {
81
+ final JarRunMessageParameters parameters = getDummyMessageParameters ();
73
82
74
83
final HandlerRequest <JarPlanRequestBody > request =
75
84
HandlerRequest .create (new JarPlanRequestBody (), parameters );
76
85
77
86
final JarHandlerUtils .JarHandlerContext jarHandlerContext =
78
- JarHandlerUtils .JarHandlerContext .fromRequest (request , tmp , LOG );
87
+ JarHandlerUtils .JarHandlerContext .fromRequest (request , tempDir , LOG );
79
88
assertThat (jarHandlerContext .getEntryClass ()).isNull ();
80
89
assertThat (jarHandlerContext .getProgramArgs ()).isEmpty ();
81
90
assertThat (jarHandlerContext .getParallelism ())
@@ -84,25 +93,12 @@ void testFromRequestDefaults(@TempDir Path tmp) throws RestHandlerException {
84
93
}
85
94
86
95
@ Test
87
- void testFromRequestRequestBody (@ TempDir Path tmp ) throws RestHandlerException {
88
- final JarRunMessageParameters parameters =
89
- JarRunHeaders .getInstance ().getUnresolvedMessageParameters ();
90
-
91
- parameters .jarIdPathParameter .resolve ("someJar" );
92
-
93
- final JarPlanRequestBody requestBody =
94
- new JarPlanRequestBody (
95
- "entry-class" ,
96
- null ,
97
- Arrays .asList ("arg1" , "arg2" ),
98
- 37 ,
99
- JobID .generate (),
100
- null );
101
- final HandlerRequest <JarPlanRequestBody > request =
102
- HandlerRequest .create (requestBody , parameters );
96
+ void testFromRequestRequestBody () throws Exception {
97
+ final JarPlanRequestBody requestBody = getDummyJarPlanRequestBody ("entry-class" , 37 , null );
98
+ final HandlerRequest <JarPlanRequestBody > request = getDummyRequest (requestBody );
103
99
104
100
final JarHandlerUtils .JarHandlerContext jarHandlerContext =
105
- JarHandlerUtils .JarHandlerContext .fromRequest (request , tmp , LOG );
101
+ JarHandlerUtils .JarHandlerContext .fromRequest (request , tempDir , LOG );
106
102
assertThat (jarHandlerContext .getEntryClass ()).isEqualTo (requestBody .getEntryClassName ());
107
103
assertThat (jarHandlerContext .getProgramArgs ())
108
104
.containsExactlyElementsOf (requestBody .getProgramArgumentsList ());
@@ -111,28 +107,91 @@ void testFromRequestRequestBody(@TempDir Path tmp) throws RestHandlerException {
111
107
}
112
108
113
109
@ Test
114
- void testFromRequestWithParallelismConfig (@ TempDir Path tmp ) throws RestHandlerException {
110
+ void testFromRequestWithParallelismConfig () throws Exception {
115
111
final int parallelism = 37 ;
116
- final JarRunMessageParameters parameters =
117
- JarRunHeaders .getInstance ().getUnresolvedMessageParameters ();
118
-
119
- parameters .jarIdPathParameter .resolve ("someJar" );
120
-
121
112
final JarPlanRequestBody requestBody =
122
- new JarPlanRequestBody (
113
+ getDummyJarPlanRequestBody (
123
114
"entry-class" ,
124
115
null ,
125
- Arrays .asList ("arg1" , "arg2" ),
126
- null ,
127
- JobID .generate (),
128
116
Collections .singletonMap (
129
117
CoreOptions .DEFAULT_PARALLELISM .key (),
130
118
String .valueOf (parallelism )));
131
- final HandlerRequest <JarPlanRequestBody > request =
132
- HandlerRequest .create (requestBody , parameters );
119
+ final HandlerRequest <JarPlanRequestBody > request = getDummyRequest (requestBody );
133
120
134
121
final JarHandlerUtils .JarHandlerContext jarHandlerContext =
135
- JarHandlerUtils .JarHandlerContext .fromRequest (request , tmp , LOG );
122
+ JarHandlerUtils .JarHandlerContext .fromRequest (request , tempDir , LOG );
136
123
assertThat (jarHandlerContext .getParallelism ()).isEqualTo (parallelism );
137
124
}
125
+
126
+ @ Test
127
+ void testClasspathsConfigNotErased () throws Exception {
128
+ final JarPlanRequestBody requestBody =
129
+ getDummyJarPlanRequestBody (
130
+ null ,
131
+ null ,
132
+ Collections .singletonMap (
133
+ PipelineOptions .CLASSPATHS .key (),
134
+ "file:/tmp/some.jar;file:/tmp/another.jar" ));
135
+
136
+ final HandlerRequest <JarPlanRequestBody > request = getDummyRequest (requestBody );
137
+
138
+ final JarHandlerUtils .JarHandlerContext jarHandlerContext =
139
+ JarHandlerUtils .JarHandlerContext .fromRequest (request , tempDir , LOG );
140
+
141
+ final Configuration originalConfig = request .getRequestBody ().getFlinkConfiguration ();
142
+ final PackagedProgram .Builder builder =
143
+ jarHandlerContext .initPackagedProgramBuilder (originalConfig );
144
+ final List <String > retrievedClasspaths =
145
+ builder .getUserClassPaths ().stream ()
146
+ .map (URL ::toString )
147
+ .collect (Collectors .toList ());
148
+
149
+ assertThat (retrievedClasspaths ).isEqualTo (originalConfig .get (PipelineOptions .CLASSPATHS ));
150
+ }
151
+
152
+ @ Test
153
+ void testMalformedClasspathsConfig () throws Exception {
154
+ final JarPlanRequestBody requestBody =
155
+ getDummyJarPlanRequestBody (
156
+ null ,
157
+ null ,
158
+ Collections .singletonMap (
159
+ PipelineOptions .CLASSPATHS .key (), "invalid|:/jar" ));
160
+ final HandlerRequest <JarPlanRequestBody > request = getDummyRequest (requestBody );
161
+
162
+ final JarHandlerUtils .JarHandlerContext jarHandlerContext =
163
+ JarHandlerUtils .JarHandlerContext .fromRequest (request , tempDir , LOG );
164
+
165
+ final Configuration originalConfig = request .getRequestBody ().getFlinkConfiguration ();
166
+
167
+ assertThatThrownBy (() -> jarHandlerContext .initPackagedProgramBuilder (originalConfig ))
168
+ .satisfies (anyCauseMatches (RestHandlerException .class , "invalid|:/jar" ));
169
+ }
170
+
171
+ private HandlerRequest <JarPlanRequestBody > getDummyRequest (
172
+ @ Nullable JarPlanRequestBody requestBody ) {
173
+ return HandlerRequest .create (
174
+ requestBody == null ? new JarPlanRequestBody () : requestBody ,
175
+ getDummyMessageParameters ());
176
+ }
177
+
178
+ private JarRunMessageParameters getDummyMessageParameters () {
179
+ final JarRunMessageParameters parameters =
180
+ JarRunHeaders .getInstance ().getUnresolvedMessageParameters ();
181
+
182
+ parameters .jarIdPathParameter .resolve ("someJar" );
183
+
184
+ return parameters ;
185
+ }
186
+
187
+ private JarPlanRequestBody getDummyJarPlanRequestBody (
188
+ String entryClass , Integer parallelism , Map <String , String > flinkConfiguration ) {
189
+ return new JarPlanRequestBody (
190
+ entryClass ,
191
+ null ,
192
+ Arrays .asList ("arg1" , "arg2" ),
193
+ parallelism ,
194
+ JobID .generate (),
195
+ flinkConfiguration );
196
+ }
138
197
}
0 commit comments