@Test
  @SuppressWarnings("unchecked")
  public void testSlidingEventTimeWindowsApply() throws Exception {
    closeCalled.set(0);

    final int WINDOW_SIZE = 3;
    final int WINDOW_SLIDE = 1;

    TypeInformation<Tuple2<String, Integer>> inputType =
        TypeInfoParser.parse("Tuple2<String, Integer>");

    ListStateDescriptor<Tuple2<String, Integer>> stateDesc =
        new ListStateDescriptor<>(
            "window-contents", inputType.createSerializer(new ExecutionConfig()));

    WindowOperator<
            String,
            Tuple2<String, Integer>,
            Iterable<Tuple2<String, Integer>>,
            Tuple2<String, Integer>,
            TimeWindow>
        operator =
            new WindowOperator<>(
                SlidingEventTimeWindows.of(
                    Time.of(WINDOW_SIZE, TimeUnit.SECONDS),
                    Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
                new TimeWindow.Serializer(),
                new TupleKeySelector(),
                BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
                stateDesc,
                new InternalIterableWindowFunction<>(new RichSumReducer<TimeWindow>()),
                EventTimeTrigger.create());

    operator.setInputType(inputType, new ExecutionConfig());

    OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>>
        testHarness = new OneInputStreamOperatorTestHarness<>(operator);

    testHarness.configureForKeyedStream(new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);

    testHarness.open();

    testSlidingEventTimeWindows(testHarness);

    testHarness.close();

    Assert.assertEquals("Close was not called.", 1, closeCalled.get());
  }
  @Test
  @SuppressWarnings("unchecked")
  public void testTumblingEventTimeWindowsReduce() throws Exception {
    closeCalled.set(0);

    final int WINDOW_SIZE = 3;

    TypeInformation<Tuple2<String, Integer>> inputType =
        TypeInfoParser.parse("Tuple2<String, Integer>");

    ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc =
        new ReducingStateDescriptor<>(
            "window-contents", new SumReducer(), inputType.createSerializer(new ExecutionConfig()));

    WindowOperator<
            String,
            Tuple2<String, Integer>,
            Tuple2<String, Integer>,
            Tuple2<String, Integer>,
            TimeWindow>
        operator =
            new WindowOperator<>(
                TumblingEventTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
                new TimeWindow.Serializer(),
                new TupleKeySelector(),
                BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
                stateDesc,
                new InternalSingleValueWindowFunction<>(
                    new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()),
                EventTimeTrigger.create());

    operator.setInputType(
        TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"),
        new ExecutionConfig());

    OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>>
        testHarness = new OneInputStreamOperatorTestHarness<>(operator);

    testHarness.configureForKeyedStream(new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);

    testHarness.open();

    testTumblingEventTimeWindows(testHarness);

    testHarness.close();
  }
 @Override
 public Trigger<Object, TimeWindow> getDefaultTrigger(StreamExecutionEnvironment env) {
   return EventTimeTrigger.create();
 }