Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
a859e2cc
F
flink
项目概览
doujutun3207
/
flink
与 Fork 源项目一致
从无法访问的项目Fork
通知
24
Star
0
Fork
0
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
0
列表
看板
标记
里程碑
合并请求
0
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
F
flink
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
0
Issue
0
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
提交
Issue看板
体验新版 GitCode,发现更多精彩内容 >>
未验证
提交
a859e2cc
编写于
12月 27, 2020
作者:
S
Shengkai
提交者:
GitHub
12月 27, 2020
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-19650][jdbc][table] Support limit push down for the JDBC connector
This closes #13800
上级
a25b9b84
变更
13
隐藏空白更改
内联
并排
Showing
13 changed file
with
912 addition
and
24 deletion
+912
-24
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/DerbyDialect.java
...org/apache/flink/connector/jdbc/dialect/DerbyDialect.java
+5
-0
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/JdbcDialect.java
.../org/apache/flink/connector/jdbc/dialect/JdbcDialect.java
+7
-0
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/MySQLDialect.java
...org/apache/flink/connector/jdbc/dialect/MySQLDialect.java
+5
-0
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/PostgresDialect.java
.../apache/flink/connector/jdbc/dialect/PostgresDialect.java
+5
-0
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java
...he/flink/connector/jdbc/table/JdbcDynamicTableSource.java
+19
-4
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceITCase.java
...nk/connector/jdbc/table/JdbcDynamicTableSourceITCase.java
+49
-14
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcTablePlanTest.java
.../apache/flink/connector/jdbc/table/JdbcTablePlanTest.java
+11
-2
flink-connectors/flink-connector-jdbc/src/test/resources/org/apache/flink/connector/jdbc/table/JdbcTablePlanTest.xml
...g/apache/flink/connector/jdbc/table/JdbcTablePlanTest.xml
+19
-0
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala
.../flink/table/planner/plan/rules/FlinkStreamRuleSets.scala
+1
-0
flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/LimitableSourceTest.xml
...ink/table/planner/plan/stream/sql/LimitableSourceTest.xml
+641
-0
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LimitTest.scala
...pache/flink/table/planner/plan/stream/sql/LimitTest.scala
+8
-4
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LimitableSourceTest.scala
...k/table/planner/plan/stream/sql/LimitableSourceTest.scala
+47
-0
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/LimitableSourceITCase.scala
...le/planner/runtime/stream/sql/LimitableSourceITCase.scala
+95
-0
未找到文件。
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/DerbyDialect.java
浏览文件 @
a859e2cc
...
...
@@ -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
;
...
...
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/JdbcDialect.java
浏览文件 @
a859e2cc
...
...
@@ -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.
...
...
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/MySQLDialect.java
浏览文件 @
a859e2cc
...
...
@@ -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"
);
...
...
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/PostgresDialect.java
浏览文件 @
a859e2cc
...
...
@@ -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"
);
...
...
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java
浏览文件 @
a859e2cc
...
...
@@ -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
;
}
}
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceITCase.java
浏览文件 @
a859e2cc
...
...
@@ -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
));
}
}
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcTablePlanTest.java
浏览文件 @
a859e2cc
...
...
@@ -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"
);
}
}
flink-connectors/flink-connector-jdbc/src/test/resources/org/apache/flink/connector/jdbc/table/JdbcTablePlanTest.xml
浏览文件 @
a859e2cc
...
...
@@ -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]]>
...
...
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala
浏览文件 @
a859e2cc
...
...
@@ -244,6 +244,7 @@ object FlinkStreamRuleSets {
PushProjectIntoLegacyTableSourceScanRule
.
INSTANCE
,
PushFilterIntoTableSourceScanRule
.
INSTANCE
,
PushFilterIntoLegacyTableSourceScanRule
.
INSTANCE
,
PushLimitIntoTableSourceScanRule
.
INSTANCE
,
// reorder the projecct and watermark assigner
ProjectWatermarkAssignerTransposeRule
.
INSTANCE
,
...
...
flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/LimitableSourceTest.xml
0 → 100644
浏览文件 @
a859e2cc
<?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>
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LimitTest.scala
浏览文件 @
a859e2cc
...
...
@@ -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
=
{
...
...
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LimitableSourceTest.scala
0 → 100644
浏览文件 @
a859e2cc
/*
* 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
)
}
}
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/LimitableSourceITCase.scala
0 → 100644
浏览文件 @
a859e2cc
/*
* 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.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录