Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
9bec3359
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,体验更适合开发者的 AI 搜索 >>
未验证
提交
9bec3359
编写于
1月 04, 2021
作者:
A
Aljoscha Krettek
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-20517] Support mixed keyed/non-keyed operations in BATCH mode
上级
606c44bf
变更
4
隐藏空白更改
内联
并排
Showing
4 changed file
with
181 addition
and
17 deletion
+181
-17
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java
...ache/flink/streaming/api/datastream/ConnectedStreams.java
+16
-6
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/MultiInputTransformationTranslator.java
...ntime/translators/MultiInputTransformationTranslator.java
+17
-4
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/TwoInputTransformationTranslator.java
...runtime/translators/TwoInputTransformationTranslator.java
+13
-7
flink-tests/src/test/java/org/apache/flink/api/datastream/DataStreamBatchExecutionITCase.java
.../flink/api/datastream/DataStreamBatchExecutionITCase.java
+135
-0
未找到文件。
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java
浏览文件 @
9bec3359
...
...
@@ -436,25 +436,35 @@ public class ConnectedStreams<IN1, IN2> {
outTypeInfo
,
environment
.
getParallelism
());
if
(
inputStream1
instanceof
KeyedStream
&&
inputStream2
instanceof
KeyedStream
)
{
TypeInformation
<?>
keyType
=
null
;
if
(
inputStream1
instanceof
KeyedStream
)
{
KeyedStream
<
IN1
,
?>
keyedInput1
=
(
KeyedStream
<
IN1
,
?>)
inputStream1
;
keyType
=
keyedInput1
.
getKeyType
();
transform
.
setStateKeySelectors
(
keyedInput1
.
getKeySelector
(),
null
);
transform
.
setStateKeyType
(
keyType
);
}
if
(
inputStream2
instanceof
KeyedStream
)
{
KeyedStream
<
IN2
,
?>
keyedInput2
=
(
KeyedStream
<
IN2
,
?>)
inputStream2
;
TypeInformation
<?>
keyType1
=
keyedInput1
.
getKeyType
();
TypeInformation
<?>
keyType2
=
keyedInput2
.
getKeyType
();
if
(!(
keyType1
.
canEqual
(
keyType2
)
&&
keyType1
.
equals
(
keyType2
)))
{
if
(
keyType
!=
null
&&
!(
keyType
.
canEqual
(
keyType2
)
&&
keyType
.
equals
(
keyType2
)))
{
throw
new
UnsupportedOperationException
(
"Key types if input KeyedStreams "
+
"don't match: "
+
keyType
1
+
keyType
+
" and "
+
keyType2
+
"."
);
}
transform
.
setStateKeySelectors
(
keyedInput1
.
getKeySelector
(),
keyedInput2
.
getKeySelector
());
transform
.
setStateKeyType
(
keyType1
);
transform
.
getStateKeySelector1
(),
keyedInput2
.
getKeySelector
());
// we might be overwriting the one that's already set, but it's the same
transform
.
setStateKeyType
(
keyType2
);
}
@SuppressWarnings
({
"unchecked"
,
"rawtypes"
})
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/MultiInputTransformationTranslator.java
浏览文件 @
9bec3359
...
...
@@ -22,6 +22,7 @@ import org.apache.flink.annotation.Internal;
import
org.apache.flink.api.common.ExecutionConfig
;
import
org.apache.flink.api.common.typeutils.TypeSerializer
;
import
org.apache.flink.api.dag.Transformation
;
import
org.apache.flink.api.java.functions.KeySelector
;
import
org.apache.flink.streaming.api.graph.SimpleTransformationTranslator
;
import
org.apache.flink.streaming.api.graph.StreamConfig
;
import
org.apache.flink.streaming.api.graph.StreamGraph
;
...
...
@@ -34,6 +35,7 @@ import org.apache.flink.streaming.runtime.io.MultipleInputSelectionHandler;
import
java.util.Collection
;
import
java.util.Collections
;
import
java.util.List
;
import
java.util.stream.IntStream
;
import
static
org
.
apache
.
flink
.
util
.
Preconditions
.
checkArgument
;
import
static
org
.
apache
.
flink
.
util
.
Preconditions
.
checkNotNull
;
...
...
@@ -52,13 +54,24 @@ public class MultiInputTransformationTranslator<OUT>
protected
Collection
<
Integer
>
translateForBatchInternal
(
final
AbstractMultipleInputTransformation
<
OUT
>
transformation
,
final
Context
context
)
{
Collection
<
Integer
>
ids
=
translateInternal
(
transformation
,
context
);
boolean
isKeyed
=
transformation
instanceof
KeyedMultipleInputTransformation
;
if
(
isKeyed
)
{
if
(
transformation
instanceof
KeyedMultipleInputTransformation
)
{
KeyedMultipleInputTransformation
<
OUT
>
keyedTransformation
=
(
KeyedMultipleInputTransformation
<
OUT
>)
transformation
;
List
<
Transformation
<?>>
inputs
=
transformation
.
getInputs
();
List
<
KeySelector
<?,
?>>
keySelectors
=
keyedTransformation
.
getStateKeySelectors
();
StreamConfig
.
InputRequirement
[]
inputRequirements
=
inputs
.
stream
()
.
map
((
input
)
->
StreamConfig
.
InputRequirement
.
SORTED
)
IntStream
.
range
(
0
,
inputs
.
size
())
.
mapToObj
(
idx
->
{
if
(
keySelectors
.
get
(
idx
)
!=
null
)
{
return
StreamConfig
.
InputRequirement
.
SORTED
;
}
else
{
return
StreamConfig
.
InputRequirement
.
PASS_THROUGH
;
}
})
.
toArray
(
StreamConfig
.
InputRequirement
[]::
new
);
BatchExecutionUtils
.
applyBatchExecutionSettings
(
transformation
.
getId
(),
context
,
inputRequirements
);
}
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/TwoInputTransformationTranslator.java
浏览文件 @
9bec3359
...
...
@@ -43,15 +43,21 @@ public class TwoInputTransformationTranslator<IN1, IN2, OUT>
protected
Collection
<
Integer
>
translateForBatchInternal
(
final
TwoInputTransformation
<
IN1
,
IN2
,
OUT
>
transformation
,
final
Context
context
)
{
Collection
<
Integer
>
ids
=
translateInternal
(
transformation
,
context
);
boolean
isKeyed
=
StreamConfig
.
InputRequirement
input1Requirement
=
transformation
.
getStateKeySelector1
()
!=
null
&&
transformation
.
getStateKeySelector2
()
!=
null
;
if
(
isKeyed
)
{
?
StreamConfig
.
InputRequirement
.
SORTED
:
StreamConfig
.
InputRequirement
.
PASS_THROUGH
;
StreamConfig
.
InputRequirement
input2Requirement
=
transformation
.
getStateKeySelector2
()
!=
null
?
StreamConfig
.
InputRequirement
.
SORTED
:
StreamConfig
.
InputRequirement
.
PASS_THROUGH
;
if
(
input1Requirement
==
StreamConfig
.
InputRequirement
.
SORTED
||
input2Requirement
==
StreamConfig
.
InputRequirement
.
SORTED
)
{
BatchExecutionUtils
.
applyBatchExecutionSettings
(
transformation
.
getId
(),
context
,
StreamConfig
.
InputRequirement
.
SORTED
,
StreamConfig
.
InputRequirement
.
SORTED
);
transformation
.
getId
(),
context
,
input1Requirement
,
input2Requirement
);
}
return
ids
;
}
...
...
flink-tests/src/test/java/org/apache/flink/api/datastream/DataStreamBatchExecutionITCase.java
浏览文件 @
9bec3359
...
...
@@ -27,6 +27,7 @@ import org.apache.flink.api.common.state.MapStateDescriptor;
import
org.apache.flink.api.common.state.ReadOnlyBroadcastState
;
import
org.apache.flink.api.common.state.ValueStateDescriptor
;
import
org.apache.flink.api.common.time.Time
;
import
org.apache.flink.api.common.typeinfo.BasicTypeInfo
;
import
org.apache.flink.api.common.typeutils.base.StringSerializer
;
import
org.apache.flink.api.java.tuple.Tuple2
;
import
org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration
;
...
...
@@ -38,6 +39,9 @@ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
import
org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
;
import
org.apache.flink.streaming.api.functions.co.BroadcastProcessFunction
;
import
org.apache.flink.streaming.api.functions.co.KeyedBroadcastProcessFunction
;
import
org.apache.flink.streaming.api.operators.AbstractStreamOperator
;
import
org.apache.flink.streaming.api.operators.TwoInputStreamOperator
;
import
org.apache.flink.streaming.runtime.streamrecord.StreamRecord
;
import
org.apache.flink.test.util.MiniClusterWithClientResource
;
import
org.apache.flink.util.CloseableIterator
;
import
org.apache.flink.util.CollectionUtil
;
...
...
@@ -191,6 +195,119 @@ public class DataStreamBatchExecutionITCase {
}
}
/**
* Verifies that all regular input is processed before keyed input.
*
* <p>Here, the first input is keyed while the second input is not keyed.
*/
@Test
public
void
batchKeyedNonKeyedTwoInputOperator
()
throws
Exception
{
StreamExecutionEnvironment
env
=
StreamExecutionEnvironment
.
getExecutionEnvironment
();
env
.
setParallelism
(
1
);
env
.
setRuntimeMode
(
RuntimeExecutionMode
.
BATCH
);
DataStream
<
Tuple2
<
String
,
Integer
>>
keyedInput
=
env
.
fromElements
(
Tuple2
.
of
(
"regular2"
,
4
),
Tuple2
.
of
(
"regular1"
,
3
),
Tuple2
.
of
(
"regular1"
,
2
),
Tuple2
.
of
(
"regular2"
,
1
))
.
assignTimestampsAndWatermarks
(
WatermarkStrategy
.<
Tuple2
<
String
,
Integer
>>
forMonotonousTimestamps
()
.
withTimestampAssigner
((
in
,
ts
)
->
in
.
f1
));
DataStream
<
Tuple2
<
String
,
Integer
>>
regularInput
=
env
.
fromElements
(
Tuple2
.
of
(
"regular4"
,
4
),
Tuple2
.
of
(
"regular3"
,
3
),
Tuple2
.
of
(
"regular3"
,
2
),
Tuple2
.
of
(
"regular4"
,
1
))
.
assignTimestampsAndWatermarks
(
WatermarkStrategy
.<
Tuple2
<
String
,
Integer
>>
forMonotonousTimestamps
()
.
withTimestampAssigner
((
in
,
ts
)
->
in
.
f1
));
DataStream
<
String
>
result
=
keyedInput
.
keyBy
(
in
->
in
.
f0
)
.
connect
(
regularInput
)
.
transform
(
"operator"
,
BasicTypeInfo
.
STRING_TYPE_INFO
,
new
TwoInputIdentityOperator
());
try
(
CloseableIterator
<
String
>
resultIterator
=
result
.
executeAndCollect
())
{
List
<
String
>
results
=
CollectionUtil
.
iteratorToList
(
resultIterator
);
assertThat
(
results
,
equalTo
(
Arrays
.
asList
(
"(regular4,4)"
,
"(regular3,3)"
,
"(regular3,2)"
,
"(regular4,1)"
,
"(regular1,2)"
,
"(regular1,3)"
,
"(regular2,1)"
,
"(regular2,4)"
)));
}
}
/**
* Verifies that all regular input is processed before keyed input.
*
* <p>Here, the first input is not keyed while the second input is keyed.
*/
@Test
public
void
batchNonKeyedKeyedTwoInputOperator
()
throws
Exception
{
StreamExecutionEnvironment
env
=
StreamExecutionEnvironment
.
getExecutionEnvironment
();
env
.
setParallelism
(
1
);
env
.
setRuntimeMode
(
RuntimeExecutionMode
.
BATCH
);
DataStream
<
Tuple2
<
String
,
Integer
>>
keyedInput
=
env
.
fromElements
(
Tuple2
.
of
(
"regular2"
,
4
),
Tuple2
.
of
(
"regular1"
,
3
),
Tuple2
.
of
(
"regular1"
,
2
),
Tuple2
.
of
(
"regular2"
,
1
))
.
assignTimestampsAndWatermarks
(
WatermarkStrategy
.<
Tuple2
<
String
,
Integer
>>
forMonotonousTimestamps
()
.
withTimestampAssigner
((
in
,
ts
)
->
in
.
f1
));
DataStream
<
Tuple2
<
String
,
Integer
>>
regularInput
=
env
.
fromElements
(
Tuple2
.
of
(
"regular4"
,
4
),
Tuple2
.
of
(
"regular3"
,
3
),
Tuple2
.
of
(
"regular3"
,
2
),
Tuple2
.
of
(
"regular4"
,
1
))
.
assignTimestampsAndWatermarks
(
WatermarkStrategy
.<
Tuple2
<
String
,
Integer
>>
forMonotonousTimestamps
()
.
withTimestampAssigner
((
in
,
ts
)
->
in
.
f1
));
DataStream
<
String
>
result
=
regularInput
.
connect
(
keyedInput
.
keyBy
(
in
->
in
.
f0
))
.
transform
(
"operator"
,
BasicTypeInfo
.
STRING_TYPE_INFO
,
new
TwoInputIdentityOperator
());
try
(
CloseableIterator
<
String
>
resultIterator
=
result
.
executeAndCollect
())
{
List
<
String
>
results
=
CollectionUtil
.
iteratorToList
(
resultIterator
);
assertThat
(
results
,
equalTo
(
Arrays
.
asList
(
"(regular4,4)"
,
"(regular3,3)"
,
"(regular3,2)"
,
"(regular4,1)"
,
"(regular1,2)"
,
"(regular1,3)"
,
"(regular2,1)"
,
"(regular2,4)"
)));
}
}
/** Verifies that all broadcast input is processed before keyed input. */
@Test
public
void
batchKeyedBroadcastExecution
()
throws
Exception
{
...
...
@@ -402,4 +519,22 @@ public class DataStreamBatchExecutionITCase {
state
.
put
(
value
.
f0
,
value
.
f0
);
}
}
private
static
class
TwoInputIdentityOperator
extends
AbstractStreamOperator
<
String
>
implements
TwoInputStreamOperator
<
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>,
String
>
{
@Override
public
void
processElement1
(
StreamRecord
<
Tuple2
<
String
,
Integer
>>
element
)
throws
Exception
{
output
.
collect
(
new
StreamRecord
<>(
element
.
getValue
().
toString
(),
element
.
getTimestamp
()));
}
@Override
public
void
processElement2
(
StreamRecord
<
Tuple2
<
String
,
Integer
>>
element
)
throws
Exception
{
output
.
collect
(
new
StreamRecord
<>(
element
.
getValue
().
toString
(),
element
.
getTimestamp
()));
}
}
}
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录