@Test
  public void testKeyValueStateInWindowFunction() {
    try {

      StatefulFunction.globalCounts.clear();

      // tumbling window that triggers every 20 milliseconds
      AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
          new AccumulatingProcessingTimeWindowOperator<>(
              new StatefulFunction(),
              identitySelector,
              IntSerializer.INSTANCE,
              IntSerializer.INSTANCE,
              50,
              50);

      TestTimeServiceProvider timerService = new TestTimeServiceProvider();

      OneInputStreamOperatorTestHarness<Integer, Integer> testHarness =
          new KeyedOneInputStreamOperatorTestHarness<>(
              op,
              new ExecutionConfig(),
              timerService,
              identitySelector,
              BasicTypeInfo.INT_TYPE_INFO);

      testHarness.open();

      timerService.setCurrentTime(0);

      testHarness.processElement(new StreamRecord<>(1));
      testHarness.processElement(new StreamRecord<>(2));

      op.processElement(new StreamRecord<>(1));
      op.processElement(new StreamRecord<>(2));
      op.processElement(new StreamRecord<>(1));
      op.processElement(new StreamRecord<>(1));
      op.processElement(new StreamRecord<>(2));
      op.processElement(new StreamRecord<>(2));

      timerService.setCurrentTime(1000);

      List<Integer> result = extractFromStreamRecords(testHarness.getOutput());
      assertEquals(8, result.size());

      Collections.sort(result);
      assertEquals(Arrays.asList(1, 1, 1, 1, 2, 2, 2, 2), result);

      assertEquals(4, StatefulFunction.globalCounts.get(1).intValue());
      assertEquals(4, StatefulFunction.globalCounts.get(2).intValue());

      testHarness.close();
      op.dispose();
    } catch (Exception e) {
      e.printStackTrace();
      fail(e.getMessage());
    }
  }
  @Test
  public void testOpenClose() throws Exception {
    KeySelector<Integer, Integer> keySelector =
        new KeySelector<Integer, Integer>() {
          @Override
          public Integer getKey(Integer value) {
            return value;
          }
        };

    StreamGroupedFold<Integer, String, Integer> operator =
        new StreamGroupedFold<>(new TestOpenCloseFoldFunction(), "init");
    operator.setOutputType(BasicTypeInfo.STRING_TYPE_INFO, new ExecutionConfig());

    OneInputStreamOperatorTestHarness<Integer, String> testHarness =
        new OneInputStreamOperatorTestHarness<>(operator);
    testHarness.configureForKeyedStream(keySelector, BasicTypeInfo.INT_TYPE_INFO);

    long initialTime = 0L;

    testHarness.open();

    testHarness.processElement(new StreamRecord<>(1, initialTime));
    testHarness.processElement(new StreamRecord<>(2, initialTime));

    testHarness.close();

    assertTrue("RichFunction methods where not called.", TestOpenCloseFoldFunction.closeCalled);
    assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0);
  }
  @Test
  public void testGroupedFold() throws Exception {

    KeySelector<Integer, String> keySelector =
        new KeySelector<Integer, String>() {

          @Override
          public String getKey(Integer value) {
            return value.toString();
          }
        };

    StreamGroupedFold<Integer, String, String> operator =
        new StreamGroupedFold<>(new MyFolder(), "100");
    operator.setOutputType(BasicTypeInfo.STRING_TYPE_INFO, new ExecutionConfig());

    OneInputStreamOperatorTestHarness<Integer, String> testHarness =
        new OneInputStreamOperatorTestHarness<>(operator);
    testHarness.configureForKeyedStream(keySelector, BasicTypeInfo.STRING_TYPE_INFO);

    long initialTime = 0L;
    ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();

    testHarness.open();

    testHarness.processElement(new StreamRecord<>(1, initialTime + 1));
    testHarness.processElement(new StreamRecord<>(1, initialTime + 2));
    testHarness.processWatermark(new Watermark(initialTime + 2));
    testHarness.processElement(new StreamRecord<>(2, initialTime + 3));
    testHarness.processElement(new StreamRecord<>(2, initialTime + 4));
    testHarness.processElement(new StreamRecord<>(3, initialTime + 5));

    expectedOutput.add(new StreamRecord<>("1001", initialTime + 1));
    expectedOutput.add(new StreamRecord<>("10011", initialTime + 2));
    expectedOutput.add(new Watermark(initialTime + 2));
    expectedOutput.add(new StreamRecord<>("1002", initialTime + 3));
    expectedOutput.add(new StreamRecord<>("10022", initialTime + 4));
    expectedOutput.add(new StreamRecord<>("1003", initialTime + 5));

    TestHarnessUtil.assertOutputEquals(
        "Output was not correct.", expectedOutput, testHarness.getOutput());
  }
  @Test
  public void checkpointRestoreWithPendingWindowSliding() {
    try {
      final int factor = 4;
      final int windowSlide = 50;
      final int windowSize = factor * windowSlide;

      TestTimeServiceProvider timerService = new TestTimeServiceProvider();

      // sliding window (200 msecs) every 50 msecs
      AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
          new AccumulatingProcessingTimeWindowOperator<>(
              validatingIdentityFunction,
              identitySelector,
              IntSerializer.INSTANCE,
              IntSerializer.INSTANCE,
              windowSize,
              windowSlide);

      OneInputStreamOperatorTestHarness<Integer, Integer> testHarness =
          new OneInputStreamOperatorTestHarness<>(op, new ExecutionConfig(), timerService);

      timerService.setCurrentTime(0);

      testHarness.setup();
      testHarness.open();

      // inject some elements
      final int numElements = 1000;
      final int numElementsFirst = 700;

      for (int i = 0; i < numElementsFirst; i++) {
        testHarness.processElement(new StreamRecord<>(i));
      }

      // draw a snapshot
      List<Integer> resultAtSnapshot = extractFromStreamRecords(testHarness.getOutput());
      int beforeSnapShot = testHarness.getOutput().size();
      StreamStateHandle state = testHarness.snapshot(1L, System.currentTimeMillis());
      int afterSnapShot = testHarness.getOutput().size();
      assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);

      assertTrue(resultAtSnapshot.size() <= factor * numElementsFirst);

      // inject the remaining elements - these should not influence the snapshot
      for (int i = numElementsFirst; i < numElements; i++) {
        testHarness.processElement(new StreamRecord<>(i));
      }

      op.dispose();

      // re-create the operator and restore the state
      op =
          new AccumulatingProcessingTimeWindowOperator<>(
              validatingIdentityFunction,
              identitySelector,
              IntSerializer.INSTANCE,
              IntSerializer.INSTANCE,
              windowSize,
              windowSlide);

      testHarness =
          new OneInputStreamOperatorTestHarness<>(op, new ExecutionConfig(), timerService);

      testHarness.setup();
      testHarness.restore(state);
      testHarness.open();

      // inject again the remaining elements
      for (int i = numElementsFirst; i < numElements; i++) {
        testHarness.processElement(new StreamRecord<>(i));
      }

      timerService.setCurrentTime(50);
      timerService.setCurrentTime(100);
      timerService.setCurrentTime(150);
      timerService.setCurrentTime(200);
      timerService.setCurrentTime(250);
      timerService.setCurrentTime(300);
      timerService.setCurrentTime(350);

      testHarness.close();
      op.dispose();

      // get and verify the result
      List<Integer> finalResult = new ArrayList<>(resultAtSnapshot);
      List<Integer> finalPartialResult = extractFromStreamRecords(testHarness.getOutput());
      finalResult.addAll(finalPartialResult);
      assertEquals(factor * numElements, finalResult.size());

      Collections.sort(finalResult);
      for (int i = 0; i < factor * numElements; i++) {
        assertEquals(i / factor, finalResult.get(i).intValue());
      }
    } catch (Exception e) {
      e.printStackTrace();
      fail(e.getMessage());
    }
  }
  @Test
  public void checkpointRestoreWithPendingWindowTumbling() {
    try {
      final int windowSize = 200;

      // tumbling window that triggers every 200 milliseconds
      AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
          new AccumulatingProcessingTimeWindowOperator<>(
              validatingIdentityFunction,
              identitySelector,
              IntSerializer.INSTANCE,
              IntSerializer.INSTANCE,
              windowSize,
              windowSize);

      TestTimeServiceProvider timerService = new TestTimeServiceProvider();

      OneInputStreamOperatorTestHarness<Integer, Integer> testHarness =
          new OneInputStreamOperatorTestHarness<>(op, new ExecutionConfig(), timerService);

      testHarness.setup();
      testHarness.open();

      timerService.setCurrentTime(0);

      // inject some elements
      final int numElementsFirst = 700;
      final int numElements = 1000;
      for (int i = 0; i < numElementsFirst; i++) {
        testHarness.processElement(new StreamRecord<>(i));
      }

      // draw a snapshot and dispose the window
      System.out.println("GOT: " + testHarness.getOutput());
      int beforeSnapShot = testHarness.getOutput().size();
      StreamStateHandle state = testHarness.snapshot(1L, System.currentTimeMillis());
      List<Integer> resultAtSnapshot = extractFromStreamRecords(testHarness.getOutput());
      int afterSnapShot = testHarness.getOutput().size();
      assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);
      assertTrue(afterSnapShot <= numElementsFirst);

      // inject some random elements, which should not show up in the state
      for (int i = 0; i < 300; i++) {
        testHarness.processElement(new StreamRecord<>(i + numElementsFirst));
      }

      op.dispose();

      // re-create the operator and restore the state
      op =
          new AccumulatingProcessingTimeWindowOperator<>(
              validatingIdentityFunction,
              identitySelector,
              IntSerializer.INSTANCE,
              IntSerializer.INSTANCE,
              windowSize,
              windowSize);

      testHarness =
          new OneInputStreamOperatorTestHarness<>(op, new ExecutionConfig(), timerService);

      testHarness.setup();
      testHarness.restore(state);
      testHarness.open();

      // inject some more elements
      for (int i = numElementsFirst; i < numElements; i++) {
        testHarness.processElement(new StreamRecord<>(i));
      }

      timerService.setCurrentTime(400);

      // get and verify the result
      List<Integer> finalResult = new ArrayList<>();
      finalResult.addAll(resultAtSnapshot);
      List<Integer> finalPartialResult = extractFromStreamRecords(testHarness.getOutput());
      finalResult.addAll(finalPartialResult);
      assertEquals(numElements, finalResult.size());

      Collections.sort(finalResult);
      for (int i = 0; i < numElements; i++) {
        assertEquals(i, finalResult.get(i).intValue());
      }
    } catch (Exception e) {
      e.printStackTrace();
      fail(e.getMessage());
    }
  }