未验证 提交 a859e2cc 编写于 作者: S Shengkai 提交者: GitHub

[FLINK-19650][jdbc][table] Support limit push down for the JDBC connector

This closes #13800
上级 a25b9b84
......@@ -66,6 +66,11 @@ class DerbyDialect extends AbstractDialect {
return "Derby";
}
@Override
public String getLimitClause(long limit) {
return String.format("FETCH FIRST %d ROWS ONLY", limit);
}
@Override
public int maxDecimalPrecision() {
return MAX_DECIMAL_PRECISION;
......
......@@ -57,6 +57,13 @@ public interface JdbcDialect extends Serializable {
*/
JdbcRowConverter getRowConverter(RowType rowType);
/**
* Get limit clause to limit the number of emitted row from the jdbc source.
* @param limit number of row to emit. The value of the parameter should be non-negative.
* @return the limit clause.
*/
String getLimitClause(long limit);
/**
* Check if this dialect instance support a specific data type in table schema.
* @param schema the table schema.
......
......@@ -55,6 +55,11 @@ public class MySQLDialect extends AbstractDialect {
return new MySQLRowConverter(rowType);
}
@Override
public String getLimitClause(long limit) {
return "LIMIT " + limit;
}
@Override
public Optional<String> defaultDriverName() {
return Optional.of("com.mysql.jdbc.Driver");
......
......@@ -55,6 +55,11 @@ public class PostgresDialect extends AbstractDialect {
return new PostgresRowConverter(rowType);
}
@Override
public String getLimitClause(long limit) {
return "LIMIT " + limit;
}
@Override
public Optional<String> defaultDriverName() {
return Optional.of("org.postgresql.Driver");
......
......@@ -31,6 +31,7 @@ import org.apache.flink.table.connector.source.InputFormatProvider;
import org.apache.flink.table.connector.source.LookupTableSource;
import org.apache.flink.table.connector.source.ScanTableSource;
import org.apache.flink.table.connector.source.TableFunctionProvider;
import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown;
import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.table.utils.TableSchemaUtils;
......@@ -42,13 +43,18 @@ import java.util.Objects;
* A {@link DynamicTableSource} for JDBC.
*/
@Internal
public class JdbcDynamicTableSource implements ScanTableSource, LookupTableSource, SupportsProjectionPushDown {
public class JdbcDynamicTableSource implements
ScanTableSource,
LookupTableSource,
SupportsProjectionPushDown,
SupportsLimitPushDown {
private final JdbcOptions options;
private final JdbcReadOptions readOptions;
private final JdbcLookupOptions lookupOptions;
private TableSchema physicalSchema;
private final String dialectName;
private long limit = -1;
public JdbcDynamicTableSource(
JdbcOptions options,
......@@ -97,7 +103,7 @@ public class JdbcDynamicTableSource implements ScanTableSource, LookupTableSourc
}
final JdbcDialect dialect = options.getDialect();
String query = dialect.getSelectFromStatement(
options.getTableName(), physicalSchema.getFieldNames(), new String[0]);
options.getTableName(), physicalSchema.getFieldNames(), new String[0]);
if (readOptions.getPartitionColumnName().isPresent()) {
long lowerBound = readOptions.getPartitionLowerBound().get();
long upperBound = readOptions.getPartitionUpperBound().get();
......@@ -108,6 +114,9 @@ public class JdbcDynamicTableSource implements ScanTableSource, LookupTableSourc
dialect.quoteIdentifier(readOptions.getPartitionColumnName().get()) +
" BETWEEN ? AND ?";
}
if (limit >= 0) {
query = String.format("%s %s", query, dialect.getLimitClause(limit));
}
builder.setQuery(query);
final RowType rowType = (RowType) physicalSchema.toRowDataType().getLogicalType();
builder.setRowConverter(dialect.getRowConverter(rowType));
......@@ -156,11 +165,17 @@ public class JdbcDynamicTableSource implements ScanTableSource, LookupTableSourc
Objects.equals(readOptions, that.readOptions) &&
Objects.equals(lookupOptions, that.lookupOptions) &&
Objects.equals(physicalSchema, that.physicalSchema) &&
Objects.equals(dialectName, that.dialectName);
Objects.equals(dialectName, that.dialectName) &&
Objects.equals(limit, that.limit);
}
@Override
public int hashCode() {
return Objects.hash(options, readOptions, lookupOptions, physicalSchema, dialectName);
return Objects.hash(options, readOptions, lookupOptions, physicalSchema, dialectName, limit);
}
@Override
public void applyLimit(long limit) {
this.limit = limit;
}
}
......@@ -21,6 +21,7 @@ package org.apache.flink.connector.jdbc.table;
import org.apache.flink.connector.jdbc.JdbcTestBase;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.TableEnvironment;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
import org.apache.flink.table.planner.runtime.utils.StreamTestSink;
import org.apache.flink.test.util.AbstractTestBase;
......@@ -35,12 +36,15 @@ import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
/**
* ITCase for {@link JdbcDynamicTableSource}.
......@@ -51,8 +55,18 @@ public class JdbcDynamicTableSourceITCase extends AbstractTestBase {
public static final String DB_URL = "jdbc:derby:memory:test";
public static final String INPUT_TABLE = "jdbDynamicTableSource";
public static StreamExecutionEnvironment env;
public static TableEnvironment tEnv;
@Before
public void before() throws ClassNotFoundException, SQLException {
env = StreamExecutionEnvironment.getExecutionEnvironment();
EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
.useBlinkPlanner()
.inStreamingMode()
.build();
tEnv = StreamTableEnvironment.create(env, envSettings);
System.setProperty("derby.stream.error.field", JdbcTestBase.class.getCanonicalName() + ".DEV_NULL");
Class.forName(DRIVER_CLASS);
......@@ -89,13 +103,6 @@ public class JdbcDynamicTableSourceITCase extends AbstractTestBase {
@Test
public void testJdbcSource() throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
.useBlinkPlanner()
.inStreamingMode()
.build();
StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
tEnv.executeSql(
"CREATE TABLE " + INPUT_TABLE + "(" +
"id BIGINT," +
......@@ -127,13 +134,6 @@ public class JdbcDynamicTableSourceITCase extends AbstractTestBase {
@Test
public void testProject() throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
.useBlinkPlanner()
.inStreamingMode()
.build();
StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
tEnv.executeSql(
"CREATE TABLE " + INPUT_TABLE + "(" +
"id BIGINT," +
......@@ -166,4 +166,39 @@ public class JdbcDynamicTableSourceITCase extends AbstractTestBase {
.sorted().collect(Collectors.toList());
assertEquals(expected, result);
}
@Test
public void testLimit() throws Exception {
tEnv.executeSql(
"CREATE TABLE " + INPUT_TABLE + "(\n" +
"id BIGINT,\n" +
"timestamp6_col TIMESTAMP(6),\n" +
"timestamp9_col TIMESTAMP(9),\n" +
"time_col TIME,\n" +
"real_col FLOAT,\n" +
"double_col DOUBLE,\n" +
"decimal_col DECIMAL(10, 4)\n" +
") WITH (\n" +
" 'connector'='jdbc',\n" +
" 'url'='" + DB_URL + "',\n" +
" 'table-name'='" + INPUT_TABLE + "',\n" +
" 'scan.partition.column'='id',\n" +
" 'scan.partition.num'='2',\n" +
" 'scan.partition.lower-bound'='1',\n" +
" 'scan.partition.upper-bound'='2'\n" +
")"
);
Iterator<Row> collected = tEnv.executeSql("SELECT * FROM " + INPUT_TABLE + " LIMIT 1").collect();
List<String> result = CollectionUtil.iteratorToList(collected).stream()
.map(Row::toString)
.sorted()
.collect(Collectors.toList());
Set<String> expected = new HashSet<>();
expected.add("1,2020-01-01T15:35:00.123456,2020-01-01T15:35:00.123456789,15:35,1.175E-37,1.79769E308,100.1234");
expected.add("2,2020-01-01T15:36:01.123456,2020-01-01T15:36:01.123456789,15:36:01,-1.175E-37,-1.79769E308,101.1234");
assertEquals(1, result.size());
assertTrue("The actual output is not a subset of the expected set.", expected.containsAll(result));
}
}
......@@ -22,6 +22,7 @@ import org.apache.flink.table.api.TableConfig;
import org.apache.flink.table.planner.utils.StreamTableTestUtil;
import org.apache.flink.table.planner.utils.TableTestBase;
import org.junit.Before;
import org.junit.Test;
/**
......@@ -31,8 +32,8 @@ public class JdbcTablePlanTest extends TableTestBase {
private final StreamTableTestUtil util = streamTestUtil(new TableConfig());
@Test
public void testProjectionPushDown() {
@Before
public void setup() {
util.tableEnv().executeSql(
"CREATE TABLE jdbc (" +
"id BIGINT," +
......@@ -48,7 +49,15 @@ public class JdbcTablePlanTest extends TableTestBase {
" 'table-name'='test_table'" +
")"
);
}
@Test
public void testProjectionPushDown() {
util.verifyExecPlan("SELECT decimal_col, timestamp9_col, id FROM jdbc");
}
@Test
public void testLimitPushDown() {
util.verifyExecPlan("SELECT id, time_col FROM jdbc LIMIT 3");
}
}
......@@ -16,6 +16,25 @@ See the License for the specific language governing permissions and
limitations under the License.
-->
<Root>
<TestCase name="testLimitPushDown">
<Resource name="sql">
<![CDATA[SELECT id, time_col FROM jdbc LIMIT 3]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[3])
+- LogicalProject(id=[$0], time_col=[$3])
+- LogicalTableScan(table=[[default_catalog, default_database, jdbc]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[0], fetch=[3])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, jdbc, project=[id, time_col], limit=[3]]], fields=[id, time_col])
]]>
</Resource>
</TestCase>
<TestCase name="testProjectionPushDown">
<Resource name="sql">
<![CDATA[SELECT decimal_col, timestamp9_col, id FROM jdbc]]>
......
......@@ -244,6 +244,7 @@ object FlinkStreamRuleSets {
PushProjectIntoLegacyTableSourceScanRule.INSTANCE,
PushFilterIntoTableSourceScanRule.INSTANCE,
PushFilterIntoLegacyTableSourceScanRule.INSTANCE,
PushLimitIntoTableSourceScanRule.INSTANCE,
// reorder the projecct and watermark assigner
ProjectWatermarkAssignerTransposeRule.INSTANCE,
......
<?xml version="1.0" ?>
<!--
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.
-->
<Root>
<TestCase name="testFetch0WithoutOffset">
<Resource name="sql">
<![CDATA[SELECT a, c FROM MyTable FETCH FIRST 0 ROWS ONLY]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[0])
+- LogicalProject(a=[$0], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Values(tuples=[[]])
]]>
</Resource>
</TestCase>
<TestCase name="testFetchWithOffset">
<Resource name="sql">
<![CDATA[SELECT a, c FROM MyTable OFFSET 10 ROWS FETCH NEXT 10 ROWS ONLY]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(offset=[10], fetch=[10])
+- LogicalProject(a=[$0], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[10], fetch=[10])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c], limit=[20]]], fields=[a, c])
]]>
</Resource>
</TestCase>
<TestCase name="testFetchWithoutOffset">
<Resource name="sql">
<![CDATA[SELECT a, c FROM MyTable FETCH FIRST 10 ROWS ONLY]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[10])
+- LogicalProject(a=[$0], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[0], fetch=[10])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c], limit=[10]]], fields=[a, c])
]]>
</Resource>
</TestCase>
<TestCase name="testLimit0WithOffset">
<Resource name="sql">
<![CDATA[SELECT a, c FROM MyTable LIMIT 0 OFFSET 10]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(offset=[10], fetch=[0])
+- LogicalProject(a=[$0], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Values(tuples=[[]])
]]>
</Resource>
</TestCase>
<TestCase name="testLimit0WithOffset0">
<Resource name="sql">
<![CDATA[SELECT a, c FROM MyTable LIMIT 0 OFFSET 0]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(offset=[0], fetch=[0])
+- LogicalProject(a=[$0], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Values(tuples=[[]])
]]>
</Resource>
</TestCase>
<TestCase name="testLimit0WithoutOffset">
<Resource name="sql">
<![CDATA[SELECT * FROM MyTable LIMIT 0]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[0])
+- LogicalProject(a=[$0], b=[$1], c=[$2], proctime=[$3], rowtime=[$4])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Values(tuples=[[]])
]]>
</Resource>
</TestCase>
<TestCase name="testLimit0WithProcessingTime">
<Resource name="sql">
<![CDATA[SELECT proctime, c FROM MyTable LIMIT 0]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[0])
+- LogicalProject(proctime=[$3], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Values(tuples=[[]])
]]>
</Resource>
</TestCase>
<TestCase name="testLimit0WithProcessingTimeDesc">
<Resource name="sql">
<![CDATA[SELECT proctime desc, c FROM MyTable LIMIT 0]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[0])
+- LogicalProject(desc=[$3], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Values(tuples=[[]])
]]>
</Resource>
</TestCase>
<TestCase name="testLimit0WithProcessingTimeDescSecond">
<Resource name="sql">
<![CDATA[SELECT c, proctime desc FROM MyTable LIMIT 0]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[0])
+- LogicalProject(c=[$2], desc=[$3])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Values(tuples=[[]])
]]>
</Resource>
</TestCase>
<TestCase name="testLimit0WithProcessingTimeSecond">
<Resource name="sql">
<![CDATA[SELECT c, proctime FROM MyTable LIMIT 0]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[0])
+- LogicalProject(c=[$2], proctime=[$3])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Values(tuples=[[]])
]]>
</Resource>
</TestCase>
<TestCase name="testLimit0WithRowTime">
<Resource name="sql">
<![CDATA[SELECT rowtime, c FROM MyTable LIMIT 0]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[0])
+- LogicalProject(rowtime=[$4], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Values(tuples=[[]])
]]>
</Resource>
</TestCase>
<TestCase name="testLimit0WithRowTimeDesc">
<Resource name="sql">
<![CDATA[SELECT rowtime desc, c FROM MyTable LIMIT 0]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[0])
+- LogicalProject(desc=[$4], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Values(tuples=[[]])
]]>
</Resource>
</TestCase>
<TestCase name="testLimit0WithRowTimeDescSecond">
<Resource name="sql">
<![CDATA[SELECT c, rowtime desc FROM MyTable LIMIT 0]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[0])
+- LogicalProject(c=[$2], desc=[$4])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Values(tuples=[[]])
]]>
</Resource>
</TestCase>
<TestCase name="testLimit0WithRowTimeSecond">
<Resource name="sql">
<![CDATA[SELECT c, rowtime FROM MyTable LIMIT 0]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[0])
+- LogicalProject(c=[$2], rowtime=[$4])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Values(tuples=[[]])
]]>
</Resource>
</TestCase>
<TestCase name="testLimitWithOffset">
<Resource name="sql">
<![CDATA[SELECT a, c FROM MyTable LIMIT 10 OFFSET 1]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(offset=[1], fetch=[10])
+- LogicalProject(a=[$0], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[1], fetch=[10])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c], limit=[11]]], fields=[a, c])
]]>
</Resource>
</TestCase>
<TestCase name="testLimitWithOffset0">
<Resource name="sql">
<![CDATA[SELECT a, c FROM MyTable LIMIT 10 OFFSET 0]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(offset=[0], fetch=[10])
+- LogicalProject(a=[$0], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[0], fetch=[10])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c], limit=[10]]], fields=[a, c])
]]>
</Resource>
</TestCase>
<TestCase name="testLimitWithoutOffset">
<Resource name="sql">
<![CDATA[SELECT * FROM MyTable LIMIT 5]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[5])
+- LogicalProject(a=[$0], b=[$1], c=[$2], proctime=[$3], rowtime=[$4])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[0], fetch=[5])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, limit=[5]]], fields=[a, b, c, proctime, rowtime])
]]>
</Resource>
</TestCase>
<TestCase name="testLimitWithProcessingTime">
<Resource name="sql">
<![CDATA[SELECT proctime, c FROM MyTable LIMIT 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[2])
+- LogicalProject(proctime=[$3], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[0], fetch=[2])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[proctime, c], limit=[2]]], fields=[proctime, c])
]]>
</Resource>
</TestCase>
<TestCase name="testLimitWithProcessingTimeDesc">
<Resource name="sql">
<![CDATA[SELECT proctime desc, c FROM MyTable LIMIT 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[2])
+- LogicalProject(desc=[$3], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[0], fetch=[2])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[proctime, c], limit=[2]]], fields=[proctime, c])
]]>
</Resource>
</TestCase>
<TestCase name="testLimitWithProcessingTimeDescSecond">
<Resource name="sql">
<![CDATA[SELECT c, proctime desc FROM MyTable LIMIT 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[2])
+- LogicalProject(c=[$2], desc=[$3])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[0], fetch=[2])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, proctime], limit=[2]]], fields=[c, proctime])
]]>
</Resource>
</TestCase>
<TestCase name="testLimitWithProcessingTimeSecond">
<Resource name="sql">
<![CDATA[SELECT c, proctime FROM MyTable LIMIT 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[2])
+- LogicalProject(c=[$2], proctime=[$3])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[0], fetch=[2])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, proctime], limit=[2]]], fields=[c, proctime])
]]>
</Resource>
</TestCase>
<TestCase name="testLimitWithRowTime">
<Resource name="sql">
<![CDATA[SELECT rowtime, c FROM MyTable LIMIT 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[2])
+- LogicalProject(rowtime=[$4], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[0], fetch=[2])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[rowtime, c], limit=[2]]], fields=[rowtime, c])
]]>
</Resource>
</TestCase>
<TestCase name="testLimitWithRowTimeDesc">
<Resource name="sql">
<![CDATA[SELECT rowtime desc, c FROM MyTable LIMIT 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[2])
+- LogicalProject(desc=[$4], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[0], fetch=[2])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[rowtime, c], limit=[2]]], fields=[rowtime, c])
]]>
</Resource>
</TestCase>
<TestCase name="testLimitWithRowTimeDescSecond">
<Resource name="sql">
<![CDATA[SELECT c, rowtime desc FROM MyTable LIMIT 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[2])
+- LogicalProject(c=[$2], desc=[$4])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[0], fetch=[2])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, rowtime], limit=[2]]], fields=[c, rowtime])
]]>
</Resource>
</TestCase>
<TestCase name="testLimitWithRowTimeSecond">
<Resource name="sql">
<![CDATA[SELECT c, rowtime FROM MyTable LIMIT 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(fetch=[2])
+- LogicalProject(c=[$2], rowtime=[$4])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[0], fetch=[2])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, rowtime], limit=[2]]], fields=[c, rowtime])
]]>
</Resource>
</TestCase>
<TestCase name="testOffsetWithProcessingTime">
<Resource name="sql">
<![CDATA[SELECT proctime, c FROM MyTable OFFSET 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(offset=[2])
+- LogicalProject(proctime=[$3], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[2], fetch=[unlimited])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[proctime, c]]], fields=[proctime, c])
]]>
</Resource>
</TestCase>
<TestCase name="testOffsetWithRowTimeDesc">
<Resource name="sql">
<![CDATA[SELECT rowtime desc, c FROM MyTable OFFSET 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(offset=[2])
+- LogicalProject(desc=[$4], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[2], fetch=[unlimited])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[rowtime, c]]], fields=[rowtime, c])
]]>
</Resource>
</TestCase>
<TestCase name="testOffsetWithProcessingTimeDesc">
<Resource name="sql">
<![CDATA[SELECT proctime desc, c FROM MyTable OFFSET 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(offset=[2])
+- LogicalProject(desc=[$3], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[2], fetch=[unlimited])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[proctime, c]]], fields=[proctime, c])
]]>
</Resource>
</TestCase>
<TestCase name="testOffsetWithProcessingTimeDescSecond">
<Resource name="sql">
<![CDATA[SELECT c, proctime desc FROM MyTable OFFSET 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(offset=[2])
+- LogicalProject(c=[$2], desc=[$3])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[2], fetch=[unlimited])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, proctime]]], fields=[c, proctime])
]]>
</Resource>
</TestCase>
<TestCase name="testOffsetWithProcessingTimeSecond">
<Resource name="sql">
<![CDATA[SELECT c, proctime FROM MyTable OFFSET 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(offset=[2])
+- LogicalProject(c=[$2], proctime=[$3])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[2], fetch=[unlimited])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, proctime]]], fields=[c, proctime])
]]>
</Resource>
</TestCase>
<TestCase name="testOffsetWithRowTime">
<Resource name="sql">
<![CDATA[SELECT rowtime, c FROM MyTable OFFSET 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(offset=[2])
+- LogicalProject(rowtime=[$4], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[2], fetch=[unlimited])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[rowtime, c]]], fields=[rowtime, c])
]]>
</Resource>
</TestCase>
<TestCase name="testOffsetWithRowTimeDescSecond">
<Resource name="sql">
<![CDATA[SELECT c, rowtime desc FROM MyTable OFFSET 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(offset=[2])
+- LogicalProject(c=[$2], desc=[$4])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[2], fetch=[unlimited])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, rowtime]]], fields=[c, rowtime])
]]>
</Resource>
</TestCase>
<TestCase name="testOnlyOffset">
<Resource name="sql">
<![CDATA[SELECT a, c FROM MyTable OFFSET 10 ROWS]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(offset=[10])
+- LogicalProject(a=[$0], c=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[10], fetch=[unlimited])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c]]], fields=[a, c])
]]>
</Resource>
</TestCase>
<TestCase name="testOffsetWithRowTimeSecond">
<Resource name="sql">
<![CDATA[SELECT c, rowtime FROM MyTable OFFSET 2]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSort(offset=[2])
+- LogicalProject(c=[$2], rowtime=[$4])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Limit(offset=[2], fetch=[unlimited])
+- Exchange(distribution=[single])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, rowtime]]], fields=[c, rowtime])
]]>
</Resource>
</TestCase>
</Root>
......@@ -22,13 +22,17 @@ import org.apache.flink.api.scala._
import org.apache.flink.table.api._
import org.apache.flink.table.planner.utils.TableTestBase
import org.junit.Test
import org.junit.{Before, Test}
class LimitTest extends TableTestBase {
private val util = streamTestUtil()
util.addDataStream[(Int, String, Long)](
"MyTable", 'a, 'b, 'c, 'proctime.proctime, 'rowtime.rowtime)
protected val util = streamTestUtil()
@Before
def setup(): Unit = {
util.addDataStream[(Int, String, Long)](
"MyTable", 'a, 'b, 'c, 'proctime.proctime, 'rowtime.rowtime)
}
@Test
def testLimitWithoutOffset(): Unit = {
......
/*
* 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.planner.plan.stream.sql
import org.apache.flink.table.planner.plan.rules.logical.PushLimitIntoTableSourceScanRule
import org.junit.Before
/**
* Test for [[PushLimitIntoTableSourceScanRule]].
*/
class LimitableSourceTest extends LimitTest {
@Before
override def setup(): Unit = {
val ddl =
s"""
|CREATE TABLE MyTable (
| a INT,
| b BIGINT,
| c STRING,
| proctime TIMESTAMP(3),
| rowtime TIMESTAMP(3)
|) WITH (
| 'connector' = 'values',
| 'bounded' = 'true'
|)
""".stripMargin
util.tableEnv.executeSql(ddl)
}
}
/*
* 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.planner.runtime.stream.sql
import org.apache.flink.api.scala._
import org.apache.flink.table.api.bridge.scala._
import org.apache.flink.table.planner.factories.TestValuesTableFactory
import org.apache.flink.table.planner.plan.rules.logical.PushLimitIntoTableSourceScanRule
import org.apache.flink.table.planner.runtime.utils.{StreamingTestBase, TestingRetractSink}
import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row
import org.apache.flink.types.Row
import org.junit.Assert.assertEquals
import org.junit.{Before, Test}
/**
* Test for [[PushLimitIntoTableSourceScanRule]].
*/
class LimitableSourceITCase extends StreamingTestBase() {
val data = Seq(
row("book", 1, 12),
row("book", 2, 19),
row("book", 4, 11),
row("fruit", 4, 33),
row("fruit", 3, 44),
row("fruit", 5, 22))
@Before
def setup(): Unit = {
val dataId = TestValuesTableFactory.registerData(data)
val ddl =
s"""
|CREATE TABLE Source (
| category STRING,
| shopId INT,
| num INT
|) WITH (
| 'connector' = 'values',
| 'data-id' = '$dataId',
| 'bounded' = 'false'
|)
|""".stripMargin
tEnv.executeSql(ddl)
}
@Test
def testLimit(): Unit = {
val sql = "SELECT * FROM Source LIMIT 4"
val sink = new TestingRetractSink
tEnv.sqlQuery(sql).toRetractStream[Row].addSink(sink).setParallelism(1)
env.execute()
val expected = Seq(
"book,1,12",
"book,2,19",
"book,4,11",
"fruit,4,33")
assertEquals(expected.sorted, sink.getRetractResults.sorted)
}
@Test
def testOffsetAndFetch(): Unit = {
val sql = "SELECT * FROM Source LIMIT 4 OFFSET 2"
val sink = new TestingRetractSink
tEnv.sqlQuery(sql).toRetractStream[Row].addSink(sink).setParallelism(1)
env.execute()
val expected = Seq(
"book,4,11",
"fruit,4,33",
"fruit,3,44",
"fruit,5,22")
assertEquals(expected.sorted, sink.getRetractResults.sorted)
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册