@Test public void testTumblingTimeWindow() { final int NUM_ELEMENTS_PER_KEY = 3000; final int WINDOW_SIZE = 100; final int NUM_KEYS = 100; FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment( "localhost", cluster.getLeaderRPCPort()); env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); env.setNumberOfExecutionRetries(3); env.getConfig().disableSysoutLogging(); env.addSource(new FailingSource(NUM_KEYS, NUM_ELEMENTS_PER_KEY, NUM_ELEMENTS_PER_KEY / 3)) .rebalance() .keyBy(0) .timeWindow(Time.of(WINDOW_SIZE, MILLISECONDS)) .apply( new RichWindowFunction< Tuple2<Long, IntType>, Tuple4<Long, Long, Long, IntType>, Tuple, TimeWindow>() { private boolean open = false; @Override public void open(Configuration parameters) { assertEquals(PARALLELISM, getRuntimeContext().getNumberOfParallelSubtasks()); open = true; } @Override public void apply( Tuple tuple, TimeWindow window, Iterable<Tuple2<Long, IntType>> values, Collector<Tuple4<Long, Long, Long, IntType>> out) { // validate that the function has been opened properly assertTrue(open); int sum = 0; long key = -1; for (Tuple2<Long, IntType> value : values) { sum += value.f1.value; key = value.f0; } out.collect( new Tuple4<>(key, window.getStart(), window.getEnd(), new IntType(sum))); } }) .addSink(new ValidatingSink(NUM_KEYS, NUM_ELEMENTS_PER_KEY / WINDOW_SIZE)) .setParallelism(1); tryExecute(env, "Tumbling Window Test"); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }
@Test public void testTumblingTimeWindowWithKVState() { final int NUM_ELEMENTS_PER_KEY = 3000; final int WINDOW_SIZE = 100; final int NUM_KEYS = 100; FailingSource.reset(); try { StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment( "localhost", cluster.getLeaderRPCPort()); env.setParallelism(PARALLELISM); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); env.enableCheckpointing(100); env.setNumberOfExecutionRetries(3); env.getConfig().disableSysoutLogging(); env.addSource(new FailingSource(NUM_KEYS, NUM_ELEMENTS_PER_KEY, NUM_ELEMENTS_PER_KEY / 3)) .rebalance() .keyBy(0) .timeWindow(Time.of(WINDOW_SIZE, MILLISECONDS)) .apply( new RichWindowFunction< Tuple2<Long, IntType>, Tuple4<Long, Long, Long, IntType>, Tuple, TimeWindow>() { private boolean open = false; private OperatorState<Integer> count; @Override public void open(Configuration parameters) { assertEquals(PARALLELISM, getRuntimeContext().getNumberOfParallelSubtasks()); open = true; count = getRuntimeContext().getKeyValueState("count", Integer.class, 0); } @Override public void apply( Tuple tuple, TimeWindow window, Iterable<Tuple2<Long, IntType>> values, Collector<Tuple4<Long, Long, Long, IntType>> out) throws Exception { // the window count state starts with the key, so that we get // different count results for each key if (count.value() == 0) { count.update(tuple.<Long>getField(0).intValue()); } // validate that the function has been opened properly assertTrue(open); count.update(count.value() + 1); out.collect( new Tuple4<>( tuple.<Long>getField(0), window.getStart(), window.getEnd(), new IntType(count.value()))); } }) .addSink(new CountValidatingSink(NUM_KEYS, NUM_ELEMENTS_PER_KEY / WINDOW_SIZE)) .setParallelism(1); tryExecute(env, "Tumbling Window Test"); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } }