/** * Executes the remote job. * * @param streamGraph * Stream Graph to execute * @param jarFiles * List of jar file URLs to ship to the cluster * @return The result of the job execution, containing elapsed time and accumulators. */ @Override protected JobExecutionResult executeRemotely(StreamGraph streamGraph, List<URL> jarFiles) throws ProgramInvocationException { URL jarUrl; try { jarUrl = flinkILoop.writeFilesToDisk().getAbsoluteFile().toURI().toURL(); } catch (MalformedURLException e) { throw new ProgramInvocationException("Could not write the user code classes to disk.", streamGraph.getJobGraph().getJobID(), e); } List<URL> allJarFiles = new ArrayList<>(jarFiles.size() + 1); allJarFiles.addAll(jarFiles); allJarFiles.add(jarUrl); return super.executeRemotely(streamGraph, allJarFiles); }
@Override public JobExecutionResult execute(String jobName) throws Exception { final StreamGraph streamGraph = getStreamGraph(); streamGraph.setJobName(jobName); final JobGraph jobGraph = streamGraph.getJobGraph(); for (Path jarFile : jarFiles) { jobGraph.addJar(jarFile); } jobGraph.setClasspaths(new ArrayList<>(classPaths)); return jobExecutor.executeJobBlocking(jobGraph); }
public static void main(String[] args) throws Exception { ParameterTool params = ParameterTool.fromArgs(args); // define the dataflow StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(2); env.setRestartStrategy(RestartStrategies.fixedDelayRestart(10, 1000)); env.readFileStream("input/", 60000, FileMonitoringFunction.WatchType.ONLY_NEW_FILES) .addSink(new DiscardingSink<String>()); // generate a job graph final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); File jobGraphFile = new File(params.get("output", "job.graph")); try (FileOutputStream output = new FileOutputStream(jobGraphFile); ObjectOutputStream obOutput = new ObjectOutputStream(output)){ obOutput.writeObject(jobGraph); } } }
AutoCancellableJob(Deadline deadline, final ClusterClient<?> clusterClient, final StreamExecutionEnvironment env) { Preconditions.checkNotNull(env); this.clusterClient = Preconditions.checkNotNull(clusterClient); this.jobGraph = env.getStreamGraph().getJobGraph(); this.jobId = Preconditions.checkNotNull(jobGraph.getJobID()); this.deadline = deadline; }
/** * Tests that in a streaming use case where checkpointing is enabled, there is no default strategy set on the * client side. */ @Test public void testFallbackStrategyOnClientSideWhenCheckpointingEnabled() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(500); env.fromElements(1).print(); StreamGraph graph = env.getStreamGraph(); JobGraph jobGraph = graph.getJobGraph(); RestartStrategies.RestartStrategyConfiguration restartStrategy = jobGraph.getSerializedExecutionConfig().deserializeValue(getClass().getClassLoader()).getRestartStrategy(); Assert.assertNotNull(restartStrategy); Assert.assertTrue(restartStrategy instanceof RestartStrategies.FallbackRestartStrategyConfiguration); }
@Test public void testUserProvidedHashing() { StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); List<String> userHashes = Arrays.asList("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa", "bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb"); env.addSource(new NoOpSourceFunction(), "src").setUidHash(userHashes.get(0)) .map(new NoOpMapFunction()) .filter(new NoOpFilterFunction()) .keyBy(new NoOpKeySelector()) .reduce(new NoOpReduceFunction()).name("reduce").setUidHash(userHashes.get(1)); StreamGraph streamGraph = env.getStreamGraph(); int idx = 1; for (JobVertex jobVertex : streamGraph.getJobGraph().getVertices()) { List<JobVertexID> idAlternatives = jobVertex.getIdAlternatives(); Assert.assertEquals(idAlternatives.get(idAlternatives.size() - 1).toString(), userHashes.get(idx)); --idx; } }
@Test public void testInheritOverride() { // verify that we can explicitly disable inheritance of the input slot sharing groups StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); FilterFunction<Long> dummyFilter = new FilterFunction<Long>() { @Override public boolean filter(Long value) { return false; } }; DataStream<Long> src1 = env.generateSequence(1, 10).slotSharingGroup("group-1"); DataStream<Long> src2 = env.generateSequence(1, 10).slotSharingGroup("group-1"); // this should not inherit group but be in "default" src1.union(src2).filter(dummyFilter).slotSharingGroup("default"); JobGraph jobGraph = env.getStreamGraph().getJobGraph(); List<JobVertex> vertices = jobGraph.getVerticesSortedTopologicallyFromSources(); assertEquals(vertices.get(0).getSlotSharingGroup(), vertices.get(1).getSlotSharingGroup()); assertNotEquals(vertices.get(0).getSlotSharingGroup(), vertices.get(2).getSlotSharingGroup()); assertNotEquals(vertices.get(1).getSlotSharingGroup(), vertices.get(2).getSlotSharingGroup()); }
/** * Checks that in a streaming use case where checkpointing is enabled and the number * of execution retries is set to 0, restarting is deactivated. */ @Test public void testNoRestartingWhenCheckpointingAndExplicitExecutionRetriesZero() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(500); env.setNumberOfExecutionRetries(0); env.fromElements(1).print(); StreamGraph graph = env.getStreamGraph(); JobGraph jobGraph = graph.getJobGraph(); RestartStrategies.RestartStrategyConfiguration restartStrategy = jobGraph.getSerializedExecutionConfig().deserializeValue(getClass().getClassLoader()).getRestartStrategy(); Assert.assertNotNull(restartStrategy); Assert.assertTrue(restartStrategy instanceof RestartStrategies.NoRestartStrategyConfiguration); }
@Test public void testCheckpointModeTranslation() { try { // with deactivated fault tolerance, the checkpoint mode should be at-least-once StreamExecutionEnvironment deactivated = getSimpleJob(); for (JobVertex vertex : deactivated.getStreamGraph().getJobGraph().getVertices()) { assertEquals(CheckpointingMode.AT_LEAST_ONCE, new StreamConfig(vertex.getConfiguration()).getCheckpointMode()); } // with activated fault tolerance, the checkpoint mode should be by default exactly once StreamExecutionEnvironment activated = getSimpleJob(); activated.enableCheckpointing(1000L); for (JobVertex vertex : activated.getStreamGraph().getJobGraph().getVertices()) { assertEquals(CheckpointingMode.EXACTLY_ONCE, new StreamConfig(vertex.getConfiguration()).getCheckpointMode()); } // explicitly setting the mode StreamExecutionEnvironment explicit = getSimpleJob(); explicit.enableCheckpointing(1000L, CheckpointingMode.AT_LEAST_ONCE); for (JobVertex vertex : explicit.getStreamGraph().getJobGraph().getVertices()) { assertEquals(CheckpointingMode.AT_LEAST_ONCE, new StreamConfig(vertex.getConfiguration()).getCheckpointMode()); } } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
public static JobGraph stoppableJob(final StopJobSignal stopJobSignal) { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.addSource(new InfiniteSourceFunction(stopJobSignal)) .setParallelism(2) .shuffle() .addSink(new DiscardingSink<>()) .setParallelism(2); return env.getStreamGraph().getJobGraph(); }
/** * Tests that there are no collisions with two identical sources. * * <pre> * [ (src0) ] --\ * +--> [ (sink) ] * [ (src1) ] --/ * </pre> */ @Test public void testNodeHashIdenticalSources() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); env.setParallelism(4); env.disableOperatorChaining(); DataStream<String> src0 = env.addSource(new NoOpSourceFunction()); DataStream<String> src1 = env.addSource(new NoOpSourceFunction()); src0.union(src1).addSink(new NoOpSinkFunction()); JobGraph jobGraph = env.getStreamGraph().getJobGraph(); List<JobVertex> vertices = jobGraph.getVerticesSortedTopologicallyFromSources(); assertTrue(vertices.get(0).isInputVertex()); assertTrue(vertices.get(1).isInputVertex()); assertNotNull(vertices.get(0).getID()); assertNotNull(vertices.get(1).getID()); assertNotEquals(vertices.get(0).getID(), vertices.get(1).getID()); }
/** * Tests that a changed operator name does not affect the hash. */ @Test public void testChangedOperatorName() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); env.addSource(new NoOpSourceFunction(), "A").map(new NoOpMapFunction()); JobGraph jobGraph = env.getStreamGraph().getJobGraph(); JobVertexID expected = jobGraph.getVerticesAsArray()[0].getID(); env = StreamExecutionEnvironment.createLocalEnvironment(); env.addSource(new NoOpSourceFunction(), "B").map(new NoOpMapFunction()); jobGraph = env.getStreamGraph().getJobGraph(); JobVertexID actual = jobGraph.getVerticesAsArray()[0].getID(); assertEquals(expected, actual); }
@Test public void testStreaming() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); DataStream<Integer> input = env.fromCollection(inputData); input .flatMap(new NotifyingMapper()) .writeUsingOutputFormat(new DummyOutputFormat()).disableChaining(); JobGraph jobGraph = env.getStreamGraph().getJobGraph(); submitJobAndVerifyResults(jobGraph); }
/** * Tests that a manual hash at the beginning of a chain is accepted. */ @Test public void testManualHashAssignmentForStartNodeInInChain() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); env.setParallelism(4); env.addSource(new NoOpSourceFunction()).uid("source") .map(new NoOpMapFunction()) .addSink(new NoOpSinkFunction()); env.getStreamGraph().getJobGraph(); }
/** * Tests that a manual hash for an intermediate chain node is accepted. */ @Test public void testManualHashAssignmentForIntermediateNodeInChain() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); env.setParallelism(4); env.addSource(new NoOpSourceFunction()) // Intermediate chained node .map(new NoOpMapFunction()).uid("map") .addSink(new NoOpSinkFunction()); env.getStreamGraph().getJobGraph(); }
/** * Checks that in a streaming use case where checkpointing is enabled and the number * of execution retries is set to 42 and the delay to 1337, fixed delay restarting is used. */ @Test public void testFixedRestartingWhenCheckpointingAndExplicitExecutionRetriesNonZero() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(500); env.setNumberOfExecutionRetries(42); env.getConfig().setExecutionRetryDelay(1337); env.fromElements(1).print(); StreamGraph graph = env.getStreamGraph(); JobGraph jobGraph = graph.getJobGraph(); RestartStrategies.RestartStrategyConfiguration restartStrategy = jobGraph.getSerializedExecutionConfig().deserializeValue(getClass().getClassLoader()).getRestartStrategy(); Assert.assertNotNull(restartStrategy); Assert.assertTrue(restartStrategy instanceof RestartStrategies.FixedDelayRestartStrategyConfiguration); Assert.assertEquals(42, ((RestartStrategies.FixedDelayRestartStrategyConfiguration) restartStrategy).getRestartAttempts()); Assert.assertEquals(1337, ((RestartStrategies.FixedDelayRestartStrategyConfiguration) restartStrategy).getDelayBetweenAttemptsInterval().toMilliseconds()); } }
/** * Tests that there are no collisions with two identical intermediate nodes connected to the * same predecessor. * * <pre> * /-> [ (map) ] -> [ (sink) ] * [ (src) ] -+ * \-> [ (map) ] -> [ (sink) ] * </pre> */ @Test public void testNodeHashIdenticalNodes() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); env.setParallelism(4); env.disableOperatorChaining(); DataStream<String> src = env.addSource(new NoOpSourceFunction()); src.map(new NoOpMapFunction()).addSink(new NoOpSinkFunction()); src.map(new NoOpMapFunction()).addSink(new NoOpSinkFunction()); JobGraph jobGraph = env.getStreamGraph().getJobGraph(); Set<JobVertexID> vertexIds = new HashSet<>(); for (JobVertex vertex : jobGraph.getVertices()) { assertTrue(vertexIds.add(vertex.getID())); } }
/** * Tests that a collision on the manual hash throws an Exception. */ @Test(expected = IllegalArgumentException.class) public void testManualHashAssignmentCollisionThrowsException() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); env.setParallelism(4); env.disableOperatorChaining(); env.addSource(new NoOpSourceFunction()).uid("source") .map(new NoOpMapFunction()).uid("source") // Collision .addSink(new NoOpSinkFunction()); // This call is necessary to generate the job graph env.getStreamGraph().getJobGraph(); }
@Test public void testUserProvidedHashingOnChainSupported() { StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); env.addSource(new NoOpSourceFunction(), "src").setUidHash("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa") .map(new NoOpMapFunction()).setUidHash("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb") .filter(new NoOpFilterFunction()).setUidHash("cccccccccccccccccccccccccccccccc") .keyBy(new NoOpKeySelector()) .reduce(new NoOpReduceFunction()).name("reduce").setUidHash("dddddddddddddddddddddddddddddddd"); env.getStreamGraph().getJobGraph(); }
/** * Creates a streaming JobGraph from the StreamEnvironment. */ private JobGraph createJobGraph( int parallelism, int numberOfRetries, long restartDelay) { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(parallelism); env.disableOperatorChaining(); env.getConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(numberOfRetries, restartDelay)); env.getConfig().disableSysoutLogging(); DataStream<Integer> stream = env .addSource(new InfiniteTestSource()) .shuffle() .map(new StatefulCounter()); stream.addSink(new DiscardingSink<>()); return env.getStreamGraph().getJobGraph(); }