|
|
|
@ -22,6 +22,8 @@ import org.apache.flink.configuration.ConfigConstants;
|
|
|
|
|
import org.apache.flink.util.IOUtils;
|
|
|
|
|
import org.apache.flink.yarn.configuration.YarnConfigOptions;
|
|
|
|
|
|
|
|
|
|
import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap;
|
|
|
|
|
|
|
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
|
|
import org.apache.hadoop.fs.LocalFileSystem;
|
|
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
|
@ -75,6 +77,30 @@ class YarnApplicationFileUploaderTest {
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
void testRegisterProvidedLocalResourcesWithParentDir(@TempDir File flinkLibDir)
|
|
|
|
|
throws IOException {
|
|
|
|
|
final String xmlContent = "XML Content";
|
|
|
|
|
final Map<String, String> xmlResources =
|
|
|
|
|
ImmutableMap.of(
|
|
|
|
|
"conf/hive-site.xml", xmlContent, "conf/ivysettings.xml", xmlContent);
|
|
|
|
|
generateFilesInDirectory(flinkLibDir, xmlResources);
|
|
|
|
|
|
|
|
|
|
try (final YarnApplicationFileUploader yarnApplicationFileUploader =
|
|
|
|
|
YarnApplicationFileUploader.from(
|
|
|
|
|
FileSystem.get(new YarnConfiguration()),
|
|
|
|
|
new Path(flinkLibDir.toURI()),
|
|
|
|
|
Collections.singletonList(new Path(flinkLibDir.toURI())),
|
|
|
|
|
ApplicationId.newInstance(0, 0),
|
|
|
|
|
DFSConfigKeys.DFS_REPLICATION_DEFAULT)) {
|
|
|
|
|
|
|
|
|
|
List<String> classPath = yarnApplicationFileUploader.registerProvidedLocalResources();
|
|
|
|
|
List<String> expectedClassPathEntries = Arrays.asList("conf");
|
|
|
|
|
|
|
|
|
|
assertThat(classPath).containsExactlyInAnyOrderElementsOf(expectedClassPathEntries);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
void testRegisterProvidedLocalResourcesWithDuplication(@TempDir java.nio.file.Path tempDir)
|
|
|
|
|
throws IOException {
|
|
|
|
|