public static void generateLongStringTupleSequence( StreamExecutionEnvironment env, String brokerConnection, String topic, int numPartitions, final int from, final int to) throws Exception { TypeInformation<Tuple2<Integer, Integer>> resultType = TypeInfoParser.parse("Tuple2<Integer, Integer>"); env.setParallelism(numPartitions); env.getConfig().disableSysoutLogging(); env.setNumberOfExecutionRetries(0); DataStream<Tuple2<Integer, Integer>> stream = env.addSource( new RichParallelSourceFunction<Tuple2<Integer, Integer>>() { private volatile boolean running = true; @Override public void run(SourceContext<Tuple2<Integer, Integer>> ctx) throws Exception { int cnt = from; int partition = getRuntimeContext().getIndexOfThisSubtask(); while (running && cnt <= to) { ctx.collect(new Tuple2<Integer, Integer>(partition, cnt)); cnt++; } } @Override public void cancel() { running = false; } }); stream.addSink( new FlinkKafkaProducer<>( topic, new TypeInformationSerializationSchema<>(resultType, env.getConfig()), FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnection), new Tuple2Partitioner(numPartitions))); env.execute("Data generator (Int, Int) stream to topic " + topic); }
@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()); } }
@Test(timeout = 60000) public void testOffsetAutocommitTest() throws Exception { final String topicName = "testOffsetAutocommit"; final int parallelism = 3; createTestTopic(topicName, parallelism, 1); StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env1.getConfig().disableSysoutLogging(); env1.setNumberOfExecutionRetries(0); env1.setParallelism(parallelism); StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); // NOTE: We are not enabling the checkpointing! env2.getConfig().disableSysoutLogging(); env2.setNumberOfExecutionRetries(0); env2.setParallelism(parallelism); // write a sequence from 0 to 99 to each of the 3 partitions. writeSequence(env1, topicName, 100, parallelism); // the readSequence operation sleeps for 20 ms between each record. // setting a delay of 25*20 = 500 for the commit interval makes // sure that we commit roughly 3-4 times while reading, however // at least once. Properties readProps = new Properties(); readProps.putAll(standardProps); readProps.setProperty("auto.commit.interval.ms", "500"); // read so that the offset can be committed to ZK readSequence(env2, readProps, parallelism, topicName, 100, 0); // get the offset CuratorFramework curatorFramework = ((KafkaTestEnvironmentImpl) kafkaServer).createCuratorClient(); long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper( curatorFramework, standardProps.getProperty("group.id"), topicName, 0); long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper( curatorFramework, standardProps.getProperty("group.id"), topicName, 1); long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper( curatorFramework, standardProps.getProperty("group.id"), topicName, 2); LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3); // ensure that the offset has been committed boolean atLeastOneOffsetSet = (o1 > 0 && o1 <= 100) || (o2 > 0 && o2 <= 100) || (o3 > 0 && o3 <= 100); assertTrue( "Expecting at least one offset to be set o1=" + o1 + " o2=" + o2 + " o3=" + o3, atLeastOneOffsetSet); // assertTrue("Offset of o1=" + o1 + " was not in range", o1 > 0 && o1 <= 100); // assertTrue("Offset of o2=" + o2 + " was not in range", o2 > 0 && o2 <= 100); // assertTrue("Offset of o3=" + o3 + " was not in range", o3 > 0 && o3 <= 100); deleteTestTopic(topicName); }
/** * Tests that offsets are properly committed to ZooKeeper and initial offsets are read from * ZooKeeper. * * <p>This test is only applicable if the Flink Kafka Consumer uses the ZooKeeperOffsetHandler. */ @Test(timeout = 60000) public void testOffsetInZookeeper() throws Exception { final String topicName = "testOffsetInZK"; final int parallelism = 3; createTestTopic(topicName, parallelism, 1); StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env1.getConfig().disableSysoutLogging(); env1.enableCheckpointing(50); env1.setNumberOfExecutionRetries(0); env1.setParallelism(parallelism); StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env2.getConfig().disableSysoutLogging(); env2.enableCheckpointing(50); env2.setNumberOfExecutionRetries(0); env2.setParallelism(parallelism); StreamExecutionEnvironment env3 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env3.getConfig().disableSysoutLogging(); env3.enableCheckpointing(50); env3.setNumberOfExecutionRetries(0); env3.setParallelism(parallelism); // write a sequence from 0 to 99 to each of the 3 partitions. writeSequence(env1, topicName, 100, parallelism); readSequence(env2, standardProps, parallelism, topicName, 100, 0); CuratorFramework curatorClient = ((KafkaTestEnvironmentImpl) kafkaServer).createCuratorClient(); long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper( curatorClient, standardProps.getProperty("group.id"), topicName, 0); long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper( curatorClient, standardProps.getProperty("group.id"), topicName, 1); long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper( curatorClient, standardProps.getProperty("group.id"), topicName, 2); LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3); assertTrue(o1 == FlinkKafkaConsumer08.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100)); assertTrue(o2 == FlinkKafkaConsumer08.OFFSET_NOT_SET || (o2 >= 0 && o2 <= 100)); assertTrue(o3 == FlinkKafkaConsumer08.OFFSET_NOT_SET || (o3 >= 0 && o3 <= 100)); LOG.info("Manipulating offsets"); // set the offset to 50 for the three partitions ZookeeperOffsetHandler.setOffsetInZooKeeper( curatorClient, standardProps.getProperty("group.id"), topicName, 0, 49); ZookeeperOffsetHandler.setOffsetInZooKeeper( curatorClient, standardProps.getProperty("group.id"), topicName, 1, 49); ZookeeperOffsetHandler.setOffsetInZooKeeper( curatorClient, standardProps.getProperty("group.id"), topicName, 2, 49); curatorClient.close(); // create new env readSequence(env3, standardProps, parallelism, topicName, 50, 50); deleteTestTopic(topicName); }
public static void generateRandomizedIntegerSequence( StreamExecutionEnvironment env, String brokerConnection, String topic, final int numPartitions, final int numElements, final boolean randomizeOrder) throws Exception { env.setParallelism(numPartitions); env.getConfig().disableSysoutLogging(); env.setNumberOfExecutionRetries(0); DataStream<Integer> stream = env.addSource( new RichParallelSourceFunction<Integer>() { private volatile boolean running = true; @Override public void run(SourceContext<Integer> ctx) { // create a sequence int[] elements = new int[numElements]; for (int i = 0, val = getRuntimeContext().getIndexOfThisSubtask(); i < numElements; i++, val += getRuntimeContext().getNumberOfParallelSubtasks()) { elements[i] = val; } // scramble the sequence if (randomizeOrder) { Random rnd = new Random(); for (int i = 0; i < elements.length; i++) { int otherPos = rnd.nextInt(elements.length); int tmp = elements[i]; elements[i] = elements[otherPos]; elements[otherPos] = tmp; } } // emit the sequence int pos = 0; while (running && pos < elements.length) { ctx.collect(elements[pos++]); } } @Override public void cancel() { running = false; } }); stream .rebalance() .addSink( new FlinkKafkaProducer<>( topic, new TypeInformationSerializationSchema<>( BasicTypeInfo.INT_TYPE_INFO, env.getConfig()), FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnection), new KafkaPartitioner() { @Override public int partition(Object key, int numPartitions) { return ((Integer) key) % numPartitions; } })); env.execute("Scrambles int sequence generator"); }