|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.flink.yarn; |
| 19 | + |
| 20 | +import org.apache.flink.configuration.ConfigConstants; |
| 21 | +import org.apache.flink.configuration.Configuration; |
| 22 | +import org.apache.flink.test.util.TestBaseUtils; |
| 23 | +import org.apache.hadoop.fs.Path; |
| 24 | +import org.junit.Assert; |
| 25 | +import org.junit.Rule; |
| 26 | +import org.junit.Test; |
| 27 | +import org.junit.rules.TemporaryFolder; |
| 28 | + |
| 29 | +import java.io.File; |
| 30 | +import java.util.ArrayList; |
| 31 | +import java.util.HashMap; |
| 32 | +import java.util.HashSet; |
| 33 | +import java.util.List; |
| 34 | +import java.util.Map; |
| 35 | +import java.util.Set; |
| 36 | + |
| 37 | +public class YarnClusterDescriptorTest { |
| 38 | + |
| 39 | + @Rule |
| 40 | + public TemporaryFolder temporaryFolder = new TemporaryFolder(); |
| 41 | + |
| 42 | + /** |
| 43 | + * Tests to ship a lib folder through the {@code YarnClusterDescriptor.addShipFiles} |
| 44 | + */ |
| 45 | + @Test |
| 46 | + public void testExplicitLibShipping() throws Exception { |
| 47 | + AbstractYarnClusterDescriptor descriptor = new YarnClusterDescriptor(); |
| 48 | + descriptor.setLocalJarPath(new Path("/path/to/flink.jar")); |
| 49 | + |
| 50 | + descriptor.setConfigurationDirectory(temporaryFolder.getRoot().getAbsolutePath()); |
| 51 | + descriptor.setConfigurationFilePath(new Path(temporaryFolder.getRoot().getPath())); |
| 52 | + descriptor.setFlinkConfiguration(new Configuration()); |
| 53 | + |
| 54 | + File libFile = temporaryFolder.newFile("libFile.jar"); |
| 55 | + File libFolder = temporaryFolder.newFolder().getAbsoluteFile(); |
| 56 | + |
| 57 | + Assert.assertFalse(descriptor.shipFiles.contains(libFile)); |
| 58 | + Assert.assertFalse(descriptor.shipFiles.contains(libFolder)); |
| 59 | + |
| 60 | + List<File> shipFiles = new ArrayList<>(); |
| 61 | + shipFiles.add(libFile); |
| 62 | + shipFiles.add(libFolder); |
| 63 | + |
| 64 | + descriptor.addShipFiles(shipFiles); |
| 65 | + |
| 66 | + Assert.assertTrue(descriptor.shipFiles.contains(libFile)); |
| 67 | + Assert.assertTrue(descriptor.shipFiles.contains(libFolder)); |
| 68 | + |
| 69 | + // only execute part of the deployment to test for shipped files |
| 70 | + Set<File> effectiveShipFiles = new HashSet<>(); |
| 71 | + descriptor.addLibFolderToShipFiles(effectiveShipFiles); |
| 72 | + |
| 73 | + Assert.assertEquals(0, effectiveShipFiles.size()); |
| 74 | + Assert.assertEquals(2, descriptor.shipFiles.size()); |
| 75 | + Assert.assertTrue(descriptor.shipFiles.contains(libFile)); |
| 76 | + Assert.assertTrue(descriptor.shipFiles.contains(libFolder)); |
| 77 | + } |
| 78 | + |
| 79 | + /** |
| 80 | + * Tests to ship a lib folder through the {@code ConfigConstants.ENV_FLINK_LIB_DIR} |
| 81 | + */ |
| 82 | + @Test |
| 83 | + public void testEnvironmentLibShipping() throws Exception { |
| 84 | + AbstractYarnClusterDescriptor descriptor = new YarnClusterDescriptor(); |
| 85 | + |
| 86 | + descriptor.setConfigurationDirectory(temporaryFolder.getRoot().getAbsolutePath()); |
| 87 | + descriptor.setConfigurationFilePath(new Path(temporaryFolder.getRoot().getPath())); |
| 88 | + descriptor.setFlinkConfiguration(new Configuration()); |
| 89 | + |
| 90 | + File libFolder = temporaryFolder.newFolder().getAbsoluteFile(); |
| 91 | + File libFile = new File(libFolder, "libFile.jar"); |
| 92 | + libFile.createNewFile(); |
| 93 | + |
| 94 | + Set<File> effectiveShipFiles = new HashSet<>(); |
| 95 | + |
| 96 | + final Map<String, String> oldEnv = System.getenv(); |
| 97 | + try { |
| 98 | + Map<String, String> env = new HashMap<>(1); |
| 99 | + env.put(ConfigConstants.ENV_FLINK_LIB_DIR, libFolder.getAbsolutePath()); |
| 100 | + TestBaseUtils.setEnv(env); |
| 101 | + // only execute part of the deployment to test for shipped files |
| 102 | + descriptor.addLibFolderToShipFiles(effectiveShipFiles); |
| 103 | + } finally { |
| 104 | + TestBaseUtils.setEnv(oldEnv); |
| 105 | + } |
| 106 | + |
| 107 | + // only add the ship the folder, not the contents |
| 108 | + Assert.assertFalse(effectiveShipFiles.contains(libFile)); |
| 109 | + Assert.assertTrue(effectiveShipFiles.contains(libFolder)); |
| 110 | + Assert.assertFalse(descriptor.shipFiles.contains(libFile)); |
| 111 | + Assert.assertFalse(descriptor.shipFiles.contains(libFolder)); |
| 112 | + } |
| 113 | + |
| 114 | +} |
0 commit comments