You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/01/13 12:04:24 UTC

[GitHub] [flink] twalthr commented on a change in pull request #18176: [FLINK-20286][connector-files] Support directory watching in filesystem table source

twalthr commented on a change in pull request #18176:
URL: https://github.com/apache/flink/pull/18176#discussion_r783890749



##########
File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemConnectorOptions.java
##########
@@ -46,6 +46,16 @@
                             "The default partition name in case the dynamic partition"
                                     + " column value is null/empty string.");
 
+    public static final ConfigOption<Duration> SOURCE_WATCH_INTERVAL =

Review comment:
       sync with option name `SOURCE_MONITOR_INTERVAL `

##########
File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemConnectorOptions.java
##########
@@ -46,6 +46,16 @@
                             "The default partition name in case the dynamic partition"
                                     + " column value is null/empty string.");
 
+    public static final ConfigOption<Duration> SOURCE_WATCH_INTERVAL =
+            key("source.monitor-interval")
+                    .durationType()
+                    .defaultValue(Duration.ofMinutes(1))
+                    .withDescription(
+                            "The interval in which the source checks for new files. "

Review comment:
       Could we document what we consider as `new files`? By file name, by modification time, how does this feature work in general? If by file name how long do we keep the info of read files in state?

##########
File path: flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSinkStreamingTest.java
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.file.table;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.TableDescriptor;
+import org.apache.flink.table.planner.runtime.utils.StreamingTestBase;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.Test;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test of the filesystem source in streaming mode. */
+public class FileSystemTableSinkStreamingTest extends StreamingTestBase {
+
+    @Test
+    public void testMonitorContinuously() throws Exception {
+        // Create temp dir
+        File testPath = TEMPORARY_FOLDER.newFolder();
+
+        // Write first csv file out
+        Files.write(
+                Paths.get(testPath.getPath(), "input_0.csv"),
+                Arrays.asList("1", "2", "3"),
+                StandardOpenOption.CREATE);
+
+        Duration monitorInterval = Duration.ofSeconds(1);
+
+        tEnv().createTable(
+                        "my_streaming_table",
+                        TableDescriptor.forConnector("filesystem")
+                                .schema(Schema.newBuilder().column("data", DataTypes.INT()).build())
+                                .format("testcsv")
+                                .option(FileSystemConnectorOptions.PATH, testPath.getPath())
+                                .option(
+                                        FileSystemConnectorOptions.SOURCE_WATCH_INTERVAL,
+                                        monitorInterval)
+                                .build());
+
+        CloseableIterator<Row> resultsIterator =
+                tEnv().sqlQuery("SELECT * FROM my_streaming_table").execute().collect();
+
+        List<Integer> actual = new ArrayList<>();
+
+        // Iterate over the first 3 rows
+        for (int i = 0; i < 3; i++) {
+            actual.add(resultsIterator.next().<Integer>getFieldAs(0));
+        }
+
+        // Write second csv file out
+        Files.write(

Review comment:
       clean up afterwards?

##########
File path: flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSinkStreamingTest.java
##########
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.file.table;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.TableDescriptor;
+import org.apache.flink.table.planner.runtime.utils.StreamingTestBase;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.Test;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test of the filesystem source in streaming mode. */
+public class FileSystemTableSinkStreamingTest extends StreamingTestBase {
+
+    @Test
+    public void testMonitorContinuously() throws Exception {
+        // Create temp dir
+        File testPath = TEMPORARY_FOLDER.newFolder();
+
+        // Write first csv file out
+        Files.write(
+                Paths.get(testPath.getPath(), "input_0.csv"),
+                Arrays.asList("1", "2", "3"),
+                StandardOpenOption.CREATE);
+
+        Duration monitorInterval = Duration.ofSeconds(1);
+
+        tEnv().createTable(
+                        "my_streaming_table",
+                        TableDescriptor.forConnector("filesystem")
+                                .schema(Schema.newBuilder().column("data", DataTypes.INT()).build())
+                                .format("testcsv")
+                                .option(FileSystemConnectorOptions.PATH, testPath.getPath())
+                                .option(
+                                        FileSystemConnectorOptions.SOURCE_WATCH_INTERVAL,
+                                        monitorInterval)
+                                .build());
+
+        CloseableIterator<Row> resultsIterator =
+                tEnv().sqlQuery("SELECT * FROM my_streaming_table").execute().collect();
+
+        List<Integer> actual = new ArrayList<>();
+
+        // Iterate over the first 3 rows
+        for (int i = 0; i < 3; i++) {
+            actual.add(resultsIterator.next().<Integer>getFieldAs(0));
+        }
+
+        // Write second csv file out
+        Files.write(
+                Paths.get(testPath.getPath(), "input_1.csv"),
+                Arrays.asList("4", "5", "6"),
+                StandardOpenOption.CREATE);
+
+        // Iterate over the next 3 rows
+        for (int i = 0; i < 3; i++) {
+            actual.add(resultsIterator.next().<Integer>getFieldAs(0));
+        }
+
+        // Close the streaming query
+        resultsIterator.close();

Review comment:
       try with resources?

##########
File path: flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java
##########
@@ -272,7 +274,17 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
     }
 
     private SourceProvider createSourceProvider(BulkFormat<RowData, FileSourceSplit> bulkFormat) {
-        return SourceProvider.of(FileSource.forBulkFileFormat(bulkFormat, paths()).build());
+        FileSource.FileSourceBuilder<RowData> fileSourceBuilder =
+                FileSource.forBulkFileFormat(bulkFormat, paths());
+
+        // If streaming, enable directory watching
+        if (tableOptions.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING) {

Review comment:
       the `tableOptions` don't contain the full Flink config. Not sure if even the `Context#getConfiguration` does. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org