/**
   * 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 int parallelism = 3;

    // write a sequence from 0 to 99 to each of the 3 partitions.
    final String topicName = writeSequence("testOffsetInZK", 100, parallelism, 1);

    StreamExecutionEnvironment env1 =
        StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
    env1.getConfig().disableSysoutLogging();
    env1.enableCheckpointing(50);
    env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
    env1.setParallelism(parallelism);

    StreamExecutionEnvironment env2 =
        StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
    env2.getConfig().disableSysoutLogging();
    env2.enableCheckpointing(50);
    env2.getConfig().setRestartStrategy(RestartStrategies.noRestart());
    env2.setParallelism(parallelism);

    readSequence(env1, 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 == null || (o1 >= 0 && o1 <= 100));
    assertTrue(o2 == null || (o2 >= 0 && o2 <= 100));
    assertTrue(o3 == null || (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(env2, standardProps, parallelism, topicName, 50, 50);

    deleteTestTopic(topicName);
  }
  private static JobGraph createJobGraphWithKeyedState(
      int parallelism,
      int maxParallelism,
      int numberKeys,
      int numberElements,
      boolean terminateAfterEmission,
      int checkpointingInterval) {

    StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    env.setParallelism(parallelism);
    env.getConfig().setMaxParallelism(maxParallelism);
    env.enableCheckpointing(checkpointingInterval);
    env.setRestartStrategy(RestartStrategies.noRestart());

    DataStream<Integer> input =
        env.addSource(new SubtaskIndexSource(numberKeys, numberElements, terminateAfterEmission))
            .keyBy(
                new KeySelector<Integer, Integer>() {
                  private static final long serialVersionUID = -7952298871120320940L;

                  @Override
                  public Integer getKey(Integer value) throws Exception {
                    return value;
                  }
                });

    SubtaskIndexFlatMapper.workCompletedLatch = new CountDownLatch(numberKeys);

    DataStream<Tuple2<Integer, Integer>> result =
        input.flatMap(new SubtaskIndexFlatMapper(numberElements));

    result.addSink(new CollectionSink<Tuple2<Integer, Integer>>());

    return env.getStreamGraph().getJobGraph();
  }
  private static JobGraph createJobGraphWithOperatorState(
      int parallelism, int maxParallelism, boolean partitionedOperatorState) {

    StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    env.setParallelism(parallelism);
    env.getConfig().setMaxParallelism(maxParallelism);
    env.enableCheckpointing(Long.MAX_VALUE);
    env.setRestartStrategy(RestartStrategies.noRestart());

    StateSourceBase.workStartedLatch = new CountDownLatch(1);

    DataStream<Integer> input =
        env.addSource(
            partitionedOperatorState
                ? new PartitionedStateSource()
                : new NonPartitionedStateSource());

    input.addSink(new DiscardingSink<Integer>());

    return env.getStreamGraph().getJobGraph();
  }
  /**
   * Runs the following program:
   *
   * <pre>
   *     [ (source)->(filter)->(map) ] -> [ (map) ] -> [ (groupBy/reduce)->(sink) ]
   * </pre>
   */
  @Test
  public void runCheckpointedProgram() {

    final long NUM_STRINGS = 10000000L;
    assertTrue("Broken test setup", NUM_STRINGS % 40 == 0);

    try {
      StreamExecutionEnvironment env =
          StreamExecutionEnvironment.createRemoteEnvironment(
              "localhost", cluster.getJobManagerRPCPort());
      env.setParallelism(PARALLELISM);
      env.enableCheckpointing(500);
      env.getConfig().disableSysoutLogging();

      DataStream<String> stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS));

      stream
          // -------------- first vertex, chained to the source ----------------
          .filter(new StringRichFilterFunction())

          // -------------- seconds vertex - the stateful one that also fails ----------------
          .map(new StringPrefixCountRichMapFunction())
          .startNewChain()
          .map(new StatefulCounterFunction())

          // -------------- third vertex - reducer and the sink ----------------
          .groupBy("prefix")
          .reduce(new OnceFailingReducer(NUM_STRINGS))
          .addSink(
              new RichSinkFunction<PrefixCount>() {

                private Map<Character, Long> counts = new HashMap<Character, Long>();

                @Override
                public void invoke(PrefixCount value) {
                  Character first = value.prefix.charAt(0);
                  Long previous = counts.get(first);
                  if (previous == null) {
                    counts.put(first, value.count);
                  } else {
                    counts.put(first, Math.max(previous, value.count));
                  }
                }

                //						@Override
                //						public void close() {
                //							for (Long count : counts.values()) {
                //								assertEquals(NUM_STRINGS / 40, count.longValue());
                //							}
                //						}
              });

      env.execute();

      long filterSum = 0;
      for (long l : StringRichFilterFunction.counts) {
        filterSum += l;
      }

      long mapSum = 0;
      for (long l : StringPrefixCountRichMapFunction.counts) {
        mapSum += l;
      }

      long countSum = 0;
      for (long l : StatefulCounterFunction.counts) {
        countSum += l;
      }

      // verify that we counted exactly right

      // this line should be uncommented once the "exactly one off by one" is fixed
      // if this fails we see at which point the count is off
      assertEquals(NUM_STRINGS, filterSum);
      assertEquals(NUM_STRINGS, mapSum);
      assertEquals(NUM_STRINGS, countSum);
    } catch (Exception e) {
      e.printStackTrace();
      fail(e.getMessage());
    }
  }
  @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());
    }
  }
  /**
   * This test ensures that when the consumers retrieve some start offset from kafka (earliest,
   * latest), that this offset is committed to Zookeeper, even if some partitions are not read
   *
   * <p>Test: - Create 3 topics - write 50 messages into each. - Start three consumers with
   * auto.offset.reset='latest' and wait until they committed into ZK. - Check if the offsets in ZK
   * are set to 50 for the three partitions
   *
   * <p>See FLINK-3440 as well
   */
  @Test(timeout = 60000)
  public void testKafkaOffsetRetrievalToZookeeper() throws Exception {
    final int parallelism = 3;

    // write a sequence from 0 to 49 to each of the 3 partitions.
    final String topicName = writeSequence("testKafkaOffsetToZk", 50, parallelism, 1);

    final StreamExecutionEnvironment env2 =
        StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
    env2.getConfig().disableSysoutLogging();
    env2.getConfig().setRestartStrategy(RestartStrategies.noRestart());
    env2.setParallelism(parallelism);
    env2.enableCheckpointing(200);

    Properties readProps = new Properties();
    readProps.putAll(standardProps);
    readProps.setProperty("auto.offset.reset", "latest");

    DataStream<String> stream =
        env2.addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), readProps));
    stream.addSink(new DiscardingSink<String>());

    final AtomicReference<Throwable> errorRef = new AtomicReference<>();
    final Thread runner =
        new Thread("runner") {
          @Override
          public void run() {
            try {
              env2.execute();
            } catch (Throwable t) {
              if (!(t.getCause() instanceof JobCancellationException)) {
                errorRef.set(t);
              }
            }
          }
        };
    runner.start();

    final CuratorFramework curatorFramework =
        ((KafkaTestEnvironmentImpl) kafkaServer).createCuratorClient();
    final Long l49 = 49L;

    final long deadline = 30000 + System.currentTimeMillis();
    do {
      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);

      if (l49.equals(o1) && l49.equals(o2) && l49.equals(o3)) {
        break;
      }

      Thread.sleep(100);
    } while (System.currentTimeMillis() < deadline);

    // cancel the job
    JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));

    final Throwable t = errorRef.get();
    if (t != null) {
      throw new RuntimeException("Job failed with an exception", t);
    }

    // check if offsets are correctly in ZK
    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);
    Assert.assertEquals(Long.valueOf(49L), o1);
    Assert.assertEquals(Long.valueOf(49L), o2);
    Assert.assertEquals(Long.valueOf(49L), o3);

    curatorFramework.close();
  }