@SuppressWarnings("unchecked")
  @Test
  public void testEmitWindow2() throws Exception {

    List<Tuple2<Integer, Integer>> inputs = new ArrayList<Tuple2<Integer, Integer>>();
    inputs.add(new Tuple2<Integer, Integer>(1, 1));
    inputs.add(new Tuple2<Integer, Integer>(0, 0));
    inputs.add(new Tuple2<Integer, Integer>(1, -1));
    inputs.add(new Tuple2<Integer, Integer>(1, -2));

    TestCollector<StreamWindow<Tuple2<Integer, Integer>>> collector =
        new TestCollector<StreamWindow<Tuple2<Integer, Integer>>>();
    List<StreamWindow<Tuple2<Integer, Integer>>> collected = collector.getCollected();

    WindowBuffer<Tuple2<Integer, Integer>> wb =
        new TumblingGroupedPreReducer<Tuple2<Integer, Integer>>(reducer, key, serializer)
            .sequentialID();

    wb.store(serializer.copy(inputs.get(0)));
    wb.store(serializer.copy(inputs.get(1)));
    wb.emitWindow(collector);
    wb.evict(2);

    assertSetEquals(StreamWindow.fromElements(inputs.get(0), inputs.get(1)), collected.get(0));

    wb.store(serializer.copy(inputs.get(0)));
    wb.store(serializer.copy(inputs.get(1)));
    wb.store(serializer.copy(inputs.get(2)));
    wb.emitWindow(collector);
    wb.evict(3);

    assertSetEquals(
        StreamWindow.fromElements(new Tuple2<Integer, Integer>(2, 0), inputs.get(1)),
        collected.get(1));
  }
  @Test
  public void testPreReduce3() throws Exception {
    TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();

    SlidingTimePreReducer<Integer> preReducer =
        new SlidingTimePreReducer<Integer>(
            reducer,
            serializer,
            6,
            3,
            new TimestampWrapper<Integer>(
                new Timestamp<Integer>() {

                  private static final long serialVersionUID = 1L;

                  @Override
                  public long getTimestamp(Integer value) {
                    return value;
                  }
                },
                1));

    preReducer.store(1);
    preReducer.store(2);
    preReducer.store(3);
    preReducer.emitWindow(collector);
    preReducer.store(4);
    preReducer.store(5);
    preReducer.store(6);
    preReducer.emitWindow(collector);
    preReducer.evict(3);
    preReducer.store(7);
    preReducer.store(8);
    preReducer.store(9);
    preReducer.emitWindow(collector);
    preReducer.evict(3);
    preReducer.store(10);
    preReducer.store(11);
    preReducer.store(12);
    preReducer.emitWindow(collector);
    preReducer.evict(3);
    preReducer.store(13);

    List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
    expected.add(StreamWindow.fromElements(6));
    expected.add(StreamWindow.fromElements(21));
    expected.add(StreamWindow.fromElements(39));
    expected.add(StreamWindow.fromElements(57));

    assertEquals(expected, collector.getCollected());
  }
  @Test
  public void testEmitWindow() throws Exception {

    TestCollector<StreamWindow<Integer>> collector = new TestCollector<StreamWindow<Integer>>();
    List<StreamWindow<Integer>> collected = collector.getCollected();

    WindowBuffer<Integer> wb =
        new JumpingTimePreReducer<Integer>(
            reducer,
            serializer,
            3,
            2,
            new TimestampWrapper<Integer>(
                new Timestamp<Integer>() {

                  private static final long serialVersionUID = 1L;

                  @Override
                  public long getTimestamp(Integer value) {
                    return value;
                  }
                },
                1));

    wb.store(1);
    wb.store(2);
    wb.store(3);
    wb.evict(1);
    wb.emitWindow(collector);

    assertEquals(1, collected.size());
    assertEquals(StreamWindow.fromElements(5), collected.get(0));

    wb.store(4);
    wb.store(5);

    // Nothing should happen here
    wb.evict(2);

    wb.store(6);

    wb.emitWindow(collector);
    wb.evict(2);
    wb.emitWindow(collector);
    wb.store(12);
    wb.emitWindow(collector);

    assertEquals(3, collected.size());
    assertEquals(StreamWindow.fromElements(11), collected.get(1));
    assertEquals(StreamWindow.fromElements(12), collected.get(2));
  }
  @SuppressWarnings("unchecked")
  @Test
  public void testEmitWindow() throws Exception {

    List<Tuple2<Integer, Integer>> inputs = new ArrayList<Tuple2<Integer, Integer>>();
    inputs.add(new Tuple2<Integer, Integer>(1, 1));
    inputs.add(new Tuple2<Integer, Integer>(0, 0));
    inputs.add(new Tuple2<Integer, Integer>(1, -1));
    inputs.add(new Tuple2<Integer, Integer>(1, -2));

    TestCollector<StreamWindow<Tuple2<Integer, Integer>>> collector =
        new TestCollector<StreamWindow<Tuple2<Integer, Integer>>>();
    List<StreamWindow<Tuple2<Integer, Integer>>> collected = collector.getCollected();

    WindowBuffer<Tuple2<Integer, Integer>> wb =
        new TumblingGroupedPreReducer<Tuple2<Integer, Integer>>(reducer, key, serializer);

    wb.store(serializer.copy(inputs.get(0)));
    wb.store(serializer.copy(inputs.get(1)));
    wb.emitWindow(collector);
    wb.evict(2);

    assertEquals(1, collected.size());

    assertSetEquals(
        StreamWindow.fromElements(
            new Tuple2<Integer, Integer>(1, 1), new Tuple2<Integer, Integer>(0, 0)),
        collected.get(0));

    wb.store(serializer.copy(inputs.get(0)));
    wb.store(serializer.copy(inputs.get(1)));
    wb.store(serializer.copy(inputs.get(2)));

    wb.store(serializer.copy(inputs.get(3)));

    wb.emitWindow(collector);
    wb.evict(4);

    assertEquals(2, collected.size());

    assertSetEquals(
        StreamWindow.fromElements(
            new Tuple2<Integer, Integer>(3, -2), new Tuple2<Integer, Integer>(0, 0)),
        collected.get(1));

    // Test whether function is mutating inputs or not
    assertEquals(2, reducer.allInputs.size());
    assertEquals(reducer.allInputs.get(0), inputs.get(2));
    assertEquals(reducer.allInputs.get(1), inputs.get(3));
  }
  @Test
  @SuppressWarnings("unchecked")
  public void testPreReduce1() throws Exception {
    // This ensures that the buffer is properly cleared after a burst of elements by
    // replaying the same sequence of elements with a later timestamp and expecting the same
    // result.

    TestOutput<StreamWindow<Tuple2<Integer, Integer>>> collector =
        new TestOutput<StreamWindow<Tuple2<Integer, Integer>>>();

    SlidingTimePreReducer<Tuple2<Integer, Integer>> preReducer =
        new SlidingTimePreReducer<Tuple2<Integer, Integer>>(
            tupleReducer,
            tupleType.createSerializer(new ExecutionConfig()),
            3,
            2,
            new TimestampWrapper<Tuple2<Integer, Integer>>(
                new Timestamp<Tuple2<Integer, Integer>>() {

                  private static final long serialVersionUID = 1L;

                  @Override
                  public long getTimestamp(Tuple2<Integer, Integer> value) {
                    return value.f0;
                  }
                },
                1));

    int timeOffset = 0;

    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 1, 1));
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 2, 2));
    preReducer.emitWindow(collector);
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 3, 3));
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 4, 4));
    preReducer.evict(1);
    preReducer.emitWindow(collector);
    preReducer.evict(2);
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 5, 5));
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 6, 6));
    preReducer.emitWindow(collector);
    preReducer.evict(2);
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 7, 7));
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 8, 8));
    preReducer.emitWindow(collector);
    preReducer.evict(2);
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 9, 9));
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 10, 10));
    preReducer.emitWindow(collector);
    preReducer.evict(2);
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 11, 11));
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 12, 12));
    preReducer.emitWindow(collector);
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 13, 13));

    // ensure that everything is cleared out
    preReducer.evict(100);

    timeOffset = 25; // a little while later...

    // Repeat the same sequence, this should produce the same result
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 1, 1));
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 2, 2));
    preReducer.emitWindow(collector);
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 3, 3));
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 4, 4));
    preReducer.evict(1);
    preReducer.emitWindow(collector);
    preReducer.evict(2);
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 5, 5));
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 6, 6));
    preReducer.emitWindow(collector);
    preReducer.evict(2);
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 7, 7));
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 8, 8));
    preReducer.emitWindow(collector);
    preReducer.evict(2);
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 9, 9));
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 10, 10));
    preReducer.emitWindow(collector);
    preReducer.evict(2);
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 11, 11));
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 12, 12));
    preReducer.emitWindow(collector);
    preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 13, 13));

    List<StreamWindow<Tuple2<Integer, Integer>>> expected =
        new ArrayList<StreamWindow<Tuple2<Integer, Integer>>>();
    timeOffset = 0; // rewind ...
    expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 1, 3)));
    expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 2, 9)));
    expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 4, 15)));
    expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 6, 21)));
    expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 8, 27)));
    expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 10, 33)));

    timeOffset = 25; // and back to the future ...
    expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 1, 3)));
    expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 2, 9)));
    expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 4, 15)));
    expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 6, 21)));
    expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 8, 27)));
    expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 10, 33)));

    assertEquals(expected, collector.getCollected());
  }