Skip to content

Commit

Permalink
- Add unified table sink instantiation.
Browse files Browse the repository at this point in the history
- Consolidate table sink and table source instantiation.
- Add support to register a Calcite table with both tableSource and tableSink.
- Add Insert command support in SQL client.
- Add CsvTableSinkFactory.
  • Loading branch information
Shuyi Chen committed Jun 22, 2018
1 parent ae8cef3 commit a7ce69b
Show file tree
Hide file tree
Showing 78 changed files with 1,489 additions and 369 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,13 +21,14 @@
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.connector.TableConnectorFactory;
import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.descriptors.FormatDescriptorValidator;
import org.apache.flink.table.descriptors.KafkaValidator;
import org.apache.flink.table.descriptors.SchemaValidator;
import org.apache.flink.table.descriptors.TableDescriptorValidator;
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 @@ -68,7 +69,12 @@
/**
* Factory for creating configured instances of {@link KafkaJsonTableSource}.
*/
abstract class KafkaTableSourceFactory implements TableSourceFactory<Row> {
abstract class KafkaTableSourceFactory implements TableConnectorFactory<TableSource<Row>> {

@Override
public String tableType() {
return TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE();
}

@Override
public Map<String, String> requiredContext() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,13 +22,13 @@
import org.apache.flink.formats.avro.generated.Address;
import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.connector.TableSourceFactoryService;
import org.apache.flink.table.descriptors.Avro;
import org.apache.flink.table.descriptors.FormatDescriptor;
import org.apache.flink.table.descriptors.Kafka;
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.junit.Test;

Expand Down Expand Up @@ -116,7 +116,8 @@ private void testTableSource(FormatDescriptor format) {
.field("zip", Types.STRING)
.field("proctime", Types.SQL_TIMESTAMP).proctime());

final TableSource<?> factorySource = TableSourceFactoryService.findAndCreateTableSource(testDesc);
final TableSource<?> factorySource =
(TableSource<?>) TableSourceFactoryService.findAndCreateTableConnector(testDesc);

assertEquals(builderSource, factorySource);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,14 +22,14 @@
import org.apache.flink.formats.json.JsonSchemaConverter;
import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.connector.TableSourceFactoryService;
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.PreserveWatermarks;

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

final TableSource<?> factorySource = TableSourceFactoryService.findAndCreateTableSource(testDesc);
final TableSource<?> factorySource =
(TableSource<?>) TableSourceFactoryService.findAndCreateTableConnector(testDesc);

assertEquals(builderSource, factorySource);
}
Expand Down
6 changes: 3 additions & 3 deletions 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.connector.TableConnectorFactory</include>
<!-- flink-sql-client -->
<include>org/jline/**</include>
<include>com/fasterxml/jackson/**</include>
Expand All @@ -177,13 +177,13 @@ under the License.
<version>2.4</version>
<executions>
<execution>
<id>create-table-source-factory-jar</id>
<id>create-table-connector-factory-jar</id>
<phase>process-test-classes</phase>
<goals>
<goal>single</goal>
</goals>
<configuration>
<finalName>table-source-factory</finalName>
<finalName>table-connector-factory</finalName>
<attach>false</attach>
<descriptors>
<descriptor>src/test/assembly/test-table-source-factory.xml</descriptor>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -210,6 +210,9 @@ public void open() {
case SELECT:
callSelect(cmdCall);
break;
case INSERT:
callInsert(cmdCall);
break;
case SOURCE:
callSource(cmdCall);
break;
Expand Down Expand Up @@ -335,6 +338,10 @@ private void callSelect(SqlCommandCall cmdCall) {
}
}

private void callInsert(SqlCommandCall cmdCall) {
executor.executeUpdate(context, cmdCall.operands[0]);
}

private void callSource(SqlCommandCall cmdCall) {
final String pathString = cmdCall.operands[0];

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@ private static String[] splitOperands(SqlCommand cmd, String originalCall, Strin
return new String[] {operands.substring(0, delimiter), operands.substring(delimiter + 1)};
}
case SELECT:
case INSERT:
return new String[] {originalCall};
default:
return new String[] {operands};
Expand All @@ -89,6 +90,7 @@ enum SqlCommand {
DESCRIBE("describe"),
EXPLAIN("explain"),
SELECT("select"),
INSERT("insert"),
SET("set"),
RESET("reset"),
SOURCE("source");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,8 @@ public class Environment {

private Deployment deployment;

private static final String NAME = "name";

public Environment() {
this.tables = Collections.emptyMap();
this.execution = new Execution();
Expand All @@ -56,23 +58,44 @@ public Map<String, TableDescriptor> getTables() {
return tables;
}

private static TableDescriptor create(String name, Map<String, Object> config) {
if (!config.containsKey(TableDescriptorValidator.TABLE_TYPE())) {
throw new SqlClientException("The 'type' attribute of a table is missing.");
}
final String tableType = (String) config.get(TableDescriptorValidator.TABLE_TYPE());
if (tableType.equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE())) {
return new Source(name, ConfigUtil.normalizeYaml(config));
} else if (tableType.equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SINK())) {
return new Sink(name, ConfigUtil.normalizeYaml(config));
} else if (tableType.equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE_SINK())) {
return new SourceSink(name, ConfigUtil.normalizeYaml(config));
}
return null;
}

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.");
}
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 + "'.");
}
this.tables.put(s.getName(), s);
} else {
final Object name = config.get(NAME);
if (name == null || !(name instanceof String) || ((String) name).length() <= 0) {
throw new SqlClientException("Invalid table name '" + name + "'.");
}
final String tableName = (String) name;
final Map<String, Object> properties = new HashMap<>(config);
properties.remove(NAME);

TableDescriptor tableDescriptor = create(tableName, properties);
if (null == tableDescriptor) {
throw new SqlClientException(
"Invalid table 'type' attribute value, only 'source' is supported");
"Invalid table 'type' attribute value, only 'source' or 'sink' is supported");
}
if (this.tables.containsKey(tableName)) {
throw new SqlClientException("Duplicate table name '" + tableName + "'.");
}
this.tables.put(tableName, tableDescriptor);
});
}

Expand Down
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. Parses an entry in the `tables` list of an environment
* file and translates to table descriptor properties.
*/
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,11 +18,9 @@

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;

/**
Expand All @@ -34,9 +32,7 @@ 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,19 +45,6 @@ public Map<String, String> getProperties() {
return properties;
}

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,60 @@
/*
* 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.TableDescriptor;
import org.apache.flink.table.descriptors.TableDescriptorValidator;

import java.util.HashMap;
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;
}

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 @@ -57,10 +57,15 @@ 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;

/**
* Submits a Flink SQL update job (detached).
*/
void executeUpdate(SessionContext session, String query) throws SqlExecutionException;

/**
* Asks for the next changelog results (non-blocking).
*/
Expand Down
Loading

0 comments on commit a7ce69b

Please sign in to comment.