Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
45e680c2
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,发现更多精彩内容 >>
提交
45e680c2
编写于
4月 06, 2015
作者:
C
Chiwan Park
提交者:
Fabian Hueske
4月 21, 2015
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-703] [scala api] Use complete element as join key
This closes #572
上级
30a74c76
变更
9
显示空白变更内容
内联
并排
Showing
9 changed file
with
236 addition
and
16 deletion
+236
-16
flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
...c/main/java/org/apache/flink/api/java/operators/Keys.java
+2
-2
flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
...a/src/main/scala/org/apache/flink/api/scala/DataSet.scala
+0
-1
flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
...g/apache/flink/api/scala/unfinishedKeyPairOperation.scala
+0
-5
flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupITCase.scala
.../org/apache/flink/api/scala/operators/CoGroupITCase.scala
+42
-0
flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupOperatorTest.scala
...pache/flink/api/scala/operators/CoGroupOperatorTest.scala
+57
-0
flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala
.../apache/flink/api/scala/operators/GroupReduceITCase.scala
+9
-0
flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupingTest.scala
...a/org/apache/flink/api/scala/operators/GroupingTest.scala
+36
-2
flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinITCase.scala
...ala/org/apache/flink/api/scala/operators/JoinITCase.scala
+22
-0
flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinOperatorTest.scala
...g/apache/flink/api/scala/operators/JoinOperatorTest.scala
+68
-6
未找到文件。
flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java
浏览文件 @
45e680c2
...
...
@@ -280,7 +280,7 @@ public abstract class Keys<T> {
if
(!
type
.
isKeyType
())
{
throw
new
InvalidProgramException
(
"This type ("
+
type
+
") cannot be used as key."
);
}
else
if
(
expressionsIn
.
length
!=
1
||
!(
Keys
.
ExpressionKeys
.
SELECT_ALL_CHAR
.
equals
(
expressionsIn
[
0
])
||
Keys
.
ExpressionKeys
.
SELECT_ALL_CHAR_SCALA
.
equals
(
expressionsIn
[
0
])))
{
throw
new
I
llegalArgument
Exception
(
"Field expression for atomic type must be equal to '*' or '_'."
);
throw
new
I
nvalidProgram
Exception
(
"Field expression for atomic type must be equal to '*' or '_'."
);
}
keyFields
=
new
ArrayList
<
FlatFieldDescriptor
>(
1
);
...
...
@@ -297,7 +297,7 @@ public abstract class Keys<T> {
for
(
int
i
=
0
;
i
<
expressions
.
length
;
i
++)
{
List
<
FlatFieldDescriptor
>
keys
=
cType
.
getFlatFields
(
expressions
[
i
]);
// use separate list to do a size check
if
(
keys
.
size
()
==
0
)
{
throw
new
I
llegalArgument
Exception
(
"Unable to extract key from expression '"
+
expressions
[
i
]+
"' on key "
+
cType
);
throw
new
I
nvalidProgram
Exception
(
"Unable to extract key from expression '"
+
expressions
[
i
]+
"' on key "
+
cType
);
}
keyFields
.
addAll
(
keys
);
}
...
...
flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
浏览文件 @
45e680c2
...
...
@@ -807,7 +807,6 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) {
* This will not create a new DataSet, it will just attach the field names which will be
* used for grouping when executing a grouped operation.
*
* This only works on CaseClass DataSets.
*/
def
groupBy
(
firstField
:
String
,
otherFields
:
String*
)
:
GroupedDataSet
[
T
]
=
{
new
GroupedDataSet
[
T
](
...
...
flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala
浏览文件 @
45e680c2
...
...
@@ -65,8 +65,6 @@ private[flink] abstract class UnfinishedKeyPairOperation[L, R, O](
* a [[HalfUnfinishedKeyPairOperation]] on which `equalTo` must be called to specify the
* key for the right side. The result after specifying the right side key is the finished
* operation.
*
* This only works on a CaseClass [[DataSet]].
*/
def
where
(
firstLeftField
:
String
,
otherLeftFields
:
String*
)
=
{
val
leftKey
=
new
ExpressionKeys
[
L
](
...
...
@@ -113,8 +111,6 @@ private[flink] class HalfUnfinishedKeyPairOperation[L, R, O](
/**
* Specify the key fields for the right side of the key based operation. This returns
* the finished operation.
*
* This only works on a CaseClass [[DataSet]].
*/
def
equalTo
(
firstRightField
:
String
,
otherRightFields
:
String*
)
:
O
=
{
val
rightKey
=
new
ExpressionKeys
[
R
](
...
...
@@ -125,7 +121,6 @@ private[flink] class HalfUnfinishedKeyPairOperation[L, R, O](
leftKey
+
" Right: "
+
rightKey
)
}
unfinished
.
finish
(
leftKey
,
rightKey
)
}
/**
...
...
flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupITCase.scala
浏览文件 @
45e680c2
...
...
@@ -383,5 +383,47 @@ class CoGroupITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mo
env
.
execute
()
expectedResult
=
"-1,20000,Flink\n"
+
"-1,10000,Flink\n"
+
"-1,30000,Flink\n"
}
@Test
def
testCoGroupWithAtomic1
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
CollectionDataSets
.
getSmall3TupleDataSet
(
env
)
val
ds2
=
env
.
fromElements
(
0
,
1
,
2
)
val
coGroupDs
=
ds1
.
coGroup
(
ds2
).
where
(
0
).
equalTo
(
"*"
)
{
(
first
,
second
,
out
:
Collector
[(
Int
,
Long
,
String
)])
=>
for
(
p
<-
first
)
{
for
(
t
<-
second
)
{
if
(
p
.
_1
==
t
)
{
out
.
collect
(
p
)
}
}
}
}
coGroupDs
.
writeAsText
(
resultPath
,
writeMode
=
WriteMode
.
OVERWRITE
)
env
.
execute
()
expectedResult
=
"(1,1,Hi)\n(2,2,Hello)"
}
@Test
def
testCoGroupWithAtomic2
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
env
.
fromElements
(
0
,
1
,
2
)
val
ds2
=
CollectionDataSets
.
getSmall3TupleDataSet
(
env
)
val
coGroupDs
=
ds1
.
coGroup
(
ds2
).
where
(
"*"
).
equalTo
(
0
)
{
(
first
,
second
,
out
:
Collector
[(
Int
,
Long
,
String
)])
=>
for
(
p
<-
first
)
{
for
(
t
<-
second
)
{
if
(
p
==
t
.
_1
)
{
out
.
collect
(
t
)
}
}
}
}
coGroupDs
.
writeAsText
(
resultPath
,
writeMode
=
WriteMode
.
OVERWRITE
)
env
.
execute
()
expectedResult
=
"(1,1,Hi)\n(2,2,Hello)"
}
}
flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupOperatorTest.scala
浏览文件 @
45e680c2
...
...
@@ -17,6 +17,9 @@
*/
package
org.apache.flink.api.scala.operators
import
java.util
import
org.apache.flink.api.common.InvalidProgramException
import
org.apache.flink.api.java.operators.Keys.IncompatibleKeysException
import
org.junit.Assert
import
org.junit.Test
...
...
@@ -268,6 +271,60 @@ class CoGroupOperatorTest {
// Should not work, more than one field position key
ds1
.
coGroup
(
ds2
).
where
(
1
,
3
).
equalTo
{
_
.
myLong
}
}
@Test
def
testCoGroupWithAtomic1
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
env
.
fromCollection
(
emptyTupleData
)
val
ds2
=
env
.
fromElements
(
0
,
1
,
2
)
ds1
.
coGroup
(
ds2
).
where
(
0
).
equalTo
(
"*"
)
}
@Test
def
testCoGroupWithAtomic2
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
env
.
fromElements
(
0
,
1
,
2
)
val
ds2
=
env
.
fromCollection
(
emptyTupleData
)
ds1
.
coGroup
(
ds2
).
where
(
"*"
).
equalTo
(
0
)
}
@Test
(
expected
=
classOf
[
InvalidProgramException
])
def
testCoGroupWithInvalidAtomic1
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
env
.
fromElements
(
0
,
1
,
2
)
val
ds2
=
env
.
fromCollection
(
emptyTupleData
)
ds1
.
coGroup
(
ds2
).
where
(
"invalidKey"
)
}
@Test
(
expected
=
classOf
[
InvalidProgramException
])
def
testCoGroupWithInvalidAtomic2
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
env
.
fromCollection
(
emptyTupleData
)
val
ds2
=
env
.
fromElements
(
0
,
1
,
2
)
ds1
.
coGroup
(
ds2
).
where
(
0
).
equalTo
(
"invalidKey"
)
}
@Test
(
expected
=
classOf
[
InvalidProgramException
])
def
testCoGroupWithInvalidAtomic3
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
env
.
fromElements
(
new
util
.
ArrayList
[
Integer
]())
val
ds2
=
env
.
fromElements
(
0
,
0
,
0
)
ds1
.
coGroup
(
ds2
).
where
(
"*"
)
}
@Test
(
expected
=
classOf
[
InvalidProgramException
])
def
testCoGroupWithInvalidAtomic4
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
env
.
fromElements
(
0
,
0
,
0
)
val
ds2
=
env
.
fromElements
(
new
util
.
ArrayList
[
Integer
]())
ds1
.
coGroup
(
ds2
).
where
(
"*"
).
equalTo
(
"*"
)
}
}
flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala
浏览文件 @
45e680c2
...
...
@@ -743,6 +743,15 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas
expected
=
"b\nccc\nee\n"
}
@Test
def
testWithAtomic1
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds
=
env
.
fromElements
(
0
,
1
,
1
,
2
)
val
reduceDs
=
ds
.
groupBy
(
"*"
).
reduceGroup
((
ints
:
Iterator
[
Int
])
=>
ints
.
next
())
reduceDs
.
writeAsText
(
resultPath
,
WriteMode
.
OVERWRITE
)
env
.
execute
()
expected
=
"0\n1\n2"
}
}
@RichGroupReduceFunction
.
Combinable
...
...
flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupingTest.scala
浏览文件 @
45e680c2
...
...
@@ -17,6 +17,8 @@
*/
package
org.apache.flink.api.scala.operators
import
java.util
import
org.apache.flink.api.scala.util.CollectionDataSets.CustomType
import
org.junit.Assert
import
org.apache.flink.api.common.InvalidProgramException
...
...
@@ -96,7 +98,7 @@ class GroupingTest {
}
}
@Test
(
expected
=
classOf
[
I
llegalArgument
Exception
])
@Test
(
expected
=
classOf
[
I
nvalidProgram
Exception
])
def
testGroupByKeyFields2
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
longDs
=
env
.
fromCollection
(
emptyLongData
)
...
...
@@ -146,7 +148,7 @@ class GroupingTest {
}
}
@Test
(
expected
=
classOf
[
I
llegalArgument
Exception
])
@Test
(
expected
=
classOf
[
I
nvalidProgram
Exception
])
def
testGroupByKeyExpressions2
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
...
...
@@ -224,5 +226,37 @@ class GroupingTest {
case
e
:
Exception
=>
Assert
.
fail
()
}
}
@Test
def
testAtomicValue1
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds
=
env
.
fromElements
(
0
,
1
,
2
)
ds
.
groupBy
(
"*"
)
}
@Test
(
expected
=
classOf
[
InvalidProgramException
])
def
testAtomicValueInvalid1
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds
=
env
.
fromElements
(
0
,
1
,
2
)
ds
.
groupBy
(
"invalidKey"
)
}
@Test
(
expected
=
classOf
[
InvalidProgramException
])
def
testAtomicValueInvalid2
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds
=
env
.
fromElements
(
0
,
1
,
2
)
ds
.
groupBy
(
"_"
,
"invalidKey"
)
}
@Test
(
expected
=
classOf
[
InvalidProgramException
])
def
testAtomicValueInvalid3
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds
=
env
.
fromElements
(
new
util
.
ArrayList
[
Integer
]())
ds
.
groupBy
(
"*"
)
}
}
flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinITCase.scala
浏览文件 @
45e680c2
...
...
@@ -384,4 +384,26 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode)
"2 Second (20,200,2000,Two) 20000,(20000,20,200,2000,Two,2,Second)\n"
+
"3 Third (30,300,3000,Three) 30000,(30000,30,300,3000,Three,3,Third)\n"
}
@Test
def
testWithAtomic1
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
CollectionDataSets
.
getSmall3TupleDataSet
(
env
)
val
ds2
=
env
.
fromElements
(
0
,
1
,
2
)
val
joinDs
=
ds1
.
join
(
ds2
).
where
(
0
).
equalTo
(
"*"
)
joinDs
.
writeAsCsv
(
resultPath
,
writeMode
=
WriteMode
.
OVERWRITE
)
env
.
execute
()
expected
=
"(1,1,Hi),1\n(2,2,Hello),2"
}
@Test
def
testWithAtomic2
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
env
.
fromElements
(
0
,
1
,
2
)
val
ds2
=
CollectionDataSets
.
getSmall3TupleDataSet
(
env
)
val
joinDs
=
ds1
.
join
(
ds2
).
where
(
"*"
).
equalTo
(
0
)
joinDs
.
writeAsCsv
(
resultPath
,
writeMode
=
WriteMode
.
OVERWRITE
)
env
.
execute
()
expected
=
"1,(1,1,Hi)\n2,(2,2,Hello)"
}
}
flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinOperatorTest.scala
浏览文件 @
45e680c2
...
...
@@ -17,14 +17,13 @@
*/
package
org.apache.flink.api.scala.operators
import
org.apache.flink.api.java.operators.Keys.IncompatibleKeysException
import
org.apache.flink.api.scala.util.CollectionDataSets.CustomType
import
org.junit.Assert
import
org.apache.flink.api.common.InvalidProgramException
import
org.junit.Ignore
import
org.junit.Test
import
java.util
import
org.apache.flink.api.common.InvalidProgramException
import
org.apache.flink.api.java.operators.Keys.IncompatibleKeysException
import
org.apache.flink.api.scala._
import
org.apache.flink.api.scala.util.CollectionDataSets.CustomType
import
org.junit.
{
Assert
,
Test
}
class
JoinOperatorTest
{
...
...
@@ -272,5 +271,68 @@ class JoinOperatorTest {
// should not work, more than one field position key
ds1
.
join
(
ds2
).
where
(
1
,
3
)
equalTo
{
_
.
myLong
}
}
@Test
def
testJoinWithAtomic
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
env
.
fromCollection
(
emptyTupleData
)
val
ds2
=
env
.
fromCollection
(
emptyLongData
)
ds1
.
join
(
ds2
).
where
(
1
).
equalTo
(
"*"
)
}
@Test
(
expected
=
classOf
[
InvalidProgramException
])
def
testJoinWithInvalidAtomic1
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
env
.
fromCollection
(
emptyTupleData
)
val
ds2
=
env
.
fromCollection
(
emptyLongData
)
ds1
.
join
(
ds2
).
where
(
1
).
equalTo
(
"invalidKey"
)
}
@Test
(
expected
=
classOf
[
InvalidProgramException
])
def
testJoinWithInvalidAtomic2
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
env
.
fromCollection
(
emptyLongData
)
val
ds2
=
env
.
fromCollection
(
emptyTupleData
)
ds1
.
join
(
ds2
).
where
(
"invalidKey"
).
equalTo
(
1
)
}
@Test
(
expected
=
classOf
[
InvalidProgramException
])
def
testJoinWithInvalidAtomic3
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
env
.
fromCollection
(
emptyTupleData
)
val
ds2
=
env
.
fromCollection
(
emptyLongData
)
ds1
.
join
(
ds2
).
where
(
1
).
equalTo
(
"_"
,
"invalidKey"
)
}
@Test
(
expected
=
classOf
[
InvalidProgramException
])
def
testJoinWithInvalidAtomic4
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
env
.
fromCollection
(
emptyLongData
)
val
ds2
=
env
.
fromCollection
(
emptyTupleData
)
ds1
.
join
(
ds2
).
where
(
"_"
,
"invalidKey"
).
equalTo
(
1
)
}
@Test
(
expected
=
classOf
[
InvalidProgramException
])
def
testJoinWithInvalidAtomic5
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
env
.
fromElements
(
new
util
.
ArrayList
[
Integer
]())
val
ds2
=
env
.
fromCollection
(
emptyLongData
)
ds1
.
join
(
ds2
).
where
(
"*"
)
}
@Test
(
expected
=
classOf
[
InvalidProgramException
])
def
testJoinWithInvalidAtomic6
()
:
Unit
=
{
val
env
=
ExecutionEnvironment
.
getExecutionEnvironment
val
ds1
=
env
.
fromCollection
(
emptyLongData
)
val
ds2
=
env
.
fromElements
(
new
util
.
ArrayList
[
Integer
]())
ds1
.
join
(
ds2
).
where
(
"*"
).
equalTo
(
"*"
)
}
}
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录