Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
a0363782
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,发现更多精彩内容 >>
提交
a0363782
编写于
11月 17, 2014
作者:
G
ghermann
提交者:
mbalassi
11月 19, 2014
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[streaming] Fixed async buffer sending at end of AbstractInvokable
上级
d92a24a1
变更
3
隐藏空白更改
内联
并排
Showing
3 changed file
with
172 addition
and
143 deletion
+172
-143
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java
...apache/flink/streaming/api/invokable/StreamInvokable.java
+1
-0
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java
...pache/flink/streaming/api/streamvertex/OutputHandler.java
+5
-2
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamRecordWriter.java
...ava/org/apache/flink/streaming/io/StreamRecordWriter.java
+166
-141
未找到文件。
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java
浏览文件 @
a0363782
...
...
@@ -152,6 +152,7 @@ public abstract class StreamInvokable<IN, OUT> implements Serializable {
*/
public
void
close
()
throws
Exception
{
isRunning
=
false
;
collector
.
close
();
if
(
userFunction
instanceof
RichFunction
)
{
((
RichFunction
)
userFunction
).
close
();
}
...
...
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/OutputHandler.java
浏览文件 @
a0363782
...
...
@@ -129,7 +129,6 @@ public class OutputHandler<OUT> {
LOG
.
trace
(
"StreamRecordWriter initiated with {} bufferTimeout for {}"
,
bufferTimeout
,
streamVertex
.
getClass
().
getSimpleName
());
}
}
else
{
output
=
new
RecordWriter
<
SerializationDelegate
<
StreamRecord
<
OUT
>>>(
streamVertex
,
outputPartitioner
);
...
...
@@ -155,7 +154,11 @@ public class OutputHandler<OUT> {
public
void
flushOutputs
()
throws
IOException
,
InterruptedException
{
for
(
RecordWriter
<
SerializationDelegate
<
StreamRecord
<
OUT
>>>
output
:
outputs
)
{
output
.
flush
();
if
(
output
instanceof
StreamRecordWriter
)
{
((
StreamRecordWriter
<
SerializationDelegate
<
StreamRecord
<
OUT
>>>)
output
).
close
();
}
else
{
output
.
flush
();
}
}
}
...
...
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamRecordWriter.java
浏览文件 @
a0363782
/*
* 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
* 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.streaming.io
;
import
java.io.IOException
;
import
org.apache.flink.core.io.IOReadableWritable
;
import
org.apache.flink.runtime.io.network.Buffer
;
import
org.apache.flink.runtime.io.network.api.ChannelSelector
;
import
org.apache.flink.runtime.io.network.api.RecordWriter
;
import
org.apache.flink.runtime.io.network.api.RoundRobinChannelSelector
;
import
org.apache.flink.runtime.io.network.bufferprovider.BufferProvider
;
import
org.apache.flink.runtime.io.network.serialization.RecordSerializer
;
import
org.apache.flink.runtime.io.network.serialization.SpanningRecordSerializer
;
import
org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable
;
public
class
StreamRecordWriter
<
T
extends
IOReadableWritable
>
extends
RecordWriter
<
T
>
{
private
final
BufferProvider
bufferPool
;
private
final
ChannelSelector
<
T
>
channelSelector
;
private
int
numChannels
;
private
long
timeout
;
/** RecordSerializer per outgoing channel */
private
RecordSerializer
<
T
>[]
serializers
;
// -----------------------------------------------------------------------------------------------------------------
public
StreamRecordWriter
(
AbstractInvokable
invokable
)
{
this
(
invokable
,
new
RoundRobinChannelSelector
<
T
>(),
100
);
}
public
StreamRecordWriter
(
AbstractInvokable
invokable
,
ChannelSelector
<
T
>
channelSelector
)
{
this
(
invokable
,
channelSelector
,
100
);
}
public
StreamRecordWriter
(
AbstractInvokable
invokable
,
ChannelSelector
<
T
>
channelSelector
,
long
timeout
)
{
// initialize the gate
super
(
invokable
);
this
.
timeout
=
timeout
;
this
.
bufferPool
=
invokable
.
getEnvironment
().
getOutputBufferProvider
();
this
.
channelSelector
=
channelSelector
;
}
// -----------------------------------------------------------------------------------------------------------------
@SuppressWarnings
(
"unchecked"
)
@Override
public
void
initializeSerializers
()
{
this
.
numChannels
=
this
.
outputGate
.
getNumChannels
();
this
.
serializers
=
new
RecordSerializer
[
numChannels
];
for
(
int
i
=
0
;
i
<
this
.
numChannels
;
i
++)
{
this
.
serializers
[
i
]
=
new
SpanningRecordSerializer
<
T
>();
*/
package
org.apache.flink.streaming.io
;
import
java.io.IOException
;
import
java.util.ArrayList
;
import
org.apache.flink.core.io.IOReadableWritable
;
import
org.apache.flink.runtime.io.network.Buffer
;
import
org.apache.flink.runtime.io.network.api.ChannelSelector
;
import
org.apache.flink.runtime.io.network.api.RecordWriter
;
import
org.apache.flink.runtime.io.network.api.RoundRobinChannelSelector
;
import
org.apache.flink.runtime.io.network.bufferprovider.BufferProvider
;
import
org.apache.flink.runtime.io.network.serialization.RecordSerializer
;
import
org.apache.flink.runtime.io.network.serialization.SpanningRecordSerializer
;
import
org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable
;
public
class
StreamRecordWriter
<
T
extends
IOReadableWritable
>
extends
RecordWriter
<
T
>
{
private
final
BufferProvider
bufferPool
;
private
final
ChannelSelector
<
T
>
channelSelector
;
private
int
numChannels
;
private
long
timeout
;
private
OutputFlusher
outputFlusher
;
/** RecordSerializer per outgoing channel */
private
RecordSerializer
<
T
>[]
serializers
;
private
ArrayList
<
TargetChannel
>
targetChannels
;
// -----------------------------------------------------------------------------------------------------------------
public
StreamRecordWriter
(
AbstractInvokable
invokable
)
{
this
(
invokable
,
new
RoundRobinChannelSelector
<
T
>(),
1000
);
}
public
StreamRecordWriter
(
AbstractInvokable
invokable
,
ChannelSelector
<
T
>
channelSelector
)
{
this
(
invokable
,
channelSelector
,
1000
);
}
public
StreamRecordWriter
(
AbstractInvokable
invokable
,
ChannelSelector
<
T
>
channelSelector
,
long
timeout
)
{
// initialize the gate
super
(
invokable
);
this
.
timeout
=
timeout
;
this
.
bufferPool
=
invokable
.
getEnvironment
().
getOutputBufferProvider
();
this
.
channelSelector
=
channelSelector
;
}
// -----------------------------------------------------------------------------------------------------------------
@SuppressWarnings
(
"unchecked"
)
@Override
public
void
initializeSerializers
()
{
this
.
numChannels
=
this
.
outputGate
.
getNumChannels
();
this
.
serializers
=
new
RecordSerializer
[
numChannels
];
this
.
targetChannels
=
new
ArrayList
<
TargetChannel
>(
numChannels
);
for
(
int
i
=
0
;
i
<
this
.
numChannels
;
i
++)
{
this
.
serializers
[
i
]
=
new
SpanningRecordSerializer
<
T
>();
this
.
targetChannels
.
add
(
new
TargetChannel
(
i
));
}
//start a separate thread to handle positive flush intervals
if
(
timeout
>
0
)
{
(
new
OutputFlusher
()).
start
();
}
}
@Override
public
void
emit
(
final
T
record
)
throws
IOException
,
InterruptedException
{
for
(
int
targetChannel
:
this
.
channelSelector
.
selectChannels
(
record
,
this
.
numChannels
))
{
// serialize with corresponding serializer and send full buffer
RecordSerializer
<
T
>
serializer
=
this
.
serializers
[
targetChannel
];
synchronized
(
serializer
)
{
RecordSerializer
.
SerializationResult
result
=
serializer
.
addRecord
(
record
);
while
(
result
.
isFullBuffer
())
{
Buffer
buffer
=
serializer
.
getCurrentBuffer
();
if
(
buffer
!=
null
)
{
sendBuffer
(
buffer
,
targetChannel
);
}
buffer
=
this
.
bufferPool
.
requestBufferBlocking
(
this
.
bufferPool
.
getBufferSize
());
result
=
serializer
.
setNextBuffer
(
buffer
);
}
outputFlusher
=
new
OutputFlusher
();
outputFlusher
.
start
();
}
@Override
public
void
emit
(
final
T
record
)
throws
IOException
,
InterruptedException
{
for
(
int
targetChannel
:
this
.
channelSelector
.
selectChannels
(
record
,
this
.
numChannels
))
{
targetChannels
.
get
(
targetChannel
).
emit
(
record
);
}
}
@Override
public
void
flush
()
throws
IOException
,
InterruptedException
{
for
(
int
targetChannel
=
0
;
targetChannel
<
this
.
numChannels
;
targetChannel
++)
{
targetChannels
.
get
(
targetChannel
).
flush
();
}
}
public
void
close
()
{
try
{
if
(
outputFlusher
!=
null
)
{
outputFlusher
.
terminate
();
outputFlusher
.
join
();
}
flush
();
}
catch
(
IOException
e
)
{
e
.
printStackTrace
();
}
catch
(
InterruptedException
e
)
{
e
.
printStackTrace
();
}
}
private
class
TargetChannel
{
private
int
targetChannel
;
private
RecordSerializer
<
T
>
serializer
;
public
TargetChannel
(
int
targetChannel
)
{
this
.
targetChannel
=
targetChannel
;
this
.
serializer
=
serializers
[
targetChannel
];
}
public
synchronized
void
emit
(
final
T
record
)
throws
IOException
,
InterruptedException
{
RecordSerializer
.
SerializationResult
result
=
serializer
.
addRecord
(
record
);
while
(
result
.
isFullBuffer
())
{
Buffer
buffer
=
serializer
.
getCurrentBuffer
();
if
(
buffer
!=
null
)
{
sendBuffer
(
buffer
,
targetChannel
);
}
buffer
=
bufferPool
.
requestBufferBlocking
(
bufferPool
.
getBufferSize
());
result
=
serializer
.
setNextBuffer
(
buffer
);
}
if
(
timeout
==
0
){
flush
();
}
}
}
@Override
public
void
flush
()
throws
IOException
,
InterruptedException
{
for
(
int
targetChannel
=
0
;
targetChannel
<
this
.
numChannels
;
targetChannel
++)
{
RecordSerializer
<
T
>
serializer
=
this
.
serializers
[
targetChannel
];
synchronized
(
serializer
)
{
Buffer
buffer
=
serializer
.
getCurrentBuffer
();
if
(
buffer
!=
null
)
{
sendBuffer
(
buffer
,
targetChannel
);
}
serializer
.
clear
();
}
}
}
private
class
OutputFlusher
extends
Thread
{
@Override
public
void
run
()
{
while
(!
outputGate
.
isClosed
())
{
try
{
Thread
.
sleep
(
timeout
);
flush
();
}
catch
(
Exception
e
)
{
throw
new
RuntimeException
(
e
);
}
}
}
}
}
}
public
synchronized
void
flush
()
throws
IOException
,
InterruptedException
{
Buffer
buffer
=
serializer
.
getCurrentBuffer
();
if
(
buffer
!=
null
)
{
sendBuffer
(
buffer
,
targetChannel
);
}
serializer
.
clear
();
}
}
private
class
OutputFlusher
extends
Thread
{
private
boolean
running
=
true
;
public
void
terminate
()
{
running
=
false
;
}
@Override
public
void
run
()
{
while
(
running
&&
!
outputGate
.
isClosed
())
{
try
{
flush
();
Thread
.
sleep
(
timeout
);
}
catch
(
Exception
e
)
{
throw
new
RuntimeException
(
e
);
}
}
}
}
}
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录