Skip to content

Commit

Permalink
[FLINK-11067][table] port Schema, Rowtime, SchematicDescriptor, Regis…
Browse files Browse the repository at this point in the history
…trableDescriptor etc.
  • Loading branch information
hequn8128 authored and dawidwys committed Apr 25, 2019
1 parent c343bb7 commit 254392c
Show file tree
Hide file tree
Showing 38 changed files with 722 additions and 620 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -81,9 +81,9 @@
import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_TYPE_VALUE_ELASTICSEARCH;
import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT;
import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_TYPE;
import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA;
import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME;
import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE;
import static org.apache.flink.table.descriptors.Schema.SCHEMA;
import static org.apache.flink.table.descriptors.Schema.SCHEMA_NAME;
import static org.apache.flink.table.descriptors.Schema.SCHEMA_TYPE;
import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE;
import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE_VALUE_APPEND;

Expand Down Expand Up @@ -136,8 +136,8 @@ public List<String> supportedProperties() {
properties.add(CONNECTOR_CONNECTION_PATH_PREFIX);

// schema
properties.add(SCHEMA() + ".#." + SCHEMA_TYPE());
properties.add(SCHEMA() + ".#." + SCHEMA_NAME());
properties.add(SCHEMA + ".#." + SCHEMA_TYPE);
properties.add(SCHEMA + ".#." + SCHEMA_NAME);

// format wildcard
properties.add(FORMAT + ".*");
Expand All @@ -151,7 +151,7 @@ public StreamTableSink<Tuple2<Boolean, Row>> createStreamTableSink(Map<String, S

return createElasticsearchUpsertTableSink(
descriptorProperties.isValue(UPDATE_MODE, UPDATE_MODE_VALUE_APPEND),
descriptorProperties.getTableSchema(SCHEMA()),
descriptorProperties.getTableSchema(SCHEMA),
getHosts(descriptorProperties),
descriptorProperties.getString(CONNECTOR_INDEX),
descriptorProperties.getString(CONNECTOR_DOCUMENT_TYPE),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,19 +66,19 @@
import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_STARTUP_MODE;
import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_TOPIC;
import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_TYPE_VALUE_KAFKA;
import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_CLASS;
import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_FROM;
import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_SERIALIZED;
import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_TYPE;
import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_CLASS;
import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_DELAY;
import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_SERIALIZED;
import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_TYPE;
import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA;
import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_FROM;
import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME;
import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_PROCTIME;
import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE;
import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_CLASS;
import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_FROM;
import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_SERIALIZED;
import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_TYPE;
import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_WATERMARKS_CLASS;
import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_WATERMARKS_DELAY;
import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_WATERMARKS_SERIALIZED;
import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_WATERMARKS_TYPE;
import static org.apache.flink.table.descriptors.Schema.SCHEMA;
import static org.apache.flink.table.descriptors.Schema.SCHEMA_FROM;
import static org.apache.flink.table.descriptors.Schema.SCHEMA_NAME;
import static org.apache.flink.table.descriptors.Schema.SCHEMA_PROCTIME;
import static org.apache.flink.table.descriptors.Schema.SCHEMA_TYPE;
import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE;
import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE_VALUE_APPEND;

Expand Down Expand Up @@ -115,20 +115,20 @@ public List<String> supportedProperties() {
properties.add(CONNECTOR_SINK_PARTITIONER_CLASS);

// schema
properties.add(SCHEMA() + ".#." + SCHEMA_TYPE());
properties.add(SCHEMA() + ".#." + SCHEMA_NAME());
properties.add(SCHEMA() + ".#." + SCHEMA_FROM());
properties.add(SCHEMA + ".#." + SCHEMA_TYPE);
properties.add(SCHEMA + ".#." + SCHEMA_NAME);
properties.add(SCHEMA + ".#." + SCHEMA_FROM);

// time attributes
properties.add(SCHEMA() + ".#." + SCHEMA_PROCTIME());
properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_TYPE());
properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_FROM());
properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_CLASS());
properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_SERIALIZED());
properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_TYPE());
properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_CLASS());
properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_SERIALIZED());
properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_DELAY());
properties.add(SCHEMA + ".#." + SCHEMA_PROCTIME);
properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_TYPE);
properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_FROM);
properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_CLASS);
properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_SERIALIZED);
properties.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_TYPE);
properties.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_CLASS);
properties.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_SERIALIZED);
properties.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_DELAY);

// format wildcard
properties.add(FORMAT + ".*");
Expand All @@ -145,7 +145,7 @@ public StreamTableSource<Row> createStreamTableSource(Map<String, String> proper
final StartupOptions startupOptions = getStartupOptions(descriptorProperties, topic);

return createKafkaTableSource(
descriptorProperties.getTableSchema(SCHEMA()),
descriptorProperties.getTableSchema(SCHEMA),
SchemaValidator.deriveProctimeAttribute(descriptorProperties),
SchemaValidator.deriveRowtimeAttributes(descriptorProperties),
SchemaValidator.deriveFieldMapping(
Expand All @@ -162,7 +162,7 @@ public StreamTableSource<Row> createStreamTableSource(Map<String, String> proper
public StreamTableSink<Row> createStreamTableSink(Map<String, String> properties) {
final DescriptorProperties descriptorProperties = getValidatedProperties(properties);

final TableSchema schema = descriptorProperties.getTableSchema(SCHEMA());
final TableSchema schema = descriptorProperties.getTableSchema(SCHEMA);
final String topic = descriptorProperties.getString(CONNECTOR_TOPIC);
final Optional<String> proctime = SchemaValidator.deriveProctimeAttribute(descriptorProperties);
final List<RowtimeAttributeDescriptor> rowtimeAttributeDescriptors =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@
import org.apache.flink.table.expressions.GreaterThan;
import org.apache.flink.table.expressions.ItemAt;
import org.apache.flink.table.expressions.Literal;
import org.apache.flink.table.expressions.ResolvedFieldReference;
import org.apache.flink.table.expressions.PlannerResolvedFieldReference;
import org.apache.flink.types.Row;

import org.apache.hadoop.conf.Configuration;
Expand Down Expand Up @@ -179,16 +179,16 @@ public void testApplyPredicate() throws Exception {

// expressions for supported predicates
Expression pred1 = new GreaterThan(
new ResolvedFieldReference("int1", Types.INT),
new PlannerResolvedFieldReference("int1", Types.INT),
new Literal(100, Types.INT));
Expression pred2 = new EqualTo(
new ResolvedFieldReference("string1", Types.STRING),
new PlannerResolvedFieldReference("string1", Types.STRING),
new Literal("hello", Types.STRING));
// unsupported predicate
Expression unsupportedPred = new EqualTo(
new GetCompositeField(
new ItemAt(
new ResolvedFieldReference(
new PlannerResolvedFieldReference(
"list",
ObjectArrayTypeInfo.getInfoFor(
Types.ROW_NAMED(new String[] {"int1", "string1"}, Types.INT, Types.STRING))),
Expand All @@ -198,7 +198,7 @@ public void testApplyPredicate() throws Exception {
);
// invalid predicate
Expression invalidPred = new EqualTo(
new ResolvedFieldReference("long1", Types.LONG),
new PlannerResolvedFieldReference("long1", Types.LONG),
// some invalid, non-serializable literal (here an object of this test class)
new Literal(new OrcTableSourceTest(), Types.LONG)
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,12 +36,12 @@
import java.util.Map;

import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE;
import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_FROM;
import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_TYPE;
import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_TYPE;
import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA;
import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME;
import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE;
import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_FROM;
import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_TYPE;
import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_WATERMARKS_TYPE;
import static org.apache.flink.table.descriptors.Schema.SCHEMA;
import static org.apache.flink.table.descriptors.Schema.SCHEMA_NAME;
import static org.apache.flink.table.descriptors.Schema.SCHEMA_TYPE;
import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE;
import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE_VALUE_APPEND;

Expand Down Expand Up @@ -70,11 +70,11 @@ public Map<String, String> requiredContext() {
public List<String> supportedProperties() {
final List<String> properties = new ArrayList<>();
properties.add("connector." + testProperty);
properties.add(SCHEMA() + ".#." + SCHEMA_TYPE());
properties.add(SCHEMA() + ".#." + SCHEMA_NAME());
properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_TYPE());
properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_FROM());
properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_TYPE());
properties.add(SCHEMA + ".#." + SCHEMA_TYPE);
properties.add(SCHEMA + ".#." + SCHEMA_NAME);
properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_TYPE);
properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_FROM);
properties.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_TYPE);
return properties;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,12 +39,12 @@
import java.util.Optional;

import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE;
import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_FROM;
import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_TYPE;
import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_TYPE;
import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA;
import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME;
import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE;
import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_FROM;
import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_TIMESTAMPS_TYPE;
import static org.apache.flink.table.descriptors.Rowtime.ROWTIME_WATERMARKS_TYPE;
import static org.apache.flink.table.descriptors.Schema.SCHEMA;
import static org.apache.flink.table.descriptors.Schema.SCHEMA_NAME;
import static org.apache.flink.table.descriptors.Schema.SCHEMA_TYPE;
import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE;
import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE_VALUE_APPEND;

Expand Down Expand Up @@ -73,11 +73,11 @@ public Map<String, String> requiredContext() {
public List<String> supportedProperties() {
final List<String> properties = new ArrayList<>();
properties.add("connector." + testProperty);
properties.add(SCHEMA() + ".#." + SCHEMA_TYPE());
properties.add(SCHEMA() + ".#." + SCHEMA_NAME());
properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_TYPE());
properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_FROM());
properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_TYPE());
properties.add(SCHEMA + ".#." + SCHEMA_TYPE);
properties.add(SCHEMA + ".#." + SCHEMA_NAME);
properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_TYPE);
properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_FROM);
properties.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_TYPE);
return properties;
}

Expand All @@ -88,7 +88,7 @@ public StreamTableSource<Row> createStreamTableSource(Map<String, String> proper
final Optional<String> proctime = SchemaValidator.deriveProctimeAttribute(params);
final List<RowtimeAttributeDescriptor> rowtime = SchemaValidator.deriveRowtimeAttributes(params);
return new TestTableSource(
params.getTableSchema(SCHEMA()),
params.getTableSchema(SCHEMA),
properties.get(testProperty),
proctime.orElse(null),
rowtime);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
/*
* 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:https://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.descriptors;

import org.apache.flink.annotation.PublicEvolving;

/**
* An interface for descriptors that allow to register table source and/or sinks.
*/
@PublicEvolving
public interface RegistrableDescriptor extends Descriptor {

/**
* Searches for the specified table source, configures it accordingly, and registers it as
* a table under the given name.
*
* @param name table name to be registered in the table environment
*/
void registerTableSource(String name);

/**
* Searches for the specified table sink, configures it accordingly, and registers it as
* a table under the given name.
*
* @param name table name to be registered in the table environment
*/
void registerTableSink(String name);

/**
* Searches for the specified table source and sink, configures them accordingly, and registers
* them as a table under the given name.
*
* @param name table name to be registered in the table environment
*/
void registerTableSourceAndSink(String name);
}
Loading

0 comments on commit 254392c

Please sign in to comment.