Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
2b16c604
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,发现更多精彩内容 >>
提交
2b16c604
编写于
12月 16, 2014
作者:
F
Fabian Hueske
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-1112] Additional checks for KeySelector group sorting and minor fixes
上级
f83db149
变更
6
显示空白变更内容
内联
并排
Showing
6 changed file
with
134 addition
and
103 deletion
+134
-103
flink-compiler/src/test/java/org/apache/flink/compiler/custompartition/GroupingKeySelectorTranslationTest.java
...r/custompartition/GroupingKeySelectorTranslationTest.java
+1
-34
flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java
...a/org/apache/flink/api/java/operators/SortedGrouping.java
+1
-1
flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java
...org/apache/flink/api/java/operators/UnsortedGrouping.java
+15
-2
flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
...ain/scala/org/apache/flink/api/scala/GroupedDataSet.scala
+18
-1
flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java
...apache/flink/test/javaApiOperators/GroupReduceITCase.java
+0
-2
flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingKeySelectorTest.scala
...anslation/CustomPartitioningGroupingKeySelectorTest.scala
+99
-63
未找到文件。
flink-compiler/src/test/java/org/apache/flink/compiler/custompartition/GroupingKeySelectorTranslationTest.java
浏览文件 @
2b16c604
...
...
@@ -115,7 +115,7 @@ public class GroupingKeySelectorTranslationTest extends CompilerTestBase {
data
.
groupBy
(
new
TestKeySelector
<
Tuple3
<
Integer
,
Integer
,
Integer
>>())
.
withPartitioner
(
new
TestPartitionerInt
())
.
sortGroup
(
1
,
Order
.
ASCENDING
)
.
sortGroup
(
new
TestKeySelector
<
Tuple3
<
Integer
,
Integer
,
Integer
>>()
,
Order
.
ASCENDING
)
.
reduceGroup
(
new
IdentityGroupReducer
<
Tuple3
<
Integer
,
Integer
,
Integer
>>())
.
print
();
...
...
@@ -136,39 +136,6 @@ public class GroupingKeySelectorTranslationTest extends CompilerTestBase {
}
}
@Test
public
void
testCustomPartitioningKeySelectorGroupReduceSorted2
()
{
try
{
ExecutionEnvironment
env
=
ExecutionEnvironment
.
getExecutionEnvironment
();
DataSet
<
Tuple4
<
Integer
,
Integer
,
Integer
,
Integer
>>
data
=
env
.
fromElements
(
new
Tuple4
<
Integer
,
Integer
,
Integer
,
Integer
>(
0
,
0
,
0
,
0
))
.
rebalance
().
setParallelism
(
4
);
data
.
groupBy
(
new
TestKeySelector
<
Tuple4
<
Integer
,
Integer
,
Integer
,
Integer
>>())
.
withPartitioner
(
new
TestPartitionerInt
())
.
sortGroup
(
1
,
Order
.
ASCENDING
)
.
sortGroup
(
2
,
Order
.
DESCENDING
)
.
reduceGroup
(
new
IdentityGroupReducer
<
Tuple4
<
Integer
,
Integer
,
Integer
,
Integer
>>())
.
print
();
Plan
p
=
env
.
createProgramPlan
();
OptimizedPlan
op
=
compileNoStats
(
p
);
SinkPlanNode
sink
=
op
.
getDataSinks
().
iterator
().
next
();
SingleInputPlanNode
reducer
=
(
SingleInputPlanNode
)
sink
.
getInput
().
getSource
();
SingleInputPlanNode
combiner
=
(
SingleInputPlanNode
)
reducer
.
getInput
().
getSource
();
assertEquals
(
ShipStrategyType
.
FORWARD
,
sink
.
getInput
().
getShipStrategy
());
assertEquals
(
ShipStrategyType
.
PARTITION_CUSTOM
,
reducer
.
getInput
().
getShipStrategy
());
assertEquals
(
ShipStrategyType
.
FORWARD
,
combiner
.
getInput
().
getShipStrategy
());
}
catch
(
Exception
e
)
{
e
.
printStackTrace
();
fail
(
e
.
getMessage
());
}
}
@Test
public
void
testCustomPartitioningKeySelectorInvalidType
()
{
try
{
...
...
flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java
浏览文件 @
2b16c604
...
...
@@ -92,7 +92,7 @@ public class SortedGrouping<T> extends Grouping<T> {
super
(
set
,
keys
);
if
(!(
this
.
keys
instanceof
Keys
.
SelectorFunctionKeys
))
{
throw
new
InvalidProgramException
(
"Sorting on KeySelector
only works for
KeySelector grouping."
);
throw
new
InvalidProgramException
(
"Sorting on KeySelector
keys only works with
KeySelector grouping."
);
}
this
.
groupSortKeyPositions
=
keySelector
.
computeLogicalKeyPositions
();
...
...
flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java
浏览文件 @
2b16c604
...
...
@@ -231,6 +231,10 @@ public class UnsortedGrouping<T> extends Grouping<T> {
* @see Order
*/
public
SortedGrouping
<
T
>
sortGroup
(
int
field
,
Order
order
)
{
if
(
this
.
getKeys
()
instanceof
Keys
.
SelectorFunctionKeys
)
{
throw
new
InvalidProgramException
(
"KeySelector grouping keys and field index group-sorting keys cannot be used together."
);
}
SortedGrouping
<
T
>
sg
=
new
SortedGrouping
<
T
>(
this
.
dataSet
,
this
.
keys
,
field
,
order
);
sg
.
customPartitioner
=
getCustomPartitioner
();
return
sg
;
...
...
@@ -248,6 +252,10 @@ public class UnsortedGrouping<T> extends Grouping<T> {
* @see Order
*/
public
SortedGrouping
<
T
>
sortGroup
(
String
field
,
Order
order
)
{
if
(
this
.
getKeys
()
instanceof
Keys
.
SelectorFunctionKeys
)
{
throw
new
InvalidProgramException
(
"KeySelector grouping keys and field expression group-sorting keys cannot be used together."
);
}
SortedGrouping
<
T
>
sg
=
new
SortedGrouping
<
T
>(
this
.
dataSet
,
this
.
keys
,
field
,
order
);
sg
.
customPartitioner
=
getCustomPartitioner
();
return
sg
;
...
...
@@ -256,7 +264,6 @@ public class UnsortedGrouping<T> extends Grouping<T> {
/**
* Sorts elements within a group on a key extracted by the specified {@link org.apache.flink.api.java.functions.KeySelector}
* in the specified {@link Order}.</br>
* <b>Note: Only groups of Tuple elements and Pojos can be sorted.</b><br/>
* Chaining {@link #sortGroup(KeySelector, Order)} calls is not supported.
*
* @param keySelector The KeySelector with which the group is sorted.
...
...
@@ -266,8 +273,14 @@ public class UnsortedGrouping<T> extends Grouping<T> {
* @see Order
*/
public
<
K
>
SortedGrouping
<
T
>
sortGroup
(
KeySelector
<
T
,
K
>
keySelector
,
Order
order
)
{
if
(!(
this
.
getKeys
()
instanceof
Keys
.
SelectorFunctionKeys
))
{
throw
new
InvalidProgramException
(
"KeySelector group-sorting keys can only be used with KeySelector grouping keys."
);
}
TypeInformation
<
K
>
keyType
=
TypeExtractor
.
getKeySelectorTypes
(
keySelector
,
this
.
dataSet
.
getType
());
return
new
SortedGrouping
<
T
>(
this
.
dataSet
,
this
.
keys
,
new
Keys
.
SelectorFunctionKeys
<
T
,
K
>(
keySelector
,
this
.
dataSet
.
getType
(),
keyType
),
order
);
SortedGrouping
<
T
>
sg
=
new
SortedGrouping
<
T
>(
this
.
dataSet
,
this
.
keys
,
new
Keys
.
SelectorFunctionKeys
<
T
,
K
>(
keySelector
,
this
.
dataSet
.
getType
(),
keyType
),
order
);
sg
.
customPartitioner
=
getCustomPartitioner
();
return
sg
;
}
}
flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala
浏览文件 @
2b16c604
...
...
@@ -67,6 +67,10 @@ class GroupedDataSet[T: ClassTag](
if
(
field
>=
set
.
getType
.
getArity
)
{
throw
new
IllegalArgumentException
(
"Order key out of tuple bounds."
)
}
if
(
keys
.
isInstanceOf
[
Keys.SelectorFunctionKeys
[
_
,
_
]])
{
throw
new
InvalidProgramException
(
"KeySelector grouping keys and field index group-sorting "
+
"keys cannot be used together."
)
}
if
(
groupSortKeySelector
.
nonEmpty
)
{
throw
new
InvalidProgramException
(
"Chaining sortGroup with KeySelector sorting is not "
+
"supported."
)
...
...
@@ -87,6 +91,10 @@ class GroupedDataSet[T: ClassTag](
throw
new
InvalidProgramException
(
"Chaining sortGroup with KeySelector sorting is not"
+
"supported."
)
}
if
(
keys
.
isInstanceOf
[
Keys.SelectorFunctionKeys
[
_
,
_
]])
{
throw
new
InvalidProgramException
(
"KeySelector grouping keys and field expression "
+
"group-sorting keys cannot be used together."
)
}
groupSortKeyPositions
+=
Right
(
field
)
groupSortOrders
+=
order
this
...
...
@@ -103,6 +111,10 @@ class GroupedDataSet[T: ClassTag](
throw
new
InvalidProgramException
(
"Chaining sortGroup with KeySelector sorting is not"
+
"supported."
)
}
if
(!
keys
.
isInstanceOf
[
Keys.SelectorFunctionKeys
[
_
,
_
]])
{
throw
new
InvalidProgramException
(
"Sorting on KeySelector keys only works with KeySelector "
+
"grouping."
)
}
groupSortOrders
+=
order
val
keyType
=
implicitly
[
TypeInformation
[
K
]]
...
...
@@ -121,7 +133,12 @@ class GroupedDataSet[T: ClassTag](
private
def
maybeCreateSortedGrouping
()
:
Grouping
[
T
]
=
{
groupSortKeySelector
match
{
case
Some
(
keySelector
)
=>
if
(
partitioner
==
null
)
{
new
SortedGrouping
[
T
](
set
.
javaSet
,
keys
,
keySelector
,
groupSortOrders
(
0
))
}
else
{
new
SortedGrouping
[
T
](
set
.
javaSet
,
keys
,
keySelector
,
groupSortOrders
(
0
))
.
withPartitioner
(
partitioner
)
}
case
None
=>
if
(
groupSortKeyPositions
.
length
>
0
)
{
val
grouping
=
groupSortKeyPositions
(
0
)
match
{
...
...
flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/GroupReduceITCase.java
浏览文件 @
2b16c604
...
...
@@ -835,7 +835,6 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
.
reduceGroup
(
new
Tuple3SortedGroupReduceWithCombine
());
reduceDs
.
writeAsCsv
(
resultPath
);
reduceDs
.
print
();
env
.
execute
();
// return expected result
...
...
@@ -1302,7 +1301,6 @@ public class GroupReduceITCase extends MultipleProgramsTestBase {
public
void
combine
(
Iterable
<
Tuple3
<
Integer
,
Long
,
String
>>
values
,
Collector
<
Tuple3
<
Integer
,
Long
,
String
>>
out
)
{
int
sum
=
0
;
long
key
=
0
;
System
.
out
.
println
(
"im in"
);
StringBuilder
concat
=
new
StringBuilder
();
for
(
Tuple3
<
Integer
,
Long
,
String
>
next
:
values
)
{
...
...
flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/CustomPartitioningGroupingKeySelectorTest.scala
浏览文件 @
2b16c604
...
...
@@ -25,7 +25,6 @@ import org.apache.flink.api.common.functions.Partitioner
import
org.apache.flink.runtime.operators.shipping.ShipStrategyType
import
org.apache.flink.compiler.plan.SingleInputPlanNode
import
org.apache.flink.test.compiler.util.CompilerTestBase
import
scala.collection.immutable.Seq
import
org.apache.flink.api.common.operators.Order
import
org.apache.flink.api.common.InvalidProgramException
...
...
@@ -73,7 +72,7 @@ class CustomPartitioningGroupingKeySelectorTest extends CompilerTestBase {
data
.
groupBy
(
_
.
_1
).
withPartitioner
(
new
TestPartitionerInt
())
.
reduceGroup
(
iter
=>
Seq
(
iter
.
next
())
)
.
reduce
(
(
a
,
b
)
=>
a
)
.
print
()
val
p
=
env
.
createProgramPlan
()
...
...
@@ -81,6 +80,7 @@ class CustomPartitioningGroupingKeySelectorTest extends CompilerTestBase {
val
sink
=
op
.
getDataSinks
.
iterator
().
next
()
val
reducer
=
sink
.
getInput
.
getSource
.
asInstanceOf
[
SingleInputPlanNode
]
.
getInput
.
getSource
.
asInstanceOf
[
SingleInputPlanNode
]
val
combiner
=
reducer
.
getInput
.
getSource
.
asInstanceOf
[
SingleInputPlanNode
]
assertEquals
(
ShipStrategyType
.
FORWARD
,
sink
.
getInput
.
getShipStrategy
)
...
...
@@ -96,17 +96,17 @@ class CustomPartitioningGroupingKeySelectorTest extends CompilerTestBase {
}
@Test
def
testCustomPartitioning
KeySelector
GroupReduceSorted
()
{
def
testCustomPartitioning
Index
GroupReduceSorted
()
{
try
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
data
=
env
.
fromElements
(
(
0
,
0
,
0
)
).
rebalance
().
setParallelism
(
4
)
data
.
groupBy
(
_
.
_1
)
.
groupBy
(
0
)
.
withPartitioner
(
new
TestPartitionerInt
())
.
sortGroup
(
1
,
Order
.
ASCENDING
)
.
reduceGroup
(
iter
=>
Seq
(
iter
.
next
())
)
.
reduce
(
(
a
,
b
)
=>
a
)
.
print
()
val
p
=
env
.
createProgramPlan
()
...
...
@@ -116,6 +116,41 @@ class CustomPartitioningGroupingKeySelectorTest extends CompilerTestBase {
val
reducer
=
sink
.
getInput
.
getSource
.
asInstanceOf
[
SingleInputPlanNode
]
val
combiner
=
reducer
.
getInput
.
getSource
.
asInstanceOf
[
SingleInputPlanNode
]
assertEquals
(
ShipStrategyType
.
FORWARD
,
sink
.
getInput
.
getShipStrategy
)
assertEquals
(
ShipStrategyType
.
PARTITION_CUSTOM
,
reducer
.
getInput
.
getShipStrategy
)
assertEquals
(
ShipStrategyType
.
FORWARD
,
combiner
.
getInput
.
getShipStrategy
)
}
catch
{
case
e
:
Exception
=>
{
e
.
printStackTrace
()
fail
(
e
.
getMessage
)
}
}
}
@Test
def
testCustomPartitioningKeySelectorGroupReduceSorted
()
{
try
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
data
=
env
.
fromElements
(
(
0
,
0
,
0
)
).
rebalance
().
setParallelism
(
4
)
data
.
groupBy
(
_
.
_1
)
.
withPartitioner
(
new
TestPartitionerInt
())
.
sortGroup
(
_
.
_2
,
Order
.
ASCENDING
)
.
reduce
(
(
a
,
b
)
=>
a
)
.
print
()
val
p
=
env
.
createProgramPlan
()
val
op
=
compileNoStats
(
p
)
val
sink
=
op
.
getDataSinks
.
iterator
().
next
()
val
reducer
=
sink
.
getInput
.
getSource
.
asInstanceOf
[
SingleInputPlanNode
]
.
getInput
.
getSource
.
asInstanceOf
[
SingleInputPlanNode
]
val
combiner
=
reducer
.
getInput
.
getSource
.
asInstanceOf
[
SingleInputPlanNode
]
assertEquals
(
ShipStrategyType
.
FORWARD
,
sink
.
getInput
.
getShipStrategy
)
assertEquals
(
ShipStrategyType
.
PARTITION_CUSTOM
,
reducer
.
getInput
.
getShipStrategy
)
assertEquals
(
ShipStrategyType
.
FORWARD
,
combiner
.
getInput
.
getShipStrategy
)
...
...
@@ -136,10 +171,10 @@ class CustomPartitioningGroupingKeySelectorTest extends CompilerTestBase {
val
data
=
env
.
fromElements
(
(
0
,
0
,
0
,
0
)
).
rebalance
().
setParallelism
(
4
)
data
.
groupBy
(
_
.
_1
).
withPartitioner
(
new
TestPartitionerInt
())
.
groupBy
(
0
).
withPartitioner
(
new
TestPartitionerInt
())
.
sortGroup
(
1
,
Order
.
ASCENDING
)
.
sortGroup
(
2
,
Order
.
DESCENDING
)
.
reduceGroup
(
iter
=>
Seq
(
iter
.
next
())
)
.
reduce
(
(
a
,
b
)
=>
a
)
.
print
()
val
p
=
env
.
createProgramPlan
()
...
...
@@ -152,6 +187,7 @@ class CustomPartitioningGroupingKeySelectorTest extends CompilerTestBase {
assertEquals
(
ShipStrategyType
.
FORWARD
,
sink
.
getInput
.
getShipStrategy
)
assertEquals
(
ShipStrategyType
.
PARTITION_CUSTOM
,
reducer
.
getInput
.
getShipStrategy
)
assertEquals
(
ShipStrategyType
.
FORWARD
,
combiner
.
getInput
.
getShipStrategy
)
}
catch
{
case
e
:
Exception
=>
{
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录