Skip to content

Commit

Permalink
[FLINK-11068][table] Convert the API classes of *Table to interfaces
Browse files Browse the repository at this point in the history
This closes apache#8006
  • Loading branch information
hequn8128 authored and dawidwys committed Mar 19, 2019
1 parent 299747f commit 6468024
Show file tree
Hide file tree
Showing 32 changed files with 1,915 additions and 1,513 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.Utils;
import org.apache.flink.table.sinks.BatchTableSink;
import org.apache.flink.table.sinks.TableSink;
import org.apache.flink.types.Row;

/**
Expand Down Expand Up @@ -59,7 +58,7 @@ public TypeInformation<?>[] getFieldTypes() {
}

@Override
public TableSink<Row> configure(String[] fieldNames, TypeInformation<?>[] fieldTypes) {
public CollectBatchTableSink configure(String[] fieldNames, TypeInformation<?>[] fieldTypes) {
final CollectBatchTableSink copy = new CollectBatchTableSink(accumulatorName, serializer);
copy.fieldNames = fieldNames;
copy.fieldTypes = fieldTypes;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.experimental.CollectSink;
import org.apache.flink.table.sinks.RetractStreamTableSink;
import org.apache.flink.table.sinks.TableSink;
import org.apache.flink.types.Row;

import java.net.InetAddress;
Expand Down Expand Up @@ -60,7 +59,7 @@ public TypeInformation<?>[] getFieldTypes() {
}

@Override
public TableSink<Tuple2<Boolean, Row>> configure(String[] fieldNames, TypeInformation<?>[] fieldTypes) {
public CollectStreamTableSink configure(String[] fieldNames, TypeInformation<?>[] fieldTypes) {
final CollectStreamTableSink copy = new CollectStreamTableSink(targetAddress, targetPort, serializer);
copy.fieldNames = fieldNames;
copy.fieldTypes = fieldTypes;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -419,7 +419,9 @@ private <C> ResultDescriptor executeQueryInternal(ExecutionContext<C> context, S
try {
// writing to a sink requires an optimization step that might reference UDFs during code compilation
context.wrapClassLoader(() -> {
table.writeToSink(result.getTableSink(), envInst.getQueryConfig());
envInst
.getTableEnvironment()
.writeToSink(table, result.getTableSink(), envInst.getQueryConfig());
return null;
});
jobGraph = envInst.createJobGraph(jobName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,7 @@
package org.apache.flink.table.client.gateway.local;

import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.JobManagerOptions;
import org.apache.flink.runtime.net.ConnectionUtils;
Expand All @@ -33,7 +32,6 @@
import org.apache.flink.table.client.gateway.local.result.DynamicResult;
import org.apache.flink.table.client.gateway.local.result.MaterializedCollectBatchResult;
import org.apache.flink.table.client.gateway.local.result.MaterializedCollectStreamResult;
import org.apache.flink.types.Row;

import java.net.InetAddress;
import java.net.InetSocketAddress;
Expand Down Expand Up @@ -61,7 +59,7 @@ public ResultStore(Configuration flinkConfig) {
*/
public <T> DynamicResult<T> createResult(Environment env, TableSchema schema, ExecutionConfig config) {

final TypeInformation<Row> outputType = Types.ROW_NAMED(schema.getFieldNames(), schema.getFieldTypes());
final RowTypeInfo outputType = new RowTypeInfo(schema.getFieldTypes(), schema.getFieldNames());

if (env.getExecution().isStreamingExecution()) {
// determine gateway address (and port if possible)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@
package org.apache.flink.table.client.gateway.local.result;

import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
import org.apache.flink.table.client.gateway.TypedResult;
import org.apache.flink.types.Row;

Expand All @@ -38,7 +38,7 @@ public class ChangelogCollectStreamResult<C> extends CollectStreamResult<C> impl
private List<Tuple2<Boolean, Row>> changeRecordBuffer;
private static final int CHANGE_RECORD_BUFFER_SIZE = 5_000;

public ChangelogCollectStreamResult(TypeInformation<Row> outputType, ExecutionConfig config,
public ChangelogCollectStreamResult(RowTypeInfo outputType, ExecutionConfig config,
InetAddress gatewayAddress, int gatewayPort) {
super(outputType, config, gatewayAddress, gatewayPort);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamUtils;
import org.apache.flink.streaming.experimental.SocketStreamIterator;
Expand Down Expand Up @@ -54,7 +55,7 @@ public abstract class CollectStreamResult<C> extends BasicResult<C> implements D
protected final Object resultLock;
protected SqlExecutionException executionException;

public CollectStreamResult(TypeInformation<Row> outputType, ExecutionConfig config,
public CollectStreamResult(RowTypeInfo outputType, ExecutionConfig config,
InetAddress gatewayAddress, int gatewayPort) {
this.outputType = outputType;

Expand All @@ -72,7 +73,8 @@ public CollectStreamResult(TypeInformation<Row> outputType, ExecutionConfig conf

// create table sink
// pass binding address and port such that sink knows where to send to
collectTableSink = new CollectStreamTableSink(iterator.getBindAddress(), iterator.getPort(), serializer);
collectTableSink = new CollectStreamTableSink(iterator.getBindAddress(), iterator.getPort(), serializer)
.configure(outputType.getFieldNames(), outputType.getFieldTypes());
retrievalThread = new ResultRetrievalThread();
monitoringThread = new JobMonitoringThread();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.accumulators.SerializedListAccumulator;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
import org.apache.flink.table.client.gateway.SqlExecutionException;
import org.apache.flink.table.client.gateway.TypedResult;
import org.apache.flink.table.client.gateway.local.CollectBatchTableSink;
Expand Down Expand Up @@ -53,11 +54,12 @@ public class MaterializedCollectBatchResult<C> extends BasicResult<C> implements

private volatile boolean snapshotted = false;

public MaterializedCollectBatchResult(TypeInformation<Row> outputType, ExecutionConfig config) {
public MaterializedCollectBatchResult(RowTypeInfo outputType, ExecutionConfig config) {
this.outputType = outputType;

accumulatorName = new AbstractID().toString();
tableSink = new CollectBatchTableSink(accumulatorName, outputType.createSerializer(config));
tableSink = new CollectBatchTableSink(accumulatorName, outputType.createSerializer(config))
.configure(outputType.getFieldNames(), outputType.getFieldTypes());
resultLock = new Object();
retrievalThread = new ResultRetrievalThread();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,8 @@

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
import org.apache.flink.table.client.gateway.SqlExecutionException;
import org.apache.flink.table.client.gateway.TypedResult;
import org.apache.flink.types.Row;
Expand Down Expand Up @@ -90,7 +90,7 @@ public class MaterializedCollectStreamResult<C> extends CollectStreamResult<C> i

@VisibleForTesting
public MaterializedCollectStreamResult(
TypeInformation<Row> outputType,
RowTypeInfo outputType,
ExecutionConfig config,
InetAddress gatewayAddress,
int gatewayPort,
Expand All @@ -117,7 +117,7 @@ public MaterializedCollectStreamResult(
}

public MaterializedCollectStreamResult(
TypeInformation<Row> outputType,
RowTypeInfo outputType,
ExecutionConfig config,
InetAddress gatewayAddress,
int gatewayPort,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,9 @@
package org.apache.flink.table.client.gateway.local.result;

import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
import org.apache.flink.table.client.gateway.TypedResult;
import org.apache.flink.types.Row;

Expand All @@ -42,7 +42,7 @@ public class MaterializedCollectStreamResultTest {

@Test
public void testSnapshot() throws UnknownHostException {
final TypeInformation<Row> type = Types.ROW(Types.STRING, Types.LONG);
final RowTypeInfo type = new RowTypeInfo(Types.STRING, Types.LONG);

TestMaterializedCollectStreamResult<?> result = null;
try {
Expand Down Expand Up @@ -90,7 +90,7 @@ public void testSnapshot() throws UnknownHostException {

@Test
public void testLimitedSnapshot() throws UnknownHostException {
final TypeInformation<Row> type = Types.ROW(Types.STRING, Types.LONG);
final RowTypeInfo type = new RowTypeInfo(Types.STRING, Types.LONG);

TestMaterializedCollectStreamResult<?> result = null;
try {
Expand Down Expand Up @@ -145,7 +145,7 @@ private static class TestMaterializedCollectStreamResult<T> extends Materialized
public boolean isRetrieving;

public TestMaterializedCollectStreamResult(
TypeInformation<Row> outputType,
RowTypeInfo outputType,
ExecutionConfig config,
InetAddress gatewayAddress,
int gatewayPort,
Expand All @@ -162,7 +162,7 @@ public TestMaterializedCollectStreamResult(
}

public TestMaterializedCollectStreamResult(
TypeInformation<Row> outputType,
RowTypeInfo outputType,
ExecutionConfig config,
InetAddress gatewayAddress,
int gatewayPort,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* 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.api;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.table.expressions.Expression;

/**
* A table that has been windowed for {@link GroupWindow}s.
*/
@PublicEvolving
public interface GroupWindowedTable {

/**
* Groups the elements by a mandatory window and one or more optional grouping attributes.
* The window is specified by referring to its alias.
*
* <p>If no additional grouping attribute is specified and if the input is a streaming table,
* the aggregation will be performed by a single task, i.e., with parallelism 1.
*
* <p>Aggregations are performed per group and defined by a subsequent {@code select(...)}
* clause similar to SQL SELECT-GROUP-BY query.
*
* <p>Example:
*
* <pre>
* {code
* tab.window([groupWindow].as("w")).groupBy("w, key").select("key, value.avg")
* }
* </pre>
*/
WindowGroupedTable groupBy(String fields);

/**
* Groups the elements by a mandatory window and one or more optional grouping attributes.
* The window is specified by referring to its alias.
*
* <p>If no additional grouping attribute is specified and if the input is a streaming table,
* the aggregation will be performed by a single task, i.e., with parallelism 1.
*
* <p>Aggregations are performed per group and defined by a subsequent {@code select(...)}
* clause similar to SQL SELECT-GROUP-BY query.
*
* <p>Scala Example:
*
* <pre>
* {code
* tab.window([groupWindow] as 'w)).groupBy('w, 'key).select('key, 'value.avg)
* }
* </pre>
*/
WindowGroupedTable groupBy(Expression... fields);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/*
* 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.api;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.table.expressions.Expression;

/**
* A table that has been grouped on a set of grouping keys.
*/
@PublicEvolving
public interface GroupedTable {

/**
* Performs a selection operation on a grouped table. Similar to an SQL SELECT statement.
* The field expressions can contain complex expressions and aggregations.
*
* <p>Example:
*
* <pre>
* {@code
* tab.groupBy("key").select("key, value.avg + ' The average' as average")
* }
* </pre>
*/
Table select(String fields);

/**
* Performs a selection operation on a grouped table. Similar to an SQL SELECT statement.
* The field expressions can contain complex expressions and aggregations.
*
* <p>Scala Example:
*
* <pre>
* {@code
* tab.groupBy('key).select('key, 'value.avg + " The average" as 'average)
* }
* </pre>
*/
Table select(Expression... fields);
}
Loading

0 comments on commit 6468024

Please sign in to comment.