Skip to content

Commit

Permalink
[FLINK-8866] [table] Create unified interfaces to configure and insta…
Browse files Browse the repository at this point in the history
…tiate TableSinks

This closes apache#6201.
  • Loading branch information
Shuyi Chen authored and twalthr committed Jul 15, 2018
1 parent 4d2a96d commit 5cfdef9
Show file tree
Hide file tree
Showing 70 changed files with 1,550 additions and 442 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,14 +23,15 @@
import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.connectors.DiscoverableTableFactory;
import org.apache.flink.table.connectors.TableSourceFactory;
import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.descriptors.KafkaValidator;
import org.apache.flink.table.descriptors.SchemaValidator;
import org.apache.flink.table.formats.DeserializationSchemaFactory;
import org.apache.flink.table.formats.TableFormatFactoryService;
import org.apache.flink.table.sources.RowtimeAttributeDescriptor;
import org.apache.flink.table.sources.TableSource;
import org.apache.flink.table.sources.TableSourceFactory;
import org.apache.flink.types.Row;

import java.util.ArrayList;
Expand Down Expand Up @@ -71,7 +72,7 @@
/**
* Factory for creating configured instances of {@link KafkaTableSource}.
*/
public abstract class KafkaTableSourceFactory implements TableSourceFactory<Row> {
public abstract class KafkaTableSourceFactory implements TableSourceFactory<Row>, DiscoverableTableFactory {

@Override
public Map<String, String> requiredContext() {
Expand Down Expand Up @@ -118,7 +119,7 @@ public List<String> supportedProperties() {
}

@Override
public TableSource<Row> create(Map<String, String> properties) {
public TableSource<Row> createTableSource(Map<String, String> properties) {
final DescriptorProperties params = new DescriptorProperties(true);
params.putProperties(properties);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,14 +22,16 @@
import org.apache.flink.formats.json.JsonRowSchemaConverter;
import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.connectors.TableFactoryService;
import org.apache.flink.table.connectors.TableSourceFactory;
import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.descriptors.FormatDescriptor;
import org.apache.flink.table.descriptors.Json;
import org.apache.flink.table.descriptors.Kafka;
import org.apache.flink.table.descriptors.Rowtime;
import org.apache.flink.table.descriptors.Schema;
import org.apache.flink.table.descriptors.TestTableSourceDescriptor;
import org.apache.flink.table.sources.TableSource;
import org.apache.flink.table.sources.TableSourceFactoryService;
import org.apache.flink.table.sources.tsextractors.ExistingField;
import org.apache.flink.table.sources.wmstrategies.AscendingTimestamps;

Expand Down Expand Up @@ -148,7 +150,11 @@ private void testTableSource(FormatDescriptor format) {
new Rowtime().timestampsFromField("time").watermarksPeriodicAscending())
.field("proc-time", Types.SQL_TIMESTAMP).proctime());

final TableSource<?> factorySource = TableSourceFactoryService.findAndCreateTableSource(testDesc);
DescriptorProperties properties = new DescriptorProperties(true);
testDesc.addProperties(properties);
final TableSource<?> factorySource =
((TableSourceFactory) TableFactoryService.find(TableSourceFactory.class, testDesc))
.createTableSource(properties.asMap());

assertEquals(builderSource, factorySource);
}
Expand Down
2 changes: 1 addition & 1 deletion flink-libraries/flink-sql-client/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -159,7 +159,7 @@ under the License.
<include>org.codehaus.commons.compiler.properties</include>
<include>org/codehaus/janino/**</include>
<include>org/codehaus/commons/**</include>
<include>META-INF/services/org.apache.flink.table.sources.TableSourceFactory</include>
<include>META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory</include>
<!-- flink-sql-client -->
<include>org/jline/**</include>
<include>com/fasterxml/jackson/**</include>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,8 @@ public class Environment {

private Deployment deployment;

private static final String NAME = "name";

public Environment() {
this.tables = Collections.emptyMap();
this.functions = Collections.emptyMap();
Expand All @@ -62,20 +64,21 @@ public Map<String, TableDescriptor> getTables() {
public void setTables(List<Map<String, Object>> tables) {
this.tables = new HashMap<>(tables.size());
tables.forEach(config -> {
if (!config.containsKey(TableDescriptorValidator.TABLE_TYPE())) {
throw new SqlClientException("The 'type' attribute of a table is missing.");
if (!config.containsKey(NAME)) {
throw new SqlClientException("The 'name' attribute of a table is missing.");
}
final Object nameObject = config.get(NAME);
if (nameObject == null || !(nameObject instanceof String) || ((String) nameObject).length() <= 0) {
throw new SqlClientException("Invalid table name '" + nameObject + "'.");
}
if (config.get(TableDescriptorValidator.TABLE_TYPE()).equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE())) {
config.remove(TableDescriptorValidator.TABLE_TYPE());
final Source s = Source.create(config);
if (this.tables.containsKey(s.getName())) {
throw new SqlClientException("Duplicate source name '" + s.getName() + "'.");
}
this.tables.put(s.getName(), s);
} else {
throw new SqlClientException(
"Invalid table 'type' attribute value, only 'source' is supported");
final String name = (String) nameObject;
final Map<String, Object> properties = new HashMap<>(config);
properties.remove(NAME);

if (this.tables.containsKey(name)) {
throw new SqlClientException("Duplicate table name '" + name + "'.");
}
this.tables.put(name, createTableDescriptor(name, properties));
});
}

Expand Down Expand Up @@ -195,4 +198,31 @@ public static Environment enrich(Environment env, Map<String, String> properties

return enrichedEnv;
}

// --------------------------------------------------------------------------------------------

/**
* Creates a table descriptor from a YAML config map.
*
* @param name name of the table
* @param config YAML config map
* @return table descriptor describing a source, sink, or both
*/
private static TableDescriptor createTableDescriptor(String name, Map<String, Object> config) {
final Object typeObject = config.get(TableDescriptorValidator.TABLE_TYPE());
if (typeObject == null || !(typeObject instanceof String)) {
throw new SqlClientException("Invalid 'type' attribute for table '" + name + "'.");
}
final String type = (String) config.get(TableDescriptorValidator.TABLE_TYPE());
final Map<String, String> normalizedConfig = ConfigUtil.normalizeYaml(config);
if (type.equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE())) {
return new Source(name, normalizedConfig);
} else if (type.equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SINK())) {
return new Sink(name, normalizedConfig);
} else if (type.equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE_SINK())) {
return new SourceSink(name, normalizedConfig);
}
throw new SqlClientException("Invalid 'type' attribute for table '" + name + "'. " +
"Only 'source', 'sink', and 'both' are supported.");
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/*
* 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.client.config;

import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.descriptors.TableSinkDescriptor;

import java.util.Map;

/**
* Configuration of a table sink.
*/
public class Sink extends TableSinkDescriptor {

private String name;
private Map<String, String> properties;

protected Sink(String name, Map<String, String> properties) {
this.name = name;
this.properties = properties;
}

public String getName() {
return name;
}

public Map<String, String> getProperties() {
return properties;
}

// --------------------------------------------------------------------------------------------

@Override
public void addProperties(DescriptorProperties properties) {
this.properties.forEach(properties::putString);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,25 +18,20 @@

package org.apache.flink.table.client.config;

import org.apache.flink.table.client.SqlClientException;
import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.descriptors.TableSourceDescriptor;

import java.util.HashMap;
import java.util.Map;

/**
* Configuration of a table source. Parses an entry in the `tables` list of an environment
* file and translates to table descriptor properties.
* Configuration of a table source.
*/
public class Source extends TableSourceDescriptor {

private String name;
private Map<String, String> properties;

private static final String NAME = "name";

private Source(String name, Map<String, String> properties) {
protected Source(String name, Map<String, String> properties) {
this.name = name;
this.properties = properties;
}
Expand All @@ -49,22 +44,6 @@ public Map<String, String> getProperties() {
return properties;
}

/**
* Creates a table source descriptor with the given config.
*/
public static Source create(Map<String, Object> config) {
if (!config.containsKey(NAME)) {
throw new SqlClientException("The 'name' attribute of a table source is missing.");
}
final Object name = config.get(NAME);
if (name == null || !(name instanceof String) || ((String) name).length() <= 0) {
throw new SqlClientException("Invalid table source name '" + name + "'.");
}
final Map<String, Object> properties = new HashMap<>(config);
properties.remove(NAME);
return new Source((String) name, ConfigUtil.normalizeYaml(properties));
}

// --------------------------------------------------------------------------------------------

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
/*
* 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.client.config;

import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.descriptors.TableDescriptor;

import java.util.Map;

/**
* Common class for all descriptors describing a table source and sink together.
*/
public class SourceSink extends TableDescriptor {

private String name;
private Map<String, String> properties;

protected SourceSink(String name, Map<String, String> properties) {
this.name = name;
this.properties = properties;
}

public String getName() {
return name;
}

public Map<String, String> getProperties() {
return properties;
}

@Override
public void addProperties(DescriptorProperties properties) {
this.properties.forEach(properties::putString);
}

public Source toSource() {
final Map<String, String> newProperties = new HashMap<>(properties);
newProperties.replace(TableDescriptorValidator.TABLE_TYPE(),
TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE());
return new Source(name, newProperties);
}

public Sink toSink() {
final Map<String, String> newProperties = new HashMap<>(properties);
newProperties.replace(TableDescriptorValidator.TABLE_TYPE(),
TableDescriptorValidator.TABLE_TYPE_VALUE_SINK());
return new Sink(name, newProperties);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ public interface Executor {
String explainStatement(SessionContext session, String statement) throws SqlExecutionException;

/**
* Submits a Flink job (detached) and returns the result descriptor.
* Submits a Flink SQL query job (detached) and returns the result descriptor.
*/
ResultDescriptor executeQuery(SessionContext session, String query) throws SqlExecutionException;

Expand Down
Loading

0 comments on commit 5cfdef9

Please sign in to comment.