Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
e058abba
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,发现更多精彩内容 >>
提交
e058abba
编写于
11月 10, 2015
作者:
R
Robert Metzger
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-2992] Remove use of SerializationUtils
This closes #1343
上级
b6f00f96
变更
6
隐藏空白更改
内联
并排
Showing
6 changed file
with
28 addition
and
38 deletion
+28
-38
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
...che/flink/streaming/api/datastream/AllWindowedStream.java
+1
-6
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
...apache/flink/streaming/api/datastream/WindowedStream.java
+1
-6
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java
...rg/apache/flink/streaming/api/operators/StreamSource.java
+1
-1
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
...g/runtime/operators/windowing/NonKeyedWindowOperator.java
+9
-13
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
...streaming/runtime/operators/windowing/WindowOperator.java
+9
-12
tools/maven/checkstyle.xml
tools/maven/checkstyle.xml
+7
-0
未找到文件。
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
浏览文件 @
e058abba
...
...
@@ -18,7 +18,6 @@
package
org.apache.flink.streaming.api.datastream
;
import
org.apache.commons.lang.SerializationUtils
;
import
org.apache.flink.api.common.functions.FoldFunction
;
import
org.apache.flink.api.common.functions.Function
;
import
org.apache.flink.api.common.functions.ReduceFunction
;
...
...
@@ -153,13 +152,9 @@ public class AllWindowedStream<T, W extends Window> {
evictor
).
enableSetProcessingTime
(
setProcessingTime
);
}
else
{
// we need to copy because we need our own instance of the pre aggregator
@SuppressWarnings
(
"unchecked"
)
ReduceFunction
<
T
>
functionCopy
=
(
ReduceFunction
<
T
>)
SerializationUtils
.
clone
(
function
);
operator
=
new
NonKeyedWindowOperator
<>(
windowAssigner
,
windowAssigner
.
getWindowSerializer
(
getExecutionEnvironment
().
getConfig
()),
new
PreAggregatingHeapWindowBuffer
.
Factory
<>(
function
Copy
),
new
PreAggregatingHeapWindowBuffer
.
Factory
<>(
function
),
new
ReduceAllWindowFunction
<
W
,
T
>(
function
),
trigger
).
enableSetProcessingTime
(
setProcessingTime
);
}
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
浏览文件 @
e058abba
...
...
@@ -18,7 +18,6 @@
package
org.apache.flink.streaming.api.datastream
;
import
org.apache.commons.lang.SerializationUtils
;
import
org.apache.flink.api.common.functions.FoldFunction
;
import
org.apache.flink.api.common.functions.Function
;
import
org.apache.flink.api.common.functions.ReduceFunction
;
...
...
@@ -167,15 +166,11 @@ public class WindowedStream<T, K, W extends Window> {
evictor
).
enableSetProcessingTime
(
setProcessingTime
);
}
else
{
// we need to copy because we need our own instance of the pre aggregator
@SuppressWarnings
(
"unchecked"
)
ReduceFunction
<
T
>
functionCopy
=
(
ReduceFunction
<
T
>)
SerializationUtils
.
clone
(
function
);
operator
=
new
WindowOperator
<>(
windowAssigner
,
windowAssigner
.
getWindowSerializer
(
getExecutionEnvironment
().
getConfig
()),
keySel
,
input
.
getKeyType
().
createSerializer
(
getExecutionEnvironment
().
getConfig
()),
new
PreAggregatingHeapWindowBuffer
.
Factory
<>(
function
Copy
),
new
PreAggregatingHeapWindowBuffer
.
Factory
<>(
function
),
new
ReduceWindowFunction
<
K
,
W
,
T
>(
function
),
trigger
).
enableSetProcessingTime
(
setProcessingTime
);
}
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java
浏览文件 @
e058abba
...
...
@@ -65,7 +65,7 @@ public class StreamSource<T> extends AbstractUdfStreamOperator<T, SourceFunction
public
void
cancel
()
{
userFunction
.
cancel
();
// the context may not be initialized if the source was never running.
if
(
ctx
!=
null
)
{
if
(
ctx
!=
null
)
{
ctx
.
close
();
}
}
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
浏览文件 @
e058abba
...
...
@@ -18,7 +18,6 @@
package
org.apache.flink.streaming.runtime.operators.windowing
;
import
com.google.common.annotations.VisibleForTesting
;
import
org.apache.commons.lang.SerializationUtils
;
import
org.apache.flink.api.common.ExecutionConfig
;
import
org.apache.flink.api.common.state.OperatorState
;
import
org.apache.flink.api.common.typeinfo.TypeInformation
;
...
...
@@ -42,11 +41,10 @@ import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBuff
import
org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer
;
import
org.apache.flink.streaming.runtime.streamrecord.StreamRecord
;
import
org.apache.flink.streaming.runtime.tasks.StreamTaskState
;
import
org.apache.flink.util.InstantiationUtil
;
import
org.slf4j.Logger
;
import
org.slf4j.LoggerFactory
;
import
java.io.ByteArrayInputStream
;
import
java.io.ByteArrayOutputStream
;
import
java.io.IOException
;
import
java.io.ObjectInputStream
;
import
java.io.Serializable
;
...
...
@@ -372,7 +370,7 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
@SuppressWarnings
(
"unchecked"
)
protected
Context
(
DataInputView
in
)
throws
Exception
{
protected
Context
(
DataInputView
in
,
ClassLoader
userClassloader
)
throws
Exception
{
this
.
window
=
windowSerializer
.
deserialize
(
in
);
this
.
watermarkTimer
=
in
.
readLong
();
this
.
processingTimeTimer
=
in
.
readLong
();
...
...
@@ -380,8 +378,7 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
int
stateSize
=
in
.
readInt
();
byte
[]
stateData
=
new
byte
[
stateSize
];
in
.
read
(
stateData
);
ByteArrayInputStream
bais
=
new
ByteArrayInputStream
(
stateData
);
state
=
(
HashMap
<
String
,
Serializable
>)
SerializationUtils
.
deserialize
(
bais
);
state
=
InstantiationUtil
.
deserializeObject
(
stateData
,
userClassloader
);
this
.
windowBuffer
=
windowBufferFactory
.
create
();
int
numElements
=
in
.
readInt
();
...
...
@@ -396,10 +393,9 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
out
.
writeLong
(
watermarkTimer
);
out
.
writeLong
(
processingTimeTimer
);
ByteArrayOutputStream
baos
=
new
ByteArrayOutputStream
();
SerializationUtils
.
serialize
(
state
,
baos
);
out
.
writeInt
(
baos
.
size
());
out
.
write
(
baos
.
toByteArray
(),
0
,
baos
.
size
());
byte
[]
serializedState
=
InstantiationUtil
.
serializeObject
(
state
);
out
.
writeInt
(
serializedState
.
length
);
out
.
write
(
serializedState
,
0
,
serializedState
.
length
);
MultiplexingStreamRecordSerializer
<
IN
>
recordSerializer
=
new
MultiplexingStreamRecordSerializer
<>(
inputSerializer
);
out
.
writeInt
(
windowBuffer
.
size
());
...
...
@@ -534,10 +530,10 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
public
void
restoreState
(
StreamTaskState
taskState
)
throws
Exception
{
super
.
restoreState
(
taskState
);
final
ClassLoader
userClassloader
=
getUserCodeClassloader
();
@SuppressWarnings
(
"unchecked"
)
StateHandle
<
DataInputView
>
inputState
=
(
StateHandle
<
DataInputView
>)
taskState
.
getOperatorState
();
DataInputView
in
=
inputState
.
getState
(
getUserCodeClassloader
()
);
DataInputView
in
=
inputState
.
getState
(
userClassloader
);
int
numWindows
=
in
.
readInt
();
this
.
windows
=
new
HashMap
<>(
numWindows
);
...
...
@@ -545,7 +541,7 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
this
.
watermarkTimers
=
new
HashMap
<>();
for
(
int
j
=
0
;
j
<
numWindows
;
j
++)
{
Context
context
=
new
Context
(
in
);
Context
context
=
new
Context
(
in
,
userClassloader
);
windows
.
put
(
context
.
window
,
context
);
}
}
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
浏览文件 @
e058abba
...
...
@@ -18,7 +18,6 @@
package
org.apache.flink.streaming.runtime.operators.windowing
;
import
com.google.common.annotations.VisibleForTesting
;
import
org.apache.commons.lang.SerializationUtils
;
import
org.apache.flink.api.common.ExecutionConfig
;
import
org.apache.flink.api.common.state.OperatorState
;
import
org.apache.flink.api.common.typeinfo.TypeInformation
;
...
...
@@ -43,11 +42,10 @@ import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBuff
import
org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer
;
import
org.apache.flink.streaming.runtime.streamrecord.StreamRecord
;
import
org.apache.flink.streaming.runtime.tasks.StreamTaskState
;
import
org.apache.flink.util.InstantiationUtil
;
import
org.slf4j.Logger
;
import
org.slf4j.LoggerFactory
;
import
java.io.ByteArrayInputStream
;
import
java.io.ByteArrayOutputStream
;
import
java.io.IOException
;
import
java.io.ObjectInputStream
;
import
java.io.Serializable
;
...
...
@@ -436,7 +434,7 @@ public class WindowOperator<K, IN, OUT, W extends Window>
* {@link #writeToState(StateBackend.CheckpointStateOutputView)}
*/
@SuppressWarnings
(
"unchecked"
)
protected
Context
(
DataInputView
in
)
throws
Exception
{
protected
Context
(
DataInputView
in
,
ClassLoader
userClassloader
)
throws
Exception
{
this
.
key
=
keySerializer
.
deserialize
(
in
);
this
.
window
=
windowSerializer
.
deserialize
(
in
);
this
.
watermarkTimer
=
in
.
readLong
();
...
...
@@ -445,8 +443,7 @@ public class WindowOperator<K, IN, OUT, W extends Window>
int
stateSize
=
in
.
readInt
();
byte
[]
stateData
=
new
byte
[
stateSize
];
in
.
read
(
stateData
);
ByteArrayInputStream
bais
=
new
ByteArrayInputStream
(
stateData
);
state
=
(
HashMap
<
String
,
Serializable
>)
SerializationUtils
.
deserialize
(
bais
);
state
=
InstantiationUtil
.
deserializeObject
(
stateData
,
userClassloader
);
this
.
windowBuffer
=
windowBufferFactory
.
create
();
int
numElements
=
in
.
readInt
();
...
...
@@ -465,10 +462,9 @@ public class WindowOperator<K, IN, OUT, W extends Window>
out
.
writeLong
(
watermarkTimer
);
out
.
writeLong
(
processingTimeTimer
);
ByteArrayOutputStream
baos
=
new
ByteArrayOutputStream
();
SerializationUtils
.
serialize
(
state
,
baos
);
out
.
writeInt
(
baos
.
size
());
out
.
write
(
baos
.
toByteArray
(),
0
,
baos
.
size
());
byte
[]
serializedState
=
InstantiationUtil
.
serializeObject
(
state
);
out
.
writeInt
(
serializedState
.
length
);
out
.
write
(
serializedState
,
0
,
serializedState
.
length
);
MultiplexingStreamRecordSerializer
<
IN
>
recordSerializer
=
new
MultiplexingStreamRecordSerializer
<>(
inputSerializer
);
out
.
writeInt
(
windowBuffer
.
size
());
...
...
@@ -608,10 +604,11 @@ public class WindowOperator<K, IN, OUT, W extends Window>
public
void
restoreState
(
StreamTaskState
taskState
)
throws
Exception
{
super
.
restoreState
(
taskState
);
final
ClassLoader
userClassloader
=
getUserCodeClassloader
();
@SuppressWarnings
(
"unchecked"
)
StateHandle
<
DataInputView
>
inputState
=
(
StateHandle
<
DataInputView
>)
taskState
.
getOperatorState
();
DataInputView
in
=
inputState
.
getState
(
getUserCodeClassloader
()
);
DataInputView
in
=
inputState
.
getState
(
userClassloader
);
int
numKeys
=
in
.
readInt
();
this
.
windows
=
new
HashMap
<>(
numKeys
);
...
...
@@ -621,7 +618,7 @@ public class WindowOperator<K, IN, OUT, W extends Window>
for
(
int
i
=
0
;
i
<
numKeys
;
i
++)
{
int
numWindows
=
in
.
readInt
();
for
(
int
j
=
0
;
j
<
numWindows
;
j
++)
{
Context
context
=
new
Context
(
in
);
Context
context
=
new
Context
(
in
,
userClassloader
);
Map
<
W
,
Context
>
keyWindows
=
windows
.
get
(
context
.
key
);
if
(
keyWindows
==
null
)
{
keyWindows
=
new
HashMap
<>(
numWindows
);
...
...
tools/maven/checkstyle.xml
浏览文件 @
e058abba
...
...
@@ -54,11 +54,18 @@ under the License.
<module
name=
"IllegalImport"
>
<property
name=
"illegalPkgs"
value=
"org.apache.flink.shaded"
/>
</module>
<!-- forbid use of commons lang validate -->
<module
name=
"Regexp"
>
<property
name=
"format"
value=
"org\.apache\.commons\.lang3\.Validate"
/>
<property
name=
"illegalPattern"
value=
"true"
/>
<property
name=
"message"
value=
"Use Guava Checks instead of Commons Validate. Please refer to the coding guidelines."
/>
</module>
<!-- forbid the use of org.apache.commons.lang.SerializationUtils -->
<module
name=
"Regexp"
>
<property
name=
"format"
value=
"org\.apache\.commons\.lang\.SerializationUtils"
/>
<property
name=
"illegalPattern"
value=
"true"
/>
<property
name=
"message"
value=
"Use Flink's InstantiationUtil instead of common's SerializationUtils"
/>
</module>
<module
name=
"NeedBraces"
>
<property
name=
"tokens"
value=
"LITERAL_IF, LITERAL_ELSE"
/>
</module>
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录