[ https://issues.apache.org/jira/browse/FLINK-4329?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15529627#comment-15529627 ]
ASF GitHub Bot commented on FLINK-4329: --------------------------------------- Github user StephanEwen commented on a diff in the pull request: https://github.com/apache/flink/pull/2546#discussion_r80915830 --- Diff: flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringTest.java --- @@ -106,6 +107,117 @@ public static void destroyHDFS() { // TESTS @Test + public void testFileReadingOperatorWithIngestionTime() throws Exception { + Set<org.apache.hadoop.fs.Path> filesCreated = new HashSet<>(); + Map<Integer, String> expectedFileContents = new HashMap<>(); + for(int i = 0; i < NO_OF_FILES; i++) { + Tuple2<org.apache.hadoop.fs.Path, String> file = fillWithData(hdfsURI, "file", i, "This is test line."); + filesCreated.add(file.f0); + expectedFileContents.put(i, file.f1); + } + + TextInputFormat format = new TextInputFormat(new Path(hdfsURI)); + TypeInformation<String> typeInfo = TypeExtractor.getInputFormatTypes(format); + + ContinuousFileReaderOperator<String, ?> reader = new ContinuousFileReaderOperator<>(format); + + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.setAutoWatermarkInterval(100); + + TestTimeServiceProvider timeServiceProvider = new TestTimeServiceProvider(); + OneInputStreamOperatorTestHarness<FileInputSplit, String> tester = + new OneInputStreamOperatorTestHarness<>(reader, executionConfig, + timeServiceProvider, TimeCharacteristic.IngestionTime); + + reader.setOutputType(typeInfo, executionConfig); + tester.open(); + + // test that watermarks are correctly emitted + + timeServiceProvider.setCurrentTime(201); + timeServiceProvider.setCurrentTime(301); + timeServiceProvider.setCurrentTime(401); + timeServiceProvider.setCurrentTime(501); + + int i = 0; + for(Object line: tester.getOutput()) { + if (!(line instanceof Watermark)) { + Assert.fail("Only watermarks are expected here "); + } + Watermark w = (Watermark) line; + Assert.assertEquals(w.getTimestamp(), 200 + (i * 100)); + i++; + } + + // clear the output to get the elements only and the final watermark + tester.getOutput().clear(); + Assert.assertEquals(tester.getOutput().size(), 0); + + // create the necessary splits for the test + FileInputSplit[] splits = format.createInputSplits( --- End diff -- What will the `getNumberOfParallelSubtasks()` be here? The test does not control the number of splits, but leave this to the implicit behavior of the test harness? > Fix Streaming File Source Timestamps/Watermarks Handling > -------------------------------------------------------- > > Key: FLINK-4329 > URL: https://issues.apache.org/jira/browse/FLINK-4329 > Project: Flink > Issue Type: Bug > Components: Streaming Connectors > Affects Versions: 1.1.0 > Reporter: Aljoscha Krettek > Assignee: Kostas Kloudas > Fix For: 1.2.0, 1.1.3 > > > The {{ContinuousFileReaderOperator}} does not correctly deal with watermarks, > i.e. they are just passed through. This means that when the > {{ContinuousFileMonitoringFunction}} closes and emits a {{Long.MAX_VALUE}} > that watermark can "overtake" the records that are to be emitted in the > {{ContinuousFileReaderOperator}}. Together with the new "allowed lateness" > setting in window operator this can lead to elements being dropped as late. > Also, {{ContinuousFileReaderOperator}} does not correctly assign ingestion > timestamps since it is not technically a source but looks like one to the > user. -- This message was sent by Atlassian JIRA (v6.3.4#6332)