@Override public void run(SourceContext<Tuple2<Integer, String>> ctx) throws Exception { for (int i = 0; i < numElements && running; i++) { if (i % 5 == 0 && i > 0) { // update the clock after "five seconds", so we get 20 seconds in total // with 5 elements in each time window latch1.await(); latch2.await(); ModifyableClock.setCurrentTime(i * 1000); } ctx.collect(Tuple2.of(i, "message #" + i)); } }
/** * This uses {@link org.apache.flink.streaming.connectors.fs.DateTimeBucketer} to produce rolling * files. The clock of DateTimeBucketer is set to {@link ModifyableClock} to keep the time in * lockstep with the processing of elements using latches. */ @Test public void testDateTimeRollingStringWriter() throws Exception { final int NUM_ELEMENTS = 20; final int PARALLELISM = 2; final String outPath = hdfsURI + "/rolling-out"; DateTimeBucketer.setClock(new ModifyableClock()); ModifyableClock.setCurrentTime(0); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(PARALLELISM); DataStream<Tuple2<Integer, String>> source = env.addSource(new WaitingTestSourceFunction(NUM_ELEMENTS)).broadcast(); // the parallel flatMap is chained to the sink, so when it has seen 5 elements it can // fire the latch DataStream<String> mapped = source.flatMap( new RichFlatMapFunction<Tuple2<Integer, String>, String>() { private static final long serialVersionUID = 1L; int count = 0; @Override public void flatMap(Tuple2<Integer, String> value, Collector<String> out) throws Exception { out.collect(value.f1); count++; if (count >= 5) { if (getRuntimeContext().getIndexOfThisSubtask() == 0) { latch1.trigger(); } else { latch2.trigger(); } count = 0; } } }); RollingSink<String> sink = new RollingSink<String>(outPath) .setBucketer(new DateTimeBucketer("ss")) .setPartPrefix("part") .setPendingPrefix("") .setPendingSuffix(""); mapped.addSink(sink); env.execute("RollingSink String Write Test"); RemoteIterator<LocatedFileStatus> files = dfs.listFiles(new Path(outPath), true); // we should have 8 rolling files, 4 time intervals and parallelism of 2 int numFiles = 0; while (files.hasNext()) { LocatedFileStatus file = files.next(); numFiles++; if (file.getPath().toString().contains("rolling-out/00")) { FSDataInputStream inStream = dfs.open(file.getPath()); BufferedReader br = new BufferedReader(new InputStreamReader(inStream)); for (int i = 0; i < 5; i++) { String line = br.readLine(); Assert.assertEquals("message #" + i, line); } inStream.close(); } else if (file.getPath().toString().contains("rolling-out/05")) { FSDataInputStream inStream = dfs.open(file.getPath()); BufferedReader br = new BufferedReader(new InputStreamReader(inStream)); for (int i = 5; i < 10; i++) { String line = br.readLine(); Assert.assertEquals("message #" + i, line); } inStream.close(); } else if (file.getPath().toString().contains("rolling-out/10")) { FSDataInputStream inStream = dfs.open(file.getPath()); BufferedReader br = new BufferedReader(new InputStreamReader(inStream)); for (int i = 10; i < 15; i++) { String line = br.readLine(); Assert.assertEquals("message #" + i, line); } inStream.close(); } else if (file.getPath().toString().contains("rolling-out/15")) { FSDataInputStream inStream = dfs.open(file.getPath()); BufferedReader br = new BufferedReader(new InputStreamReader(inStream)); for (int i = 15; i < 20; i++) { String line = br.readLine(); Assert.assertEquals("message #" + i, line); } inStream.close(); } else { Assert.fail("File " + file + " does not match any expected roll pattern."); } } Assert.assertEquals(8, numFiles); }