Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-10489

Inconsistent window information for streams with EventTime characteristic

    XMLWordPrintableJSON

Details

    Description

      Consider the following program.

      void temp() throws Exception {
          StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
          StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(execEnv);
      
          execEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
      
          DataStream<Tuple3<Timestamp, String, Double>> source = execEnv
                  .fromElements(
                          new Tuple3<>(Timestamp.valueOf("2018-09-20 22:00:00"), "a", 1.3),
                          new Tuple3<>(Timestamp.valueOf("2018-09-20 22:01:00"), "a", 2.1),
                          new Tuple3<>(Timestamp.valueOf("2018-09-20 22:02:00"), "a", 3.0),
                          new Tuple3<>(Timestamp.valueOf("2018-09-20 22:00:00"), "b", 2.2),
                          new Tuple3<>(Timestamp.valueOf("2018-09-20 22:01:00"), "b", 1.8)
                  )
                  .assignTimestampsAndWatermarks(new WaterMarker());
      
          Table table = tableEnv.fromDataStream(source, "f0, f1, f2, rowtime.rowtime")
                  .window(Slide.over("2.minutes").every("1.minute").on("rowtime").as("w"))
                  .groupBy("f1, w")
                  .select("f1, w.end - 1.minute, f2.sum");
      
          tableEnv.toAppendStream(table, Row.class).print();
          execEnv.execute();
      }
      
      public static class WaterMarker implements AssignerWithPunctuatedWatermarks<Tuple3<Timestamp, String, Double>> {
          @Nullable
          @Override
          public Watermark checkAndGetNextWatermark(Tuple3<Timestamp, String, Double> lastElement, long extractedTimestamp) {
              return new Watermark(extractedTimestamp);
          }
      
          @Override
          public long extractTimestamp(Tuple3<Timestamp, String, Double> element, long previousElementTimestamp) {
              return element.f0.getTime();
          }
      }
      

      I am seeing an output like the following, where the hour is offset by 2 hours:

      a,2018-09-20 20:00:00.0,1.3
      a,2018-09-20 20:01:00.0,3.4000000000000004
      a,2018-09-20 20:02:00.0,5.1
      a,2018-09-20 20:03:00.0,3.0
      

      My local time zone is GMT+01:00, and indeed the input timestamps are parsed with this time zone, but even if the window-end is printed as GMT, it should show 21:0x:00. In my tests, where I actually collect the output and can see the actual java.sql.Timestamp instances, I can always see this extra 1 hour offset.

      Attachments

        Activity

          People

            Unassigned Unassigned
            asardaes Alexis Sarda-Espinosa
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated: