Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
8da89a74
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,发现更多精彩内容 >>
提交
8da89a74
编写于
7月 14, 2014
作者:
G
ghermann
提交者:
Stephan Ewen
8月 18, 2014
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[streaming] Added FileSourceFunction
上级
ed422377
变更
10
隐藏空白更改
内联
并排
Showing
10 changed file
with
170 addition
and
50 deletion
+170
-50
flink-addons/flink-streaming/src/main/java/eu/stratosphere/api/datastream/BatchReduceInvokable.java
.../eu/stratosphere/api/datastream/BatchReduceInvokable.java
+25
-0
flink-addons/flink-streaming/src/main/java/eu/stratosphere/api/datastream/DataStream.java
.../main/java/eu/stratosphere/api/datastream/DataStream.java
+6
-3
flink-addons/flink-streaming/src/main/java/eu/stratosphere/api/datastream/FileSourceFunction.java
...va/eu/stratosphere/api/datastream/FileSourceFunction.java
+34
-0
flink-addons/flink-streaming/src/main/java/eu/stratosphere/api/datastream/StreamExecutionEnvironment.java
...ratosphere/api/datastream/StreamExecutionEnvironment.java
+34
-4
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/JobGraphBuilder.java
...n/java/eu/stratosphere/streaming/api/JobGraphBuilder.java
+8
-3
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/UserSourceInvokable.java
...tosphere/streaming/api/invokable/UserSourceInvokable.java
+2
-1
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentHelper.java
.../streaming/api/streamcomponent/StreamComponentHelper.java
+20
-1
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountCounter.java
...sphere/streaming/examples/wordcount/WordCountCounter.java
+10
-10
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountSplitter.java
...phere/streaming/examples/wordcount/WordCountSplitter.java
+23
-25
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/main/java/eu/stratosphere/api/datastream/BatchReduceInvokable.java
0 → 100644
浏览文件 @
8da89a74
package
eu.stratosphere.api.datastream
;
import
java.util.Iterator
;
import
eu.stratosphere.api.java.functions.GroupReduceFunction
;
import
eu.stratosphere.api.java.tuple.Tuple
;
import
eu.stratosphere.streaming.api.StreamCollector
;
import
eu.stratosphere.streaming.api.invokable.UserTaskInvokable
;
import
eu.stratosphere.streaming.api.streamrecord.StreamRecord
;
public
class
BatchReduceInvokable
<
IN
extends
Tuple
,
OUT
extends
Tuple
>
extends
UserTaskInvokable
<
IN
,
OUT
>
{
private
static
final
long
serialVersionUID
=
1L
;
private
GroupReduceFunction
<
IN
,
OUT
>
reducer
;
public
BatchReduceInvokable
(
GroupReduceFunction
<
IN
,
OUT
>
reduceFunction
)
{
this
.
reducer
=
reduceFunction
;
}
@Override
public
void
invoke
(
StreamRecord
record
,
StreamCollector
<
OUT
>
collector
)
throws
Exception
{
Iterator
<
IN
>
iterator
=
(
Iterator
<
IN
>)
record
.
getBatchIterable
().
iterator
();
reducer
.
reduce
(
iterator
,
collector
);
}
}
flink-addons/flink-streaming/src/main/java/eu/stratosphere/api/datastream/DataStream.java
浏览文件 @
8da89a74
...
...
@@ -21,12 +21,11 @@ import java.util.Random;
import
eu.stratosphere.api.datastream.StreamExecutionEnvironment.ConnectionType
;
import
eu.stratosphere.api.java.functions.FlatMapFunction
;
import
eu.stratosphere.api.java.functions.GroupReduceFunction
;
import
eu.stratosphere.api.java.functions.MapFunction
;
import
eu.stratosphere.api.java.tuple.Tuple
;
import
eu.stratosphere.types.TypeInformation
;
//TODO:get batchsize from user -> put in config -> set in streamcomponenthelper for collector
//TODO:batchReduce -> tuple iterator over tuplebatch, out tuple (reduce function)
public
class
DataStream
<
T
extends
Tuple
>
{
private
final
StreamExecutionEnvironment
context
;
...
...
@@ -48,7 +47,7 @@ public class DataStream<T extends Tuple> {
if
(
context
==
null
)
{
throw
new
NullPointerException
(
"context is null"
);
}
// TODO add name based on component number an preferable sequential id
this
.
id
=
Long
.
toHexString
(
random
.
nextLong
())
+
Long
.
toHexString
(
random
.
nextLong
());
this
.
context
=
context
;
...
...
@@ -95,6 +94,10 @@ public class DataStream<T extends Tuple> {
return
context
.
addMapFunction
(
this
,
mapper
);
}
public
<
R
extends
Tuple
>
DataStream
<
R
>
flatMap
(
GroupReduceFunction
<
T
,
R
>
reducer
)
{
return
context
.
addBatchReduceFunction
(
this
,
reducer
);
}
public
DataStream
<
T
>
addSink
(
SinkFunction
<
T
>
sinkFunction
)
{
return
context
.
addSink
(
this
,
sinkFunction
);
}
...
...
flink-addons/flink-streaming/src/main/java/eu/stratosphere/api/datastream/FileSourceFunction.java
0 → 100644
浏览文件 @
8da89a74
package
eu.stratosphere.api.datastream
;
import
java.io.BufferedReader
;
import
java.io.FileReader
;
import
java.io.IOException
;
import
eu.stratosphere.api.java.tuple.Tuple1
;
import
eu.stratosphere.util.Collector
;
public
class
FileSourceFunction
extends
SourceFunction
<
Tuple1
<
String
>>
{
private
static
final
long
serialVersionUID
=
1L
;
private
final
String
path
;
private
Tuple1
<
String
>
outTuple
=
new
Tuple1
<
String
>();
public
FileSourceFunction
(
String
path
)
{
this
.
path
=
path
;
}
@Override
public
void
invoke
(
Collector
<
Tuple1
<
String
>>
collector
)
throws
IOException
{
BufferedReader
br
=
new
BufferedReader
(
new
FileReader
(
path
));
String
line
=
br
.
readLine
();
while
(
line
!=
null
)
{
if
(
line
!=
""
)
{
outTuple
.
f0
=
line
;
collector
.
collect
(
outTuple
);
}
line
=
br
.
readLine
();
}
br
.
close
();
}
}
flink-addons/flink-streaming/src/main/java/eu/stratosphere/api/datastream/StreamExecutionEnvironment.java
浏览文件 @
8da89a74
...
...
@@ -20,6 +20,7 @@ import java.io.IOException;
import
java.io.ObjectOutputStream
;
import
eu.stratosphere.api.java.functions.FlatMapFunction
;
import
eu.stratosphere.api.java.functions.GroupReduceFunction
;
import
eu.stratosphere.api.java.functions.MapFunction
;
import
eu.stratosphere.api.java.tuple.Tuple
;
import
eu.stratosphere.api.java.tuple.Tuple1
;
...
...
@@ -34,10 +35,15 @@ import eu.stratosphere.util.Collector;
public
class
StreamExecutionEnvironment
{
JobGraphBuilder
jobGraphBuilder
;
private
static
final
int
BATCH_SIZE
=
1
;
public
StreamExecutionEnvironment
(
int
batchSize
)
{
if
(
batchSize
<
1
)
{
throw
new
IllegalArgumentException
(
"Batch size must be positive."
);
}
jobGraphBuilder
=
new
JobGraphBuilder
(
"jobGraph"
,
FaultToleranceType
.
NONE
,
batchSize
);
}
public
StreamExecutionEnvironment
()
{
jobGraphBuilder
=
new
JobGraphBuilder
(
"jobGraph"
,
FaultToleranceType
.
NONE
);
this
(
1
);
}
private
static
class
DummySource
extends
UserSourceInvokable
<
Tuple1
<
String
>>
{
...
...
@@ -49,9 +55,8 @@ public class StreamExecutionEnvironment {
collector
.
collect
(
new
Tuple1
<
String
>(
"source"
));
}
}
}
public
static
enum
ConnectionType
{
SHUFFLE
,
BROADCAST
,
FIELD
}
...
...
@@ -122,6 +127,27 @@ public class StreamExecutionEnvironment {
return
returnStream
;
}
public
<
T
extends
Tuple
,
R
extends
Tuple
>
DataStream
<
R
>
addBatchReduceFunction
(
DataStream
<
T
>
inputStream
,
final
GroupReduceFunction
<
T
,
R
>
reducer
)
{
DataStream
<
R
>
returnStream
=
new
DataStream
<
R
>(
this
);
ByteArrayOutputStream
baos
=
new
ByteArrayOutputStream
();
ObjectOutputStream
oos
;
try
{
oos
=
new
ObjectOutputStream
(
baos
);
oos
.
writeObject
(
reducer
);
}
catch
(
IOException
e
)
{
e
.
printStackTrace
();
}
jobGraphBuilder
.
setTask
(
returnStream
.
getId
(),
new
BatchReduceInvokable
<
T
,
R
>(
reducer
),
"batchReduce"
,
baos
.
toByteArray
());
connectGraph
(
inputStream
,
returnStream
.
getId
());
return
returnStream
;
}
public
<
T
extends
Tuple
>
DataStream
<
T
>
addSink
(
DataStream
<
T
>
inputStream
,
SinkFunction
<
T
>
sinkFunction
)
{
DataStream
<
T
>
returnStream
=
new
DataStream
<
T
>(
this
);
...
...
@@ -182,6 +208,10 @@ public class StreamExecutionEnvironment {
return
returnStream
;
}
public
DataStream
<
Tuple1
<
String
>>
addFileSource
(
String
path
)
{
return
addSource
(
new
FileSourceFunction
(
path
));
}
public
DataStream
<
Tuple1
<
String
>>
addDummySource
()
{
DataStream
<
Tuple1
<
String
>>
returnStream
=
new
DataStream
<
Tuple1
<
String
>>(
this
);
...
...
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/JobGraphBuilder.java
浏览文件 @
8da89a74
...
...
@@ -64,7 +64,7 @@ public class JobGraphBuilder {
protected
String
maxParallelismVertexName
;
protected
int
maxParallelism
;
protected
FaultToleranceType
faultToleranceType
;
private
int
batchSize
;
/**
* Creates a new JobGraph with the given name
*
...
...
@@ -84,6 +84,7 @@ public class JobGraphBuilder {
log
.
debug
(
"JobGraph created"
);
}
this
.
faultToleranceType
=
faultToleranceType
;
batchSize
=
1
;
}
/**
...
...
@@ -97,6 +98,11 @@ public class JobGraphBuilder {
this
(
jobGraphName
,
FaultToleranceType
.
NONE
);
}
public
JobGraphBuilder
(
String
jobGraphName
,
FaultToleranceType
faultToleranceType
,
int
batchSize
)
{
this
(
jobGraphName
,
faultToleranceType
);
this
.
batchSize
=
batchSize
;
}
/**
* Adds source to the JobGraph by user defined object with no parallelism
*
...
...
@@ -111,7 +117,6 @@ public class JobGraphBuilder {
Configuration
config
=
setSource
(
sourceName
,
InvokableObject
,
1
,
1
);
config
.
setBytes
(
"operator"
,
serializedFunction
);
config
.
setString
(
"operatorName"
,
operatorName
);
}
/**
...
...
@@ -250,7 +255,7 @@ public class JobGraphBuilder {
Configuration
config
=
new
TaskConfig
(
component
.
getConfiguration
()).
getConfiguration
();
config
.
setClass
(
"userfunction"
,
InvokableClass
);
config
.
setString
(
"componentName"
,
componentName
);
config
.
setInteger
(
"batchSize"
,
batchSize
);
// config.setBytes("operator", getSerializedFunction());
config
.
setInteger
(
"faultToleranceType"
,
faultToleranceType
.
id
);
...
...
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/UserSourceInvokable.java
浏览文件 @
8da89a74
...
...
@@ -15,6 +15,7 @@
package
eu.stratosphere.streaming.api.invokable
;
import
java.io.IOException
;
import
java.io.Serializable
;
import
eu.stratosphere.api.java.tuple.Tuple
;
...
...
@@ -25,6 +26,6 @@ public abstract class UserSourceInvokable<OUT extends Tuple> extends StreamCompo
private
static
final
long
serialVersionUID
=
1L
;
public
abstract
void
invoke
(
Collector
<
OUT
>
collector
);
public
abstract
void
invoke
(
Collector
<
OUT
>
collector
)
throws
Exception
;
}
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentHelper.java
浏览文件 @
8da89a74
...
...
@@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
import
eu.stratosphere.api.datastream.SinkFunction
;
import
eu.stratosphere.api.java.functions.FlatMapFunction
;
import
eu.stratosphere.api.java.functions.GroupReduceFunction
;
import
eu.stratosphere.api.java.functions.MapFunction
;
import
eu.stratosphere.api.java.tuple.Tuple
;
import
eu.stratosphere.api.java.typeutils.TupleTypeInfo
;
...
...
@@ -104,10 +105,12 @@ public final class StreamComponentHelper<T extends AbstractInvokable> {
public
StreamCollector
<
Tuple
>
setCollector
(
Configuration
taskConfiguration
,
int
id
,
List
<
RecordWriter
<
StreamRecord
>>
outputs
)
{
collector
=
new
StreamCollector
<
Tuple
>(
1
,
id
,
outSerializationDelegate
,
outputs
);
int
batchSize
=
taskConfiguration
.
getInteger
(
"batchSize"
,
-
1
);
collector
=
new
StreamCollector
<
Tuple
>(
batchSize
,
id
,
outSerializationDelegate
,
outputs
);
return
collector
;
}
// TODO add type parameters to avoid redundant code
public
void
setSerializers
(
Configuration
taskConfiguration
)
{
byte
[]
operatorBytes
=
taskConfiguration
.
getBytes
(
"operator"
,
null
);
String
operatorName
=
taskConfiguration
.
getString
(
"operatorName"
,
""
);
...
...
@@ -147,6 +150,22 @@ public final class StreamComponentHelper<T extends AbstractInvokable> {
outTupleSerializer
=
outTupleTypeInfo
.
createSerializer
();
outSerializationDelegate
=
new
SerializationDelegate
<
Tuple
>(
outTupleSerializer
);
}
else
if
(
operatorName
.
equals
(
"batchReduce"
))
{
GroupReduceFunction
<
Tuple
,
Tuple
>
f
=
(
GroupReduceFunction
<
Tuple
,
Tuple
>)
in
.
readObject
();
inTupleTypeInfo
=
(
TupleTypeInfo
)
TypeExtractor
.
createTypeInfo
(
GroupReduceFunction
.
class
,
f
.
getClass
(),
0
,
null
,
null
);
inTupleSerializer
=
inTupleTypeInfo
.
createSerializer
();
inDeserializationDelegate
=
new
DeserializationDelegate
<
Tuple
>(
inTupleSerializer
);
outTupleTypeInfo
=
(
TupleTypeInfo
)
TypeExtractor
.
createTypeInfo
(
GroupReduceFunction
.
class
,
f
.
getClass
(),
1
,
null
,
null
);
outTupleSerializer
=
outTupleTypeInfo
.
createSerializer
();
outSerializationDelegate
=
new
SerializationDelegate
<
Tuple
>(
outTupleSerializer
);
}
else
if
(
operatorName
.
equals
(
"sink"
))
{
SinkFunction
<
Tuple
>
f
=
(
SinkFunction
<
Tuple
>)
in
.
readObject
();
...
...
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountCounter.java
浏览文件 @
8da89a74
...
...
@@ -18,22 +18,22 @@ package eu.stratosphere.streaming.examples.wordcount;
import
java.util.HashMap
;
import
java.util.Map
;
import
eu.stratosphere.api.java.functions.MapFunction
;
import
eu.stratosphere.api.java.tuple.Tuple1
;
import
eu.stratosphere.api.java.tuple.Tuple2
;
import
eu.stratosphere.streaming.api.invokable.UserTaskInvokable
;
import
eu.stratosphere.streaming.api.streamrecord.StreamRecord
;
public
class
WordCountCounter
extends
MapFunction
<
Tuple1
<
String
>,
Tuple2
<
String
,
Integer
>>
{
public
class
WordCountCounter
extends
UserTaskInvokable
{
private
static
final
long
serialVersionUID
=
1L
;
private
Map
<
String
,
Integer
>
wordCounts
=
new
HashMap
<
String
,
Integer
>();
private
String
word
=
""
;
private
Integer
count
=
0
;
private
Tuple2
<
String
,
Integer
>
outTuple
=
new
Tuple2
<
String
,
Integer
>(
);
private
StreamRecord
outRecord
=
new
StreamRecord
(
new
Tuple2
<
String
,
Integer
>()
);
@Override
public
Tuple2
<
String
,
Integer
>
map
(
Tuple1
<
String
>
inTuple
)
throws
Exception
{
word
=
inTuple
.
f0
;
public
void
invoke
(
StreamRecord
record
)
throws
Exception
{
word
=
record
.
getString
(
0
)
;
if
(
wordCounts
.
containsKey
(
word
))
{
count
=
wordCounts
.
get
(
word
)
+
1
;
...
...
@@ -43,10 +43,10 @@ public class WordCountCounter extends MapFunction<Tuple1<String>, Tuple2<String,
wordCounts
.
put
(
word
,
1
);
}
out
Tuple
.
f0
=
word
;
out
Tuple
.
f1
=
count
;
out
Record
.
setString
(
0
,
word
)
;
out
Record
.
setInteger
(
1
,
count
)
;
return
outTuple
;
emit
(
outRecord
)
;
// performanceCounter.count();
}
...
...
flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountSplitter.java
浏览文件 @
8da89a74
...
...
@@ -15,41 +15,39 @@
package
eu.stratosphere.streaming.examples.wordcount
;
import
eu.stratosphere.api.java.functions.FlatMapFunction
;
import
eu.stratosphere.api.java.tuple.Tuple1
;
import
eu.stratosphere.util.Collector
;
import
eu.stratosphere.streaming.api.invokable.UserTaskInvokable
;
import
eu.stratosphere.streaming.api.streamrecord.StreamRecord
;
import
eu.stratosphere.streaming.util.PerformanceCounter
;
import
eu.stratosphere.streaming.util.PerformanceTimer
;
public
class
WordCountSplitter
extends
FlatMapFunction
<
Tuple1
<
String
>,
Tuple1
<
String
>>
{
public
class
WordCountSplitter
extends
UserTaskInvokable
{
private
static
final
long
serialVersionUID
=
1L
;
private
String
[]
words
=
new
String
[]
{};
private
Tuple1
<
String
>
outTuple
=
new
Tuple1
<
String
>();
//TODO move the performance tracked version to a separate package and clean this
// PerformanceCounter pCounter = new
// PerformanceCounter("SplitterEmitCounter", 1000, 1000,
// "/home/strato/stratosphere-distrib/log/counter/Splitter" + channelID);
// PerformanceTimer pTimer = new PerformanceTimer("SplitterEmitTimer", 1000,
// 1000, true,
// "/home/strato/stratosphere-distrib/log/timer/Splitter" + channelID);
private
StreamRecord
outputRecord
=
new
StreamRecord
(
new
Tuple1
<
String
>());
PerformanceCounter
pCounter
=
new
PerformanceCounter
(
"SplitterEmitCounter"
,
1000
,
1000
,
"/home/strato/stratosphere-distrib/log/counter/Splitter"
+
channelID
);
PerformanceTimer
pTimer
=
new
PerformanceTimer
(
"SplitterEmitTimer"
,
1000
,
1000
,
true
,
"/home/strato/stratosphere-distrib/log/timer/Splitter"
+
channelID
);
@Override
public
void
flatMap
(
Tuple1
<
String
>
inTuple
,
Collector
<
Tuple1
<
String
>>
out
)
throws
Exception
{
public
void
invoke
(
StreamRecord
record
)
throws
Exception
{
words
=
inTuple
.
f0
.
split
(
" "
);
words
=
record
.
getString
(
0
)
.
split
(
" "
);
for
(
String
word
:
words
)
{
out
Tuple
.
f0
=
word
;
//
pTimer.startTimer();
out
.
collect
(
outTuple
);
//
pTimer.stopTimer();
//
pCounter.count();
out
putRecord
.
setString
(
0
,
word
)
;
pTimer
.
startTimer
();
emit
(
outputRecord
);
pTimer
.
stopTimer
();
pCounter
.
count
();
}
}
//
@Override
//
public String getResult() {
//
pCounter.writeCSV();
//
pTimer.writeCSV();
//
return "";
//
}
@Override
public
String
getResult
()
{
pCounter
.
writeCSV
();
pTimer
.
writeCSV
();
return
""
;
}
}
\ No newline at end of file
flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/FlatMapTest.java
浏览文件 @
8da89a74
...
...
@@ -16,6 +16,7 @@
package
eu.stratosphere.streaming.api
;
import
static
org
.
junit
.
Assert
.
assertTrue
;
import
static
org
.
junit
.
Assert
.
fail
;
import
java.io.ByteArrayInputStream
;
import
java.io.ObjectInputStream
;
...
...
@@ -67,15 +68,19 @@ public class FlatMapTest {
for
(
int
i
=
0
;
i
<
5
;
i
++)
{
collector
.
collect
(
new
Tuple1
<
String
>(
"hi"
));
}
}
}
@Test
public
void
test
()
throws
Exception
{
StreamExecutionEnvironment
context
=
new
StreamExecutionEnvironment
();
try
{
StreamExecutionEnvironment
context2
=
new
StreamExecutionEnvironment
(
0
);
fail
();
}
catch
(
IllegalArgumentException
e
)
{
}
StreamExecutionEnvironment
context
=
new
StreamExecutionEnvironment
(
2
);
DataStream
<
Tuple1
<
String
>>
dataStream0
=
context
.
addSource
(
new
MySource
());
DataStream
<
Tuple1
<
String
>>
dataStream1
=
context
.
addDummySource
().
connectWith
(
dataStream0
)
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录