[
https://issues.apache.org/jira/browse/FLINK-4329?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15548388#comment-15548388
]
ASF GitHub Bot commented on FLINK-4329:
---------------------------------------
Github user mxm commented on a diff in the pull request:
https://github.com/apache/flink/pull/2593#discussion_r81945670
--- Diff:
flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringTest.java
---
@@ -106,6 +107,155 @@ 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);
+
+ final long watermarkInterval = 10;
+ ExecutionConfig executionConfig = new ExecutionConfig();
+ executionConfig.setAutoWatermarkInterval(watermarkInterval);
+
+ ContinuousFileReaderOperator<String, ?> reader = new
ContinuousFileReaderOperator<>(format);
+ reader.setOutputType(typeInfo, executionConfig);
+
+ final TestTimeServiceProvider timeServiceProvider = new
TestTimeServiceProvider();
+ final OneInputStreamOperatorTestHarness<FileInputSplit, String>
tester =
+ new OneInputStreamOperatorTestHarness<>(reader,
executionConfig, timeServiceProvider);
+ tester.setTimeCharacteristic(TimeCharacteristic.IngestionTime);
+ tester.open();
+
+ Assert.assertEquals(TimeCharacteristic.IngestionTime,
tester.getTimeCharacteristic());
+
+ // 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(200 + (i * 100), w.getTimestamp());
+ i++;
--- End diff --
I think this can be more easily readable if you break it up into:
```java
timeServiceProvider.setCurrentTime(201);
Assert.assertEquals(200, ((Watermark)
tester.getOutput().poll()).getTimestamp());
// ....
```
> 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)