Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
857a5b90
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,体验更适合开发者的 AI 搜索 >>
提交
857a5b90
编写于
12月 09, 2020
作者:
T
Timo Walther
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-20522][table] Add built-in TYPEOF function
上级
a9da2299
变更
5
隐藏空白更改
内联
并排
Showing
5 changed file
with
216 addition
and
0 deletion
+216
-0
docs/dev/table/functions/systemFunctions.md
docs/dev/table/functions/systemFunctions.md
+48
-0
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
...che/flink/table/functions/BuiltInFunctionDefinitions.java
+25
-0
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeUtils.java
...che/flink/table/types/logical/utils/LogicalTypeUtils.java
+1
-0
flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
...he/flink/table/planner/functions/MiscFunctionsITCase.java
+56
-0
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/scalar/TypeOfFunction.java
.../flink/table/runtime/functions/scalar/TypeOfFunction.java
+86
-0
未找到文件。
docs/dev/table/functions/systemFunctions.md
浏览文件 @
857a5b90
...
...
@@ -4600,6 +4600,22 @@ CAST(value AS type)
<p>
E.g.,
<code>
CAST('42' AS INT)
</code>
returns 42;
<code>
CAST(NULL AS VARCHAR)
</code>
returns NULL of type VARCHAR.
</p>
</td>
</tr>
<tr>
<td>
{% highlight text %}
TYPEOF(input)
TYPEOF(input, force_serializable)
{% endhighlight %}
</td>
<td>
<p>
Returns the string representation of the input expression's data type. By default, the
returned string is a summary string that might omit certain details for readability. If
<code>
force_serializable
</code>
is set to TRUE, the string represents a full data type
that could be persisted in a catalog. Note that especially anonymous, inline data types
have no serializable string representation. In this case, NULL is returned.
</p>
<p>
E.g.,
<code>
TYPEOF(12)
</code>
returns 'INT NOT NULL'.
</p>
</td>
</tr>
</tbody>
</table>
</div>
...
...
@@ -4625,6 +4641,22 @@ ANY.cast(TYPE)
<p>
E.g.,
<code>
'42'.cast(INT)
</code>
returns 42;
<code>
Null(STRING)
</code>
returns NULL of type STRING.
</p>
</td>
</tr>
<tr>
<td>
{% highlight java %}
call("TYPEOF", input)
call("TYPEOF", input, force_serializable)
{% endhighlight %}
</td>
<td>
<p>
Returns the string representation of the input expression's data type. By default, the
returned string is a summary string that might omit certain details for readability. If
<code>
force_serializable
</code>
is set to TRUE, the string represents a full data type
that could be persisted in a catalog. Note that especially anonymous, inline data types
have no serializable string representation. In this case, NULL is returned.
</p>
<p>
E.g.,
<code>
call("TYPEOF", 12)
</code>
returns 'INT NOT NULL'.
</p>
</td>
</tr>
</tbody>
</table>
</div>
...
...
@@ -4650,6 +4682,22 @@ ANY.cast(TYPE)
<p>
E.g.,
<code>
"42".cast(Types.INT)
</code>
returns 42;
<code>
Null(Types.STRING)
</code>
returns NULL of type STRING.
</p>
</td>
</tr>
<tr>
<td>
{% highlight scala %}
call("TYPEOF", input)
call("TYPEOF", input, force_serializable)
{% endhighlight %}
</td>
<td>
<p>
Returns the string representation of the input expression's data type. By default, the
returned string is a summary string that might omit certain details for readability. If
<code>
force_serializable
</code>
is set to TRUE, the string represents a full data type
that could be persisted in a catalog. Note that especially anonymous, inline data types
have no serializable string representation. In this case, NULL is returned.
</p>
<p>
E.g.,
<code>
call("TYPEOF", 12)
</code>
returns 'INT NOT NULL'.
</p>
</td>
</tr>
</tbody>
</table>
...
...
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
浏览文件 @
857a5b90
...
...
@@ -21,6 +21,7 @@ package org.apache.flink.table.functions;
import
org.apache.flink.annotation.PublicEvolving
;
import
org.apache.flink.table.api.DataTypes
;
import
org.apache.flink.table.api.TableException
;
import
org.apache.flink.table.types.inference.ArgumentTypeStrategy
;
import
org.apache.flink.table.types.inference.ConstantArgumentCount
;
import
org.apache.flink.table.types.inference.InputTypeStrategies
;
import
org.apache.flink.table.types.inference.TypeStrategies
;
...
...
@@ -39,6 +40,7 @@ import java.util.Set;
import
static
org
.
apache
.
flink
.
table
.
functions
.
FunctionKind
.
AGGREGATE
;
import
static
org
.
apache
.
flink
.
table
.
functions
.
FunctionKind
.
OTHER
;
import
static
org
.
apache
.
flink
.
table
.
functions
.
FunctionKind
.
SCALAR
;
import
static
org
.
apache
.
flink
.
table
.
types
.
inference
.
InputTypeStrategies
.
LITERAL
;
import
static
org
.
apache
.
flink
.
table
.
types
.
inference
.
InputTypeStrategies
.
NO_ARGS
;
import
static
org
.
apache
.
flink
.
table
.
types
.
inference
.
InputTypeStrategies
.
OUTPUT_IF_NULL
;
import
static
org
.
apache
.
flink
.
table
.
types
.
inference
.
InputTypeStrategies
.
SPECIFIC_FOR_CAST
;
...
...
@@ -71,6 +73,29 @@ import static org.apache.flink.table.types.inference.TypeStrategies.varyingStrin
@PublicEvolving
public
final
class
BuiltInFunctionDefinitions
{
// --------------------------------------------------------------------------------------------
// Debugging functions
// --------------------------------------------------------------------------------------------
public
static
final
BuiltInFunctionDefinition
TYPE_OF
=
BuiltInFunctionDefinition
.
newBuilder
()
.
name
(
"TYPEOF"
)
.
kind
(
SCALAR
)
.
inputTypeStrategy
(
or
(
sequence
(
new
String
[]
{
"input"
},
new
ArgumentTypeStrategy
[]
{
InputTypeStrategies
.
ANY
}),
sequence
(
new
String
[]
{
"input"
,
"force_serializable"
},
new
ArgumentTypeStrategy
[]
{
InputTypeStrategies
.
ANY
,
and
(
logical
(
LogicalTypeRoot
.
BOOLEAN
),
LITERAL
)
})))
.
outputTypeStrategy
(
explicit
(
DataTypes
.
STRING
()))
.
runtimeClass
(
"org.apache.flink.table.runtime.functions.scalar.TypeOfFunction"
)
.
build
();
// --------------------------------------------------------------------------------------------
// Logic functions
// --------------------------------------------------------------------------------------------
...
...
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeUtils.java
浏览文件 @
857a5b90
...
...
@@ -105,6 +105,7 @@ public final class LogicalTypeUtils {
case
RAW:
return
RawValueData
.
class
;
case
NULL:
return
Object
.
class
;
case
SYMBOL:
case
UNRESOLVED:
default
:
...
...
flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/MiscFunctionsITCase.java
0 → 100644
浏览文件 @
857a5b90
/*
* 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.functions
;
import
org.apache.flink.table.api.DataTypes
;
import
org.apache.flink.table.functions.BuiltInFunctionDefinitions
;
import
org.junit.runners.Parameterized
;
import
java.util.Arrays
;
import
java.util.List
;
import
static
org
.
apache
.
flink
.
table
.
api
.
Expressions
.
$
;
import
static
org
.
apache
.
flink
.
table
.
api
.
Expressions
.
call
;
/** Tests for miscellaneous {@link BuiltInFunctionDefinitions}. */
public
class
MiscFunctionsITCase
extends
BuiltInFunctionTestBase
{
@Parameterized
.
Parameters
(
name
=
"{index}: {0}"
)
public
static
List
<
TestSpec
>
testData
()
{
return
Arrays
.
asList
(
TestSpec
.
forFunction
(
BuiltInFunctionDefinitions
.
TYPE_OF
)
.
onFieldsWithData
(
12
,
"Hello world"
,
false
)
.
testResult
(
call
(
"TYPEOF"
,
$
(
"f0"
)),
"TYPEOF(f0)"
,
"INT NOT NULL"
,
DataTypes
.
STRING
())
.
testTableApiError
(
call
(
"TYPEOF"
,
$
(
"f0"
),
$
(
"f2"
)),
"Invalid input arguments."
)
.
testSqlError
(
"TYPEOF(f0, f2)"
,
"SQL validation failed. Invalid function call:\nTYPEOF(INT NOT NULL, BOOLEAN NOT NULL)"
)
.
testTableApiResult
(
call
(
"TYPEOF"
,
$
(
"f1"
),
true
),
"CHAR(11) NOT NULL"
,
DataTypes
.
STRING
())
.
testSqlResult
(
"TYPEOF(NULL)"
,
"NULL"
,
DataTypes
.
STRING
()));
}
}
flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/scalar/TypeOfFunction.java
0 → 100644
浏览文件 @
857a5b90
/*
* 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.runtime.functions.scalar
;
import
org.apache.flink.annotation.Internal
;
import
org.apache.flink.table.data.StringData
;
import
org.apache.flink.table.functions.BuiltInFunctionDefinitions
;
import
org.apache.flink.table.functions.FunctionContext
;
import
org.apache.flink.table.functions.SpecializedFunction.SpecializedContext
;
import
org.apache.flink.table.types.DataType
;
import
org.apache.flink.table.types.inference.CallContext
;
import
org.apache.flink.table.types.logical.LogicalType
;
import
javax.annotation.Nullable
;
import
java.util.List
;
/** Implementation of {@link BuiltInFunctionDefinitions#TYPE_OF}. */
@Internal
public
class
TypeOfFunction
extends
BuiltInScalarFunction
{
private
final
String
typeString
;
private
transient
StringData
typeStringData
;
public
TypeOfFunction
(
SpecializedContext
context
)
{
super
(
BuiltInFunctionDefinitions
.
TYPE_OF
,
context
);
final
CallContext
callContext
=
context
.
getCallContext
();
this
.
typeString
=
createTypeString
(
callContext
,
isForceSerializable
(
callContext
));
}
// --------------------------------------------------------------------------------------------
// Planning
// --------------------------------------------------------------------------------------------
private
static
boolean
isForceSerializable
(
CallContext
context
)
{
final
List
<
DataType
>
argumentDataTypes
=
context
.
getArgumentDataTypes
();
if
(
argumentDataTypes
.
size
()
!=
2
)
{
return
false
;
}
return
context
.
getArgumentValue
(
1
,
Boolean
.
class
).
orElse
(
false
);
}
private
static
@Nullable
String
createTypeString
(
CallContext
context
,
boolean
forceSerializable
)
{
final
LogicalType
input
=
context
.
getArgumentDataTypes
().
get
(
0
).
getLogicalType
();
if
(
forceSerializable
)
{
try
{
return
input
.
asSerializableString
();
}
catch
(
Exception
t
)
{
return
null
;
}
}
return
input
.
asSummaryString
();
}
// --------------------------------------------------------------------------------------------
// Runtime
// --------------------------------------------------------------------------------------------
@Override
public
void
open
(
FunctionContext
context
)
throws
Exception
{
this
.
typeStringData
=
StringData
.
fromString
(
typeString
);
}
@SuppressWarnings
(
"unused"
)
public
@Nullable
StringData
eval
(
Object
...
unused
)
{
return
typeStringData
;
}
}
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录