Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
269b0991
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,发现更多精彩内容 >>
提交
269b0991
编写于
7月 14, 2014
作者:
G
ghermann
提交者:
Stephan Ewen
8月 18, 2014
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[streaming] Added batch timeout
上级
26a5266b
变更
12
隐藏空白更改
内联
并排
Showing
12 changed file
with
91 addition
and
29 deletion
+91
-29
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/DataStream.java
...c/main/java/eu/stratosphere/streaming/api/DataStream.java
+1
-1
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/JobGraphBuilder.java
...n/java/eu/stratosphere/streaming/api/JobGraphBuilder.java
+5
-2
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/StreamCollector.java
...n/java/eu/stratosphere/streaming/api/StreamCollector.java
+22
-5
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/StreamExecutionEnvironment.java
...tratosphere/streaming/api/StreamExecutionEnvironment.java
+6
-3
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentHelper.java
.../streaming/api/streamcomponent/StreamComponentHelper.java
+10
-8
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/ArrayStreamRecord.java
...osphere/streaming/api/streamrecord/ArrayStreamRecord.java
+7
-2
flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/BatchReduceTest.java
...t/java/eu/stratosphere/streaming/api/BatchReduceTest.java
+1
-1
flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/BatchTest.java
...rc/test/java/eu/stratosphere/streaming/api/BatchTest.java
+1
-1
flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/FlatMapTest.java
.../test/java/eu/stratosphere/streaming/api/FlatMapTest.java
+8
-3
flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/MapTest.java
.../src/test/java/eu/stratosphere/streaming/api/MapTest.java
+1
-1
flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/StreamCollectorTest.java
...va/eu/stratosphere/streaming/api/StreamCollectorTest.java
+16
-2
flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamrecord/ArrayStreamRecordTest.java
...ere/streaming/api/streamrecord/ArrayStreamRecordTest.java
+13
-0
未找到文件。
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/DataStream.java
浏览文件 @
269b0991
...
...
@@ -94,7 +94,7 @@ public class DataStream<T extends Tuple> {
public
DataStream
<
T
>
batch
(
int
batchSize
)
{
return
context
.
setBatchSize
(
this
,
batchSize
);
}
public
<
R
extends
Tuple
>
DataStream
<
R
>
flatMap
(
FlatMapFunction
<
T
,
R
>
flatMapper
)
{
return
context
.
addFunction
(
"flatMap"
,
this
,
flatMapper
,
new
FlatMapInvokable
<
T
,
R
>(
flatMapper
));
...
...
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/JobGraphBuilder.java
浏览文件 @
269b0991
...
...
@@ -65,6 +65,7 @@ public class JobGraphBuilder {
protected
int
maxParallelism
;
protected
FaultToleranceType
faultToleranceType
;
private
int
batchSize
;
private
long
batchTimeout
;
/**
* Creates a new JobGraph with the given name
...
...
@@ -99,9 +100,10 @@ public class JobGraphBuilder {
this
(
jobGraphName
,
FaultToleranceType
.
NONE
);
}
public
JobGraphBuilder
(
String
jobGraphName
,
FaultToleranceType
faultToleranceType
,
int
batchSize
)
{
public
JobGraphBuilder
(
String
jobGraphName
,
FaultToleranceType
faultToleranceType
,
int
defaultBatchSize
,
long
defaultBatchTimeoutMillis
)
{
this
(
jobGraphName
,
faultToleranceType
);
this
.
batchSize
=
batchSize
;
this
.
batchSize
=
defaultBatchSize
;
this
.
batchTimeout
=
defaultBatchTimeoutMillis
;
}
/**
...
...
@@ -262,6 +264,7 @@ public class JobGraphBuilder {
config
.
setClass
(
"userfunction"
,
InvokableClass
);
config
.
setString
(
"componentName"
,
componentName
);
config
.
setInteger
(
"batchSize"
,
batchSize
);
config
.
setLong
(
"batchTimeout"
,
batchTimeout
);
// config.setBytes("operator", getSerializedFunction());
config
.
setInteger
(
"faultToleranceType"
,
faultToleranceType
.
id
);
...
...
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/StreamCollector.java
浏览文件 @
269b0991
...
...
@@ -28,37 +28,54 @@ public class StreamCollector<T extends Tuple> implements Collector<T> {
protected
StreamRecord
streamRecord
;
protected
int
batchSize
;
protected
long
batchTimeout
;
protected
int
counter
=
0
;
protected
int
channelID
;
private
long
timeOfLastRecordEmitted
=
System
.
currentTimeMillis
();;
private
List
<
RecordWriter
<
StreamRecord
>>
outputs
;
public
StreamCollector
(
int
batchSize
,
int
channelID
,
public
StreamCollector
(
int
batchSize
,
long
batchTimeout
,
int
channelID
,
SerializationDelegate
<
Tuple
>
serializationDelegate
,
List
<
RecordWriter
<
StreamRecord
>>
outputs
)
{
this
.
batchSize
=
batchSize
;
this
.
batchTimeout
=
batchTimeout
;
this
.
streamRecord
=
new
ArrayStreamRecord
(
batchSize
);
this
.
streamRecord
.
setSeralizationDelegate
(
serializationDelegate
);
this
.
channelID
=
channelID
;
this
.
outputs
=
outputs
;
}
public
StreamCollector
(
int
batchSize
,
int
channelID
,
public
StreamCollector
(
int
batchSize
,
long
batchTimeout
,
int
channelID
,
SerializationDelegate
<
Tuple
>
serializationDelegate
)
{
this
(
batchSize
,
channelID
,
serializationDelegate
,
null
);
this
(
batchSize
,
batchTimeout
,
channelID
,
serializationDelegate
,
null
);
}
// TODO reconsider emitting mechanism at timeout (find a place to timeout)
@Override
public
void
collect
(
T
tuple
)
{
streamRecord
.
setTuple
(
counter
,
StreamRecord
.
copyTuple
(
tuple
));
counter
++;
if
(
counter
>=
batchSize
)
{
counter
=
0
;
streamRecord
.
setId
(
channelID
);
emit
(
streamRecord
);
timeOfLastRecordEmitted
=
System
.
currentTimeMillis
();
}
else
{
timeout
();
}
}
public
void
timeout
()
{
if
(
timeOfLastRecordEmitted
+
batchTimeout
<
System
.
currentTimeMillis
())
{
StreamRecord
truncatedRecord
=
new
ArrayStreamRecord
(
streamRecord
,
counter
);
emit
(
truncatedRecord
);
timeOfLastRecordEmitted
=
System
.
currentTimeMillis
();
}
}
private
void
emit
(
StreamRecord
streamRecord
)
{
counter
=
0
;
streamRecord
.
setId
(
channelID
);
if
(
outputs
==
null
)
{
System
.
out
.
println
(
streamRecord
);
}
else
{
...
...
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/StreamExecutionEnvironment.java
浏览文件 @
269b0991
...
...
@@ -33,15 +33,18 @@ import eu.stratosphere.util.Collector;
public
class
StreamExecutionEnvironment
{
JobGraphBuilder
jobGraphBuilder
;
public
StreamExecutionEnvironment
(
int
defaultBatchSize
)
{
public
StreamExecutionEnvironment
(
int
defaultBatchSize
,
long
defaultBatchTimeoutMillis
)
{
if
(
defaultBatchSize
<
1
)
{
throw
new
IllegalArgumentException
(
"Batch size must be positive."
);
}
jobGraphBuilder
=
new
JobGraphBuilder
(
"jobGraph"
,
FaultToleranceType
.
NONE
,
defaultBatchSize
);
if
(
defaultBatchTimeoutMillis
<
1
)
{
throw
new
IllegalArgumentException
(
"Batch timeout must be positive."
);
}
jobGraphBuilder
=
new
JobGraphBuilder
(
"jobGraph"
,
FaultToleranceType
.
NONE
,
defaultBatchSize
,
defaultBatchTimeoutMillis
);
}
public
StreamExecutionEnvironment
()
{
this
(
1
);
this
(
1
,
1000
);
}
private
static
class
DummySource
extends
UserSourceInvokable
<
Tuple1
<
String
>>
{
...
...
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentHelper.java
浏览文件 @
269b0991
...
...
@@ -108,7 +108,9 @@ public final class StreamComponentHelper<T extends AbstractInvokable> {
public
StreamCollector
<
Tuple
>
setCollector
(
Configuration
taskConfiguration
,
int
id
,
List
<
RecordWriter
<
StreamRecord
>>
outputs
)
{
int
batchSize
=
taskConfiguration
.
getInteger
(
"batchSize"
,
-
1
);
collector
=
new
StreamCollector
<
Tuple
>(
batchSize
,
id
,
outSerializationDelegate
,
outputs
);
long
batchTimeout
=
taskConfiguration
.
getLong
(
"batchTimeout"
,
-
1
);
collector
=
new
StreamCollector
<
Tuple
>(
batchSize
,
batchTimeout
,
id
,
outSerializationDelegate
,
outputs
);
return
collector
;
}
...
...
@@ -121,7 +123,7 @@ public final class StreamComponentHelper<T extends AbstractInvokable> {
try
{
ObjectInputStream
in
=
new
ObjectInputStream
(
new
ByteArrayInputStream
(
operatorBytes
));
Object
function
=
in
.
readObject
();
if
(
operatorName
.
equals
(
"flatMap"
))
{
setSerializer
(
function
,
FlatMapFunction
.
class
);
}
else
if
(
operatorName
.
equals
(
"map"
))
{
...
...
@@ -151,21 +153,21 @@ public final class StreamComponentHelper<T extends AbstractInvokable> {
}
}
private
void
setSerializer
(
Object
function
,
Class
<?
extends
AbstractFunction
>
clazz
)
{
inTupleTypeInfo
=
(
TupleTypeInfo
)
TypeExtractor
.
createTypeInfo
(
clazz
,
function
.
getClass
(),
0
,
null
,
null
);
inTupleTypeInfo
=
(
TupleTypeInfo
)
TypeExtractor
.
createTypeInfo
(
clazz
,
function
.
getClass
(),
0
,
null
,
null
);
inTupleSerializer
=
inTupleTypeInfo
.
createSerializer
();
inDeserializationDelegate
=
new
DeserializationDelegate
<
Tuple
>(
inTupleSerializer
);
outTupleTypeInfo
=
(
TupleTypeInfo
)
TypeExtractor
.
createTypeInfo
(
clazz
,
function
.
getClass
(),
1
,
null
,
null
);
outTupleTypeInfo
=
(
TupleTypeInfo
)
TypeExtractor
.
createTypeInfo
(
clazz
,
function
.
getClass
(),
1
,
null
,
null
);
outTupleSerializer
=
outTupleTypeInfo
.
createSerializer
();
outSerializationDelegate
=
new
SerializationDelegate
<
Tuple
>(
outTupleSerializer
);
}
public
AbstractRecordReader
getConfigInputs
(
T
taskBase
,
Configuration
taskConfiguration
)
throws
StreamComponentException
{
int
numberOfInputs
=
taskConfiguration
.
getInteger
(
"numberOfInputs"
,
0
);
...
...
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/ArrayStreamRecord.java
浏览文件 @
269b0991
...
...
@@ -45,14 +45,19 @@ public class ArrayStreamRecord extends StreamRecord {
}
public
ArrayStreamRecord
(
StreamRecord
record
)
{
tupleBatch
=
new
Tuple
[
record
.
getBatchSize
()];
this
(
record
,
record
.
getBatchSize
());
}
public
ArrayStreamRecord
(
StreamRecord
record
,
int
truncatedSize
)
{
tupleBatch
=
new
Tuple
[
truncatedSize
];
this
.
uid
=
new
UID
(
Arrays
.
copyOf
(
record
.
getId
().
getId
(),
20
));
for
(
int
i
=
0
;
i
<
record
.
getBatchSize
()
;
++
i
)
{
for
(
int
i
=
0
;
i
<
truncatedSize
;
++
i
)
{
this
.
tupleBatch
[
i
]
=
copyTuple
(
record
.
getTuple
(
i
));
}
this
.
batchSize
=
tupleBatch
.
length
;
}
/**
* Creates a new batch of records containing the given Tuple array as
* elements
...
...
flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/BatchReduceTest.java
浏览文件 @
269b0991
...
...
@@ -72,7 +72,7 @@ public class BatchReduceTest {
@Test
public
void
test
()
throws
Exception
{
StreamExecutionEnvironment
context
=
new
StreamExecutionEnvironment
(
4
);
StreamExecutionEnvironment
context
=
new
StreamExecutionEnvironment
(
4
,
1000
);
DataStream
<
Tuple1
<
Double
>>
dataStream0
=
context
.
addSource
(
new
MySource
()).
batchReduce
(
new
MyBatchReduce
()).
addSink
(
new
MySink
());
context
.
execute
();
...
...
flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/BatchTest.java
浏览文件 @
269b0991
...
...
@@ -47,10 +47,10 @@ public class BatchTest {
DataStream
<
Tuple1
<
String
>>
dataStream
=
context
.
addSource
(
new
MySource
())
.
flatMap
(
new
MyMap
()).
batch
(
2
)
.
flatMap
(
new
MyMap
()).
batch
(
4
)
.
flatMap
(
new
MyMap
()).
batch
(
2
)
.
flatMap
(
new
MyMap
()).
batch
(
5
)
.
flatMap
(
new
MyMap
()).
batch
(
4
)
.
addSink
(
new
MySink
());
context
.
execute
();
...
...
flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/FlatMapTest.java
浏览文件 @
269b0991
...
...
@@ -71,12 +71,17 @@ public class FlatMapTest {
public
void
test
()
throws
Exception
{
try
{
StreamExecutionEnvironment
context2
=
new
StreamExecutionEnvironment
(
0
);
StreamExecutionEnvironment
context2
=
new
StreamExecutionEnvironment
(
0
,
1000
);
fail
();
}
catch
(
IllegalArgumentException
e
)
{
try
{
StreamExecutionEnvironment
context2
=
new
StreamExecutionEnvironment
(
1
,
0
);
fail
();
}
catch
(
IllegalArgumentException
e2
)
{
}
}
StreamExecutionEnvironment
context
=
new
StreamExecutionEnvironment
(
2
);
StreamExecutionEnvironment
context
=
new
StreamExecutionEnvironment
(
2
,
1000
);
DataStream
<
Tuple1
<
String
>>
dataStream0
=
context
.
addSource
(
new
MySource
());
DataStream
<
Tuple1
<
String
>>
dataStream1
=
context
.
addDummySource
().
connectWith
(
dataStream0
)
...
...
@@ -96,7 +101,7 @@ public class FlatMapTest {
FlatMapFunction
<
Tuple
,
Tuple
>
f
=
(
FlatMapFunction
<
Tuple
,
Tuple
>)
in
.
readObject
();
StreamCollector
<
Tuple
>
s
=
new
StreamCollector
<
Tuple
>(
1
,
1
,
null
);
StreamCollector
<
Tuple
>
s
=
new
StreamCollector
<
>(
1
,
1000
,
1
,
null
);
Tuple
t
=
new
Tuple1
<
String
>(
"asd"
);
f
.
flatMap
(
t
,
s
);
...
...
flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/MapTest.java
浏览文件 @
269b0991
...
...
@@ -72,7 +72,7 @@ public class MapTest {
MapFunction
<
Tuple
,
Tuple
>
f
=
(
MapFunction
<
Tuple
,
Tuple
>)
in
.
readObject
();
StreamCollector
<
Tuple
>
s
=
new
StreamCollector
<
Tuple
>(
1
,
1
,
null
);
StreamCollector
<
Tuple
>
s
=
new
StreamCollector
<
Tuple
>(
1
,
1
000
,
1
,
null
);
Tuple
t
=
new
Tuple1
<
String
>(
"asd"
);
s
.
collect
(
f
.
map
(
t
));
...
...
flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/StreamCollectorTest.java
浏览文件 @
269b0991
...
...
@@ -25,13 +25,13 @@ public class StreamCollectorTest {
@Test
public
void
testStreamCollector
()
{
StreamCollector
collector
=
new
StreamCollector
(
10
,
0
,
null
);
StreamCollector
collector
=
new
StreamCollector
(
10
,
1000
,
0
,
null
);
assertEquals
(
10
,
collector
.
batchSize
);
}
@Test
public
void
testCollect
()
{
StreamCollector
collector
=
new
StreamCollector
(
2
,
0
,
null
);
StreamCollector
collector
=
new
StreamCollector
(
2
,
1000
,
0
,
null
);
collector
.
collect
(
new
Tuple1
<
Integer
>(
3
));
collector
.
collect
(
new
Tuple1
<
Integer
>(
4
));
collector
.
collect
(
new
Tuple1
<
Integer
>(
5
));
...
...
@@ -39,6 +39,20 @@ public class StreamCollectorTest {
}
@Test
public
void
testBatchSize
()
throws
InterruptedException
{
System
.
out
.
println
(
"---------------"
);
StreamCollector
collector
=
new
StreamCollector
(
3
,
100
,
0
,
null
);
collector
.
collect
(
new
Tuple1
<
Integer
>(
0
));
collector
.
collect
(
new
Tuple1
<
Integer
>(
0
));
collector
.
collect
(
new
Tuple1
<
Integer
>(
0
));
Thread
.
sleep
(
200
);
collector
.
collect
(
new
Tuple1
<
Integer
>(
2
));
collector
.
collect
(
new
Tuple1
<
Integer
>(
3
));
System
.
out
.
println
(
"---------------"
);
}
@Test
public
void
testClose
()
{
}
...
...
flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamrecord/ArrayStreamRecordTest.java
浏览文件 @
269b0991
...
...
@@ -97,4 +97,17 @@ public class ArrayStreamRecordTest {
}
@Test
public
void
truncatedSizeTest
()
{
StreamRecord
record
=
new
ArrayStreamRecord
(
4
);
record
.
setTuple
(
0
,
new
Tuple1
<
Integer
>(
0
));
record
.
setTuple
(
1
,
new
Tuple1
<
Integer
>(
1
));
record
.
setTuple
(
2
,
new
Tuple1
<
Integer
>(
2
));
record
.
setTuple
(
3
,
new
Tuple1
<
Integer
>(
3
));
StreamRecord
truncatedRecord
=
new
ArrayStreamRecord
(
record
,
2
);
assertEquals
(
2
,
truncatedRecord
.
batchSize
);
assertEquals
(
0
,
truncatedRecord
.
getTuple
(
0
).
getField
(
0
));
assertEquals
(
1
,
truncatedRecord
.
getTuple
(
1
).
getField
(
0
));
}
}
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录