leonardBang commented on a change in pull request #11574: [FLINK-16859][table-runtime] Introduce FileSystemTableFactory URL: https://github.com/apache/flink/pull/11574#discussion_r402038867
########## File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableFactory.java ########## @@ -0,0 +1,129 @@ +/* + * 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.table.filesystem; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.core.fs.Path; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.factories.TableFactory; +import org.apache.flink.table.factories.TableFactoryService; +import org.apache.flink.table.factories.TableSinkFactory; +import org.apache.flink.table.factories.TableSourceFactory; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.table.sources.TableSource; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.configuration.ConfigOptions.key; +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR; +import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT; +import static org.apache.flink.table.descriptors.Schema.SCHEMA; + +/** + * File system {@link TableFactory}. + * + * <P>File system support: + * 1.The partition information should be in the file system path, whether it's a temporary + * table or a catalog table. + * 2.Support insert into (append) and insert overwrite. + * 3.Support static and dynamic partition inserting. + */ +public class FileSystemTableFactory implements + TableSourceFactory<BaseRow>, + TableSinkFactory<BaseRow> { + + public static final String CONNECTOR_VALUE = "filesystem"; + public static final String PATH = "path"; + + public static final ConfigOption<String> PARTITION_DEFAULT_NAME = key("partition.default-name") + .stringType() + .defaultValue("__DEFAULT_PARTITION__") + .withDescription("The default partition name in case the dynamic partition" + + " column value is null/empty string"); + + @Override + public Map<String, String> requiredContext() { + Map<String, String> context = new HashMap<>(); + context.put(CONNECTOR, CONNECTOR_VALUE); + return context; + } + + @Override + public List<String> supportedProperties() { + List<String> properties = new ArrayList<>(); + + // path + properties.add(PATH); + + // schema + properties.add(SCHEMA + ".#." + DescriptorProperties.TABLE_SCHEMA_DATA_TYPE); + properties.add(SCHEMA + ".#." + DescriptorProperties.TABLE_SCHEMA_NAME); + + properties.add(PARTITION_DEFAULT_NAME.key()); + + // format + properties.add(FORMAT); + properties.add(FORMAT + ".*"); + + return properties; + } + + @Override + public TableSource<BaseRow> createTableSource(TableSourceFactory.Context context) { + DescriptorProperties properties = new DescriptorProperties(); + properties.putProperties(context.getTable().getProperties()); Review comment: Do some basic validate properties here to align with other tableSource/Sink Factory? I think we can do validate in `FileSystemValidator`, and also move fields `CONNECTOR_VALUE、PATH、PARTITION_DEFAULT_NAME to `FileSystemValidator`, maybe this will make the class more clear? ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services