/** * Local configurations work in the same JVM and have no problems with improperly formatted files * on classpath (eg. directories with .class files or empty directories). Prepare files for * staging only when using remote cluster (passing the master address explicitly). */ private static void prepareFilesToStageForRemoteClusterExecution(SparkPipelineOptions options) { if (!options.getSparkMaster().matches("local\\[?\\d*\\]?")) { options.setFilesToStage( PipelineResources.prepareFilesForStaging( options.getFilesToStage(), options.getTempLocation())); } }
private static String packageDirectoriesToStage(File directoryToStage, String tmpJarLocation) { String hash = calculateDirectoryContentHash(directoryToStage); String pathForJar = getUniqueJarPath(hash, tmpJarLocation); zipDirectory(directoryToStage, pathForJar); return pathForJar; }
/** * Goes through the list of files that need to be staged on runner. Removes nonexistent * directories and packages existing ones. This is necessary for runners that require filesToStage * to be jars only. * * @param resourcesToStage list of resources that need to be staged * @param tmpJarLocation temporary directory to store the jars * @return A list of absolute paths to resources (jar files) */ public static List<String> prepareFilesForStaging( List<String> resourcesToStage, String tmpJarLocation) { return resourcesToStage .stream() .map(File::new) .filter(File::exists) .map( file -> file.isDirectory() ? packageDirectoriesToStage(file, tmpJarLocation) : file.getAbsolutePath()) .collect(Collectors.toList()); }
/** * Creates and returns a new SparkRunner with specified options. * * @param options The PipelineOptions to use when executing the job. * @return A pipeline runner that will execute with specified options. */ public static SparkRunner fromOptions(PipelineOptions options) { SparkPipelineOptions sparkOptions = PipelineOptionsValidator.validate(SparkPipelineOptions.class, options); if (sparkOptions.getFilesToStage() == null) { sparkOptions.setFilesToStage( detectClassPathResourcesToStage(SparkRunner.class.getClassLoader())); LOG.info( "PipelineOptions.filesToStage was not specified. " + "Defaulting to files from the classpath: will stage {} files. " + "Enable logging at DEBUG level to see which files will be staged.", sparkOptions.getFilesToStage().size()); LOG.debug("Classpath elements: {}", sparkOptions.getFilesToStage()); } return new SparkRunner(sparkOptions); }
/** * Construct a runner from the provided options. * * @param options Properties which configure the runner. * @return The newly created runner. */ public static FlinkRunner fromOptions(PipelineOptions options) { FlinkPipelineOptions flinkOptions = PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options); ArrayList<String> missing = new ArrayList<>(); if (flinkOptions.getAppName() == null) { missing.add("appName"); } if (missing.size() > 0) { throw new IllegalArgumentException( "Missing required values: " + Joiner.on(',').join(missing)); } if (flinkOptions.getFilesToStage() == null) { flinkOptions.setFilesToStage( detectClassPathResourcesToStage(FlinkRunner.class.getClassLoader())); LOG.info( "PipelineOptions.filesToStage was not specified. " + "Defaulting to files from the classpath: will stage {} files. " + "Enable logging at DEBUG level to see which files will be staged.", flinkOptions.getFilesToStage().size()); LOG.debug("Classpath elements: {}", flinkOptions.getFilesToStage()); } return new FlinkRunner(flinkOptions); }
/** * Construct a runner from the provided options. * * @param options Properties which configure the runner. * @return The newly created runner. */ public static FlinkRunner fromOptions(PipelineOptions options) { options.setRunner(FlinkRunner.class); FlinkPipelineOptions flinkOptions = PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options); ArrayList<String> missing = new ArrayList<>(); if (flinkOptions.getAppName() == null) { missing.add("appName"); } if (missing.size() > 0) { throw new IllegalArgumentException( "Missing required values: " + Joiner.on(',').join(missing)); } if (flinkOptions.getFilesToStage() == null) { flinkOptions.setFilesToStage( detectClassPathResourcesToStage(FlinkRunner.class.getClassLoader())); LOG.info( "PipelineOptions.filesToStage was not specified. " + "Defaulting to files from the classpath: will stage {} files. " + "Enable logging at DEBUG level to see which files will be staged.", flinkOptions.getFilesToStage().size()); LOG.debug("Classpath elements: {}", flinkOptions.getFilesToStage()); } return new FlinkRunner(flinkOptions); }
/** * Local configurations work in the same JVM and have no problems with improperly formatted files * on classpath (eg. directories with .class files or empty directories). Prepare files for * staging only when using remote cluster (passing the master address explicitly). */ private static void prepareFilesToStageForRemoteClusterExecution(FlinkPipelineOptions options) { if (!options.getFlinkMaster().matches("\\[auto\\]|\\[collection\\]|\\[local\\]")) { options.setFilesToStage( PipelineResources.prepareFilesForStaging( options.getFilesToStage(), options.getTempLocation())); } }
@Test public void detectClassPathResourceWithNonFileResources() throws Exception { String url = "http://www.google.com/all-the-secrets.jar"; URLClassLoader classLoader = new URLClassLoader(new URL[] {new URL(url)}); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Unable to convert url (" + url + ") to file."); PipelineResources.detectClassPathResourcesToStage(classLoader); }
/** * Local configurations work in the same JVM and have no problems with improperly formatted files * on classpath (eg. directories with .class files or empty directories). Prepare files for * staging only when using remote cluster (passing the master address explicitly). */ private static void prepareFilesToStageForRemoteClusterExecution(FlinkPipelineOptions options) { if (!options.getFlinkMaster().matches("\\[auto\\]|\\[collection\\]|\\[local\\]")) { options.setFilesToStage( PipelineResources.prepareFilesForStaging( options.getFilesToStage(), MoreObjects.firstNonNull( options.getTempLocation(), System.getProperty("java.io.tmpdir")))); } }
@Test public void detectClassPathResourceWithFileResources() throws Exception { File file = tmpFolder.newFile("file"); File file2 = tmpFolder.newFile("file2"); URLClassLoader classLoader = new URLClassLoader(new URL[] {file.toURI().toURL(), file2.toURI().toURL()}); assertEquals( ImmutableList.of(file.getAbsolutePath(), file2.getAbsolutePath()), PipelineResources.detectClassPathResourcesToStage(classLoader)); }
@Test public void testPackagingDirectoryResourceToJarFile() throws IOException { String directoryPath = tmpFolder.newFolder().getAbsolutePath(); String temporaryLocation = tmpFolder.newFolder().getAbsolutePath(); List<String> filesToStage = new ArrayList<>(); filesToStage.add(directoryPath); List<String> result = PipelineResources.prepareFilesForStaging(filesToStage, temporaryLocation); assertTrue(new File(result.get(0)).exists()); assertTrue(result.get(0).matches(".*\\.jar")); }
pathsToStage.addAll(detectClassPathResourcesToStage(PortableRunner.class.getClassLoader())); if (pathsToStage.isEmpty()) { throw new IllegalArgumentException("No classpath elements found.");
@Test public void testRemovingNonexistentFilesFromFilesToStage() throws IOException { String nonexistentFilePath = tmpFolder.getRoot().getPath() + "/nonexistent/file"; String existingFilePath = tmpFolder.newFile("existingFile").getAbsolutePath(); String temporaryLocation = tmpFolder.newFolder().getAbsolutePath(); List<String> filesToStage = Arrays.asList(nonexistentFilePath, existingFilePath); List<String> expectedFilesToStage = Arrays.asList(existingFilePath); List<String> result = PipelineResources.prepareFilesForStaging(filesToStage, temporaryLocation); assertThat(result, is(expectedFilesToStage)); }
pipeline, flinkOptions, detectClassPathResourcesToStage(FlinkJobInvoker.class.getClassLoader()));
@Test public void testIfThrowsWhenThereIsNoTemporaryFolderForJars() throws IOException { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Please provide temporary location for storing the jar files."); List<String> filesToStage = new ArrayList<>(); filesToStage.add(tmpFolder.newFolder().getAbsolutePath()); PipelineResources.prepareFilesForStaging(filesToStage, null); } }
flinkOptions, serverConfig.flinkConfDir, detectClassPathResourcesToStage(FlinkJobInvoker.class.getClassLoader()));
@Test public void detectClassPathResourcesWithUnsupportedClassLoader() { ClassLoader mockClassLoader = Mockito.mock(ClassLoader.class); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Unable to use ClassLoader to detect classpath elements."); PipelineResources.detectClassPathResourcesToStage(mockClassLoader); }
detectClassPathResourcesToStage(DataflowRunner.class.getClassLoader())); if (dataflowOptions.getFilesToStage().isEmpty()) { throw new IllegalArgumentException("No files to stage has been found.");