/**
   * This test verifies that checkpoint barriers and barrier buffers work correctly with concurrent
   * checkpoint barriers where one checkpoint is "overtaking" another checkpoint, i.e. some inputs
   * receive barriers from an earlier checkpoint, thereby blocking, then all inputs receive barriers
   * from a later checkpoint.
   */
  @Test
  @SuppressWarnings("unchecked")
  public void testOvertakingCheckpointBarriers() throws Exception {
    final TwoInputStreamTask<String, Integer, String> coMapTask =
        new TwoInputStreamTask<String, Integer, String>();
    final TwoInputStreamTaskTestHarness<String, Integer, String> testHarness =
        new TwoInputStreamTaskTestHarness<String, Integer, String>(
            coMapTask,
            2,
            2,
            new int[] {1, 2},
            BasicTypeInfo.STRING_TYPE_INFO,
            BasicTypeInfo.INT_TYPE_INFO,
            BasicTypeInfo.STRING_TYPE_INFO);

    StreamConfig streamConfig = testHarness.getStreamConfig();
    CoStreamMap<String, Integer, String> coMapOperator =
        new CoStreamMap<String, Integer, String>(new IdentityMap());
    streamConfig.setStreamOperator(coMapOperator);

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

    testHarness.invoke();

    testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 0);

    // These elements should be buffered until we receive barriers from
    // all inputs
    testHarness.processElement(new StreamRecord<String>("Hello-0-0", initialTime), 0, 0);
    testHarness.processElement(new StreamRecord<String>("Ciao-0-0", initialTime), 0, 0);

    // These elements should be forwarded, since we did not yet receive a checkpoint barrier
    // on that input, only add to same input, otherwise we would not know the ordering
    // of the output since the Task might read the inputs in any order
    testHarness.processElement(new StreamRecord<Integer>(42, initialTime), 1, 1);
    testHarness.processElement(new StreamRecord<Integer>(1337, initialTime), 1, 1);
    expectedOutput.add(new StreamRecord<String>("42", initialTime));
    expectedOutput.add(new StreamRecord<String>("1337", initialTime));

    testHarness.waitForInputProcessing();
    // we should not yet see the barrier, only the two elements from non-blocked input
    TestHarnessUtil.assertOutputEquals(
        "Output was not correct.", expectedOutput, testHarness.getOutput());

    // Now give a later barrier to all inputs, this should unblock the first channel,
    // thereby allowing the two blocked elements through
    testHarness.processEvent(new CheckpointBarrier(1, 1), 0, 0);
    testHarness.processEvent(new CheckpointBarrier(1, 1), 0, 1);
    testHarness.processEvent(new CheckpointBarrier(1, 1), 1, 0);
    testHarness.processEvent(new CheckpointBarrier(1, 1), 1, 1);

    expectedOutput.add(new StreamRecord<String>("Hello-0-0", initialTime));
    expectedOutput.add(new StreamRecord<String>("Ciao-0-0", initialTime));
    expectedOutput.add(new CheckpointBarrier(1, 1));

    testHarness.waitForInputProcessing();

    TestHarnessUtil.assertOutputEquals(
        "Output was not correct.", expectedOutput, testHarness.getOutput());

    // Then give the earlier barrier, these should be ignored
    testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 1);
    testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 0);
    testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 1);

    testHarness.waitForInputProcessing();

    testHarness.endInput();

    testHarness.waitForTaskCompletion();

    TestHarnessUtil.assertOutputEquals(
        "Output was not correct.", expectedOutput, testHarness.getOutput());
  }
  /** This test verifies that checkpoint barriers are correctly forwarded. */
  @Test
  @SuppressWarnings("unchecked")
  public void testCheckpointBarriers() throws Exception {
    final TwoInputStreamTask<String, Integer, String> coMapTask =
        new TwoInputStreamTask<String, Integer, String>();
    final TwoInputStreamTaskTestHarness<String, Integer, String> testHarness =
        new TwoInputStreamTaskTestHarness<String, Integer, String>(
            coMapTask,
            2,
            2,
            new int[] {1, 2},
            BasicTypeInfo.STRING_TYPE_INFO,
            BasicTypeInfo.INT_TYPE_INFO,
            BasicTypeInfo.STRING_TYPE_INFO);

    StreamConfig streamConfig = testHarness.getStreamConfig();
    CoStreamMap<String, Integer, String> coMapOperator =
        new CoStreamMap<String, Integer, String>(new IdentityMap());
    streamConfig.setStreamOperator(coMapOperator);

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

    testHarness.invoke();

    testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 0);

    // This element should be buffered since we received a checkpoint barrier on
    // this input
    testHarness.processElement(new StreamRecord<String>("Hello-0-0", initialTime), 0, 0);

    // This one should go through
    testHarness.processElement(new StreamRecord<String>("Ciao-0-0", initialTime), 0, 1);
    expectedOutput.add(new StreamRecord<String>("Ciao-0-0", initialTime));

    // These elements should be forwarded, since we did not yet receive a checkpoint barrier
    // on that input, only add to same input, otherwise we would not know the ordering
    // of the output since the Task might read the inputs in any order
    testHarness.processElement(new StreamRecord<Integer>(11, initialTime), 1, 1);
    testHarness.processElement(new StreamRecord<Integer>(111, initialTime), 1, 1);
    expectedOutput.add(new StreamRecord<String>("11", initialTime));
    expectedOutput.add(new StreamRecord<String>("111", initialTime));

    testHarness.waitForInputProcessing();
    // we should not yet see the barrier, only the two elements from non-blocked input
    TestHarnessUtil.assertOutputEquals(
        "Output was not correct.", testHarness.getOutput(), expectedOutput);

    testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 1);
    testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 0);
    testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 1);

    testHarness.waitForInputProcessing();

    // now we should see the barrier and after that the buffered elements
    expectedOutput.add(new CheckpointBarrier(0, 0));
    expectedOutput.add(new StreamRecord<String>("Hello-0-0", initialTime));
    TestHarnessUtil.assertOutputEquals(
        "Output was not correct.", testHarness.getOutput(), expectedOutput);

    testHarness.endInput();

    testHarness.waitForTaskCompletion();

    List<String> resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput());
    Assert.assertEquals(4, resultElements.size());
  }