Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
450b4241
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,发现更多精彩内容 >>
提交
450b4241
编写于
8月 23, 2017
作者:
Z
zjureel
提交者:
Greg Hogan
11月 28, 2017
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-6864] [core] Fix confusing "invalid POJO type" messages from TypeExtractor
This closes #4574
上级
53f2c1c2
变更
2
隐藏空白更改
内联
并排
Showing
2 changed file
with
21 addition
and
6 deletion
+21
-6
docs/dev/types_serialization.md
docs/dev/types_serialization.md
+3
-0
flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
...va/org/apache/flink/api/java/typeutils/TypeExtractor.java
+18
-6
未找到文件。
docs/dev/types_serialization.md
浏览文件 @
450b4241
...
...
@@ -115,6 +115,9 @@ conditions are fulfilled:
or have a public getter- and a setter- method that follows the Java beans
naming conventions for getters and setters.
Note that when a user-defined data type can't be recognized as a POJO type, it must be processed as GenericType and
serialized with Kryo.
#### Creating a TypeInformation or TypeSerializer
...
...
flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
浏览文件 @
450b4241
...
...
@@ -1885,7 +1885,9 @@ public class TypeExtractor {
ParameterizedType
parameterizedType
,
TypeInformation
<
IN1
>
in1Type
,
TypeInformation
<
IN2
>
in2Type
)
{
if
(!
Modifier
.
isPublic
(
clazz
.
getModifiers
()))
{
LOG
.
info
(
"Class "
+
clazz
.
getName
()
+
" is not public, cannot treat it as a POJO type. Will be handled as GenericType"
);
LOG
.
info
(
"Class "
+
clazz
.
getName
()
+
" is not public so it cannot be used as a POJO type "
+
"and must be processed as GenericType. Please read the Flink documentation "
+
"on \"Data Types & Serialization\" for details of the effect on performance."
);
return
new
GenericTypeInfo
<
OUT
>(
clazz
);
}
...
...
@@ -1900,7 +1902,9 @@ public class TypeExtractor {
List
<
Field
>
fields
=
getAllDeclaredFields
(
clazz
,
false
);
if
(
fields
.
size
()
==
0
)
{
LOG
.
info
(
"No fields detected for "
+
clazz
+
". Cannot be used as a PojoType. Will be handled as GenericType"
);
LOG
.
info
(
"No fields were detected for "
+
clazz
+
" so it cannot be used as a POJO type "
+
"and must be processed as GenericType. Please read the Flink documentation "
+
"on \"Data Types & Serialization\" for details of the effect on performance."
);
return
new
GenericTypeInfo
<
OUT
>(
clazz
);
}
...
...
@@ -1908,7 +1912,9 @@ public class TypeExtractor {
for
(
Field
field
:
fields
)
{
Type
fieldType
=
field
.
getGenericType
();
if
(!
isValidPojoField
(
field
,
clazz
,
typeHierarchy
))
{
LOG
.
info
(
clazz
+
" is not a valid POJO type because not all fields are valid POJO fields."
);
LOG
.
info
(
"Class "
+
clazz
+
" cannot be used as a POJO type because not all fields are valid POJO fields, "
+
"and must be processed as GenericType. Please read the Flink documentation "
+
"on \"Data Types & Serialization\" for details of the effect on performance."
);
return
null
;
}
try
{
...
...
@@ -1934,7 +1940,9 @@ public class TypeExtractor {
List
<
Method
>
methods
=
getAllDeclaredMethods
(
clazz
);
for
(
Method
method
:
methods
)
{
if
(
method
.
getName
().
equals
(
"readObject"
)
||
method
.
getName
().
equals
(
"writeObject"
))
{
LOG
.
info
(
clazz
+
" contains custom serialization methods we do not call."
);
LOG
.
info
(
"Class "
+
clazz
+
" contains custom serialization methods we do not call, so it cannot be used as a POJO type "
+
"and must be processed as GenericType. Please read the Flink documentation "
+
"on \"Data Types & Serialization\" for details of the effect on performance."
);
return
null
;
}
}
...
...
@@ -1949,12 +1957,16 @@ public class TypeExtractor {
LOG
.
info
(
clazz
+
" is abstract or an interface, having a concrete "
+
"type can increase performance."
);
}
else
{
LOG
.
info
(
clazz
+
" must have a default constructor to be used as a POJO."
);
LOG
.
info
(
clazz
+
" is missing a default constructor so it cannot be used as a POJO type "
+
"and must be processed as GenericType. Please read the Flink documentation "
+
"on \"Data Types & Serialization\" for details of the effect on performance."
);
return
null
;
}
}
if
(
defaultConstructor
!=
null
&&
!
Modifier
.
isPublic
(
defaultConstructor
.
getModifiers
()))
{
LOG
.
info
(
"The default constructor of "
+
clazz
+
" should be Public to be used as a POJO."
);
LOG
.
info
(
"The default constructor of "
+
clazz
+
" is not Public so it cannot be used as a POJO type "
+
"and must be processed as GenericType. Please read the Flink documentation "
+
"on \"Data Types & Serialization\" for details of the effect on performance."
);
return
null
;
}
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录