Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
e7c4c858
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,发现更多精彩内容 >>
提交
e7c4c858
编写于
7月 08, 2014
作者:
S
Stephan Ewen
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-1005] Make GroupReduce configurable to use either mutable or immutable object mode
上级
1d00cff8
变更
8
隐藏空白更改
内联
并排
Showing
8 changed file
with
199 addition
and
44 deletion
+199
-44
flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java
...org/apache/flink/runtime/operators/GroupReduceDriver.java
+22
-7
flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java
...a/org/apache/flink/runtime/operators/util/TaskConfig.java
+10
-0
flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java
...ava/org/apache/flink/runtime/util/KeyGroupedIterator.java
+0
-1
flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIteratorImmutable.java
...pache/flink/runtime/util/KeyGroupedIteratorImmutable.java
+35
-15
flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/DriverTestData.java
...pache/flink/runtime/operators/drivers/DriverTestData.java
+1
-1
flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java
...link/runtime/operators/drivers/GroupReduceDriverTest.java
+106
-3
flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
...ache/flink/runtime/operators/drivers/TestTaskContext.java
+4
-0
flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorImmutableTest.java
...e/flink/runtime/util/KeyGroupedIteratorImmutableTest.java
+21
-17
未找到文件。
flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceDriver.java
浏览文件 @
e7c4c858
...
...
@@ -26,6 +26,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
import
org.apache.flink.api.common.typeutils.TypeSerializer
;
import
org.apache.flink.runtime.operators.util.TaskConfig
;
import
org.apache.flink.runtime.util.KeyGroupedIterator
;
import
org.apache.flink.runtime.util.KeyGroupedIteratorImmutable
;
import
org.apache.flink.util.Collector
;
import
org.apache.flink.util.MutableObjectIterator
;
...
...
@@ -91,6 +92,12 @@ public class GroupReduceDriver<IT, OT> implements PactDriver<GroupReduceFunction
this
.
serializer
=
this
.
taskContext
.<
IT
>
getInputSerializer
(
0
).
getSerializer
();
this
.
comparator
=
this
.
taskContext
.
getDriverComparator
(
0
);
this
.
input
=
this
.
taskContext
.
getInput
(
0
);
this
.
mutableObjectMode
=
config
.
getMutableObjectMode
();
if
(
LOG
.
isDebugEnabled
())
{
LOG
.
debug
(
"GroupReduceDriver uses "
+
(
this
.
mutableObjectMode
?
"MUTABLE"
:
"IMMUTABLE"
)
+
" object mode."
);
}
}
@Override
...
...
@@ -99,15 +106,23 @@ public class GroupReduceDriver<IT, OT> implements PactDriver<GroupReduceFunction
LOG
.
debug
(
this
.
taskContext
.
formatLogString
(
"GroupReducer preprocessing done. Running GroupReducer code."
));
}
final
KeyGroupedIterator
<
IT
>
iter
=
new
KeyGroupedIterator
<
IT
>(
this
.
input
,
this
.
serializer
,
this
.
comparator
);
// cache references on the stack
final
GroupReduceFunction
<
IT
,
OT
>
stub
=
this
.
taskContext
.
getStub
();
final
Collector
<
OT
>
output
=
this
.
taskContext
.
getOutputCollector
();
// run stub implementation
while
(
this
.
running
&&
iter
.
nextKey
())
{
stub
.
reduce
(
iter
.
getValues
(),
output
);
if
(
mutableObjectMode
)
{
final
KeyGroupedIterator
<
IT
>
iter
=
new
KeyGroupedIterator
<
IT
>(
this
.
input
,
this
.
serializer
,
this
.
comparator
);
// run stub implementation
while
(
this
.
running
&&
iter
.
nextKey
())
{
stub
.
reduce
(
iter
.
getValues
(),
output
);
}
}
else
{
final
KeyGroupedIteratorImmutable
<
IT
>
iter
=
new
KeyGroupedIteratorImmutable
<
IT
>(
this
.
input
,
this
.
serializer
,
this
.
comparator
);
// run stub implementation
while
(
this
.
running
&&
iter
.
nextKey
())
{
stub
.
reduce
(
iter
.
getValues
(),
output
);
}
}
}
...
...
@@ -118,4 +133,4 @@ public class GroupReduceDriver<IT, OT> implements PactDriver<GroupReduceFunction
public
void
cancel
()
{
this
.
running
=
false
;
}
}
}
\ No newline at end of file
flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/TaskConfig.java
浏览文件 @
e7c4c858
...
...
@@ -76,6 +76,8 @@ public class TaskConfig {
private
static
final
String
DRIVER_COMPARATOR_PARAMETERS_PREFIX
=
"driver.comp.params."
;
private
static
final
String
DRIVER_PAIR_COMPARATOR_FACTORY
=
"driver.paircomp"
;
private
static
final
String
DRIVER_MUTABLE_OBJECT_MODE
=
"diver.mutableobjects"
;
// -------------------------------------- Inputs ----------------------------------------------
...
...
@@ -335,6 +337,14 @@ public class TaskConfig {
}
}
public
void
setMutableObjectMode
(
boolean
mode
)
{
this
.
config
.
setBoolean
(
DRIVER_MUTABLE_OBJECT_MODE
,
mode
);
}
public
boolean
getMutableObjectMode
()
{
return
this
.
config
.
getBoolean
(
DRIVER_MUTABLE_OBJECT_MODE
,
true
);
}
public
void
setDriverComparator
(
TypeComparatorFactory
<?>
factory
,
int
inputNum
)
{
setTypeComparatorFactory
(
factory
,
DRIVER_COMPARATOR_FACTORY_PREFIX
+
inputNum
,
DRIVER_COMPARATOR_PARAMETERS_PREFIX
+
inputNum
+
SEPARATOR
);
...
...
flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyGroupedIterator.java
浏览文件 @
e7c4c858
...
...
@@ -16,7 +16,6 @@
* limitations under the License.
*/
package
org.apache.flink.runtime.util
;
import
java.io.IOException
;
...
...
stratosphere-runtime/src/main/java/eu/stratosphere/pact
/runtime/util/KeyGroupedIteratorImmutable.java
→
flink-runtime/src/main/java/org/apache/flink
/runtime/util/KeyGroupedIteratorImmutable.java
浏览文件 @
e7c4c858
/***********************************************************************************************************************
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
**********************************************************************************************************************/
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package
eu.stratosphere.pact
.runtime.util
;
package
org.apache.flink
.runtime.util
;
import
java.io.IOException
;
import
java.util.Iterator
;
import
java.util.NoSuchElementException
;
import
eu.stratosphere
.api.common.typeutils.TypeComparator
;
import
eu.stratosphere
.api.common.typeutils.TypeSerializer
;
import
eu.stratosphere
.util.MutableObjectIterator
;
import
org.apache.flink
.api.common.typeutils.TypeComparator
;
import
org.apache.flink
.api.common.typeutils.TypeSerializer
;
import
org.apache.flink
.util.MutableObjectIterator
;
import
org.apache.flink.util.TraversableOnceException
;
/**
* The KeyValueIterator returns a key and all values that belong to the key (share the same key).
*
...
...
@@ -75,6 +80,7 @@ public final class KeyGroupedIteratorImmutable<E> {
this
.
valuesIterator
.
next
=
this
.
lookahead
;
this
.
lastKeyRecord
=
this
.
lookahead
;
this
.
lookahead
=
null
;
this
.
valuesIterator
.
iteratorAvailable
=
true
;
return
true
;
}
...
...
@@ -94,6 +100,7 @@ public final class KeyGroupedIteratorImmutable<E> {
this
.
comparator
.
setReference
(
next
);
this
.
valuesIterator
.
next
=
next
;
this
.
lastKeyRecord
=
next
;
this
.
valuesIterator
.
iteratorAvailable
=
true
;
return
true
;
}
}
...
...
@@ -170,10 +177,12 @@ public final class KeyGroupedIteratorImmutable<E> {
// --------------------------------------------------------------------------------------------
public
final
class
ValuesIterator
implements
Iterator
<
E
>
{
public
final
class
ValuesIterator
implements
Iterator
<
E
>
,
Iterable
<
E
>
{
private
E
next
;
private
boolean
iteratorAvailable
=
true
;
private
ValuesIterator
(
E
first
)
{
this
.
next
=
first
;
}
...
...
@@ -198,5 +207,16 @@ public final class KeyGroupedIteratorImmutable<E> {
public
void
remove
()
{
throw
new
UnsupportedOperationException
();
}
@Override
public
Iterator
<
E
>
iterator
()
{
if
(
iteratorAvailable
)
{
iteratorAvailable
=
false
;
return
this
;
}
else
{
throw
new
TraversableOnceException
();
}
}
}
}
flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/DriverTestData.java
浏览文件 @
e7c4c858
...
...
@@ -100,7 +100,7 @@ public class DriverTestData {
public
static
final
void
compareTupleArrays
(
Object
[]
expected
,
Object
[]
found
)
{
if
(
expected
.
length
!=
found
.
length
)
{
throw
new
IllegalArgumentException
(
);
Assert
.
assertEquals
(
"Length of result is wrong"
,
expected
.
length
,
found
.
length
);
}
for
(
int
i
=
0
;
i
<
expected
.
length
;
i
++)
{
...
...
flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java
浏览文件 @
e7c4c858
...
...
@@ -18,6 +18,7 @@
package
org.apache.flink.runtime.operators.drivers
;
import
java.util.ArrayList
;
import
java.util.List
;
import
org.apache.flink.api.common.functions.GroupReduceFunction
;
...
...
@@ -28,7 +29,6 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import
org.apache.flink.api.java.typeutils.TypeExtractor
;
import
org.apache.flink.runtime.operators.DriverStrategy
;
import
org.apache.flink.runtime.operators.GroupReduceDriver
;
import
org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector
;
import
org.apache.flink.runtime.util.EmptyMutableObjectIterator
;
import
org.apache.flink.runtime.util.RegularToMutableObjectIterator
;
import
org.apache.flink.types.IntValue
;
...
...
@@ -53,14 +53,18 @@ public class GroupReduceDriverTest {
TypeComparator
<
Tuple2
<
String
,
Integer
>>
comparator
=
typeInfo
.
createComparator
(
new
int
[]{
0
},
new
boolean
[]
{
true
});
context
.
setDriverStrategy
(
DriverStrategy
.
SORTED_GROUP_REDUCE
);
GatheringCollector
<
Tuple2
<
String
,
Integer
>>
result
=
new
GatheringCollector
<
Tuple2
<
String
,
Integer
>>(
typeInfo
.
createSerializer
());
context
.
setInput1
(
input
,
typeInfo
.
createSerializer
());
context
.
setComparator1
(
comparator
);
context
.
setCollector
(
new
DiscardingOutputCollector
<
Tuple2
<
String
,
Integer
>>()
);
context
.
setCollector
(
result
);
GroupReduceDriver
<
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>>
driver
=
new
GroupReduceDriver
<
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>>();
driver
.
setup
(
context
);
driver
.
prepare
();
driver
.
run
();
Assert
.
assertTrue
(
result
.
getList
().
isEmpty
());
}
catch
(
Exception
e
)
{
System
.
err
.
println
(
e
.
getMessage
());
...
...
@@ -141,7 +145,84 @@ public class GroupReduceDriverTest {
}
}
@Test
public
void
testAllReduceDriverIncorrectlyAccumulatingMutable
()
{
try
{
TestTaskContext
<
GroupReduceFunction
<
Tuple2
<
StringValue
,
IntValue
>,
Tuple2
<
StringValue
,
IntValue
>>,
Tuple2
<
StringValue
,
IntValue
>>
context
=
new
TestTaskContext
<
GroupReduceFunction
<
Tuple2
<
StringValue
,
IntValue
>,
Tuple2
<
StringValue
,
IntValue
>>,
Tuple2
<
StringValue
,
IntValue
>>();
List
<
Tuple2
<
StringValue
,
IntValue
>>
data
=
DriverTestData
.
createReduceMutableData
();
TupleTypeInfo
<
Tuple2
<
StringValue
,
IntValue
>>
typeInfo
=
(
TupleTypeInfo
<
Tuple2
<
StringValue
,
IntValue
>>)
TypeExtractor
.
getForObject
(
data
.
get
(
0
));
MutableObjectIterator
<
Tuple2
<
StringValue
,
IntValue
>>
input
=
new
RegularToMutableObjectIterator
<
Tuple2
<
StringValue
,
IntValue
>>(
data
.
iterator
(),
typeInfo
.
createSerializer
());
TypeComparator
<
Tuple2
<
StringValue
,
IntValue
>>
comparator
=
typeInfo
.
createComparator
(
new
int
[]{
0
},
new
boolean
[]
{
true
});
GatheringCollector
<
Tuple2
<
StringValue
,
IntValue
>>
result
=
new
GatheringCollector
<
Tuple2
<
StringValue
,
IntValue
>>(
typeInfo
.
createSerializer
());
context
.
setDriverStrategy
(
DriverStrategy
.
SORTED_GROUP_REDUCE
);
context
.
setInput1
(
input
,
typeInfo
.
createSerializer
());
context
.
setComparator1
(
comparator
);
context
.
setCollector
(
result
);
context
.
setUdf
(
new
ConcatSumMutableAccumulatingReducer
());
GroupReduceDriver
<
Tuple2
<
StringValue
,
IntValue
>,
Tuple2
<
StringValue
,
IntValue
>>
driver
=
new
GroupReduceDriver
<
Tuple2
<
StringValue
,
IntValue
>,
Tuple2
<
StringValue
,
IntValue
>>();
driver
.
setup
(
context
);
driver
.
prepare
();
driver
.
run
();
Object
[]
res
=
result
.
getList
().
toArray
();
Object
[]
expected
=
DriverTestData
.
createReduceMutableDataGroupedResult
().
toArray
();
try
{
DriverTestData
.
compareTupleArrays
(
expected
,
res
);
Assert
.
fail
(
"Accumulationg mutable objects is expected to result in incorrect values."
);
}
catch
(
AssertionError
e
)
{
// expected
}
}
catch
(
Exception
e
)
{
System
.
err
.
println
(
e
.
getMessage
());
e
.
printStackTrace
();
Assert
.
fail
(
e
.
getMessage
());
}
}
@Test
public
void
testAllReduceDriverAccumulatingImmutable
()
{
try
{
TestTaskContext
<
GroupReduceFunction
<
Tuple2
<
StringValue
,
IntValue
>,
Tuple2
<
StringValue
,
IntValue
>>,
Tuple2
<
StringValue
,
IntValue
>>
context
=
new
TestTaskContext
<
GroupReduceFunction
<
Tuple2
<
StringValue
,
IntValue
>,
Tuple2
<
StringValue
,
IntValue
>>,
Tuple2
<
StringValue
,
IntValue
>>();
List
<
Tuple2
<
StringValue
,
IntValue
>>
data
=
DriverTestData
.
createReduceMutableData
();
TupleTypeInfo
<
Tuple2
<
StringValue
,
IntValue
>>
typeInfo
=
(
TupleTypeInfo
<
Tuple2
<
StringValue
,
IntValue
>>)
TypeExtractor
.
getForObject
(
data
.
get
(
0
));
MutableObjectIterator
<
Tuple2
<
StringValue
,
IntValue
>>
input
=
new
RegularToMutableObjectIterator
<
Tuple2
<
StringValue
,
IntValue
>>(
data
.
iterator
(),
typeInfo
.
createSerializer
());
TypeComparator
<
Tuple2
<
StringValue
,
IntValue
>>
comparator
=
typeInfo
.
createComparator
(
new
int
[]{
0
},
new
boolean
[]
{
true
});
GatheringCollector
<
Tuple2
<
StringValue
,
IntValue
>>
result
=
new
GatheringCollector
<
Tuple2
<
StringValue
,
IntValue
>>(
typeInfo
.
createSerializer
());
context
.
setDriverStrategy
(
DriverStrategy
.
SORTED_GROUP_REDUCE
);
context
.
setInput1
(
input
,
typeInfo
.
createSerializer
());
context
.
setComparator1
(
comparator
);
context
.
setCollector
(
result
);
context
.
setUdf
(
new
ConcatSumMutableAccumulatingReducer
());
context
.
setMutableObjectMode
(
false
);
GroupReduceDriver
<
Tuple2
<
StringValue
,
IntValue
>,
Tuple2
<
StringValue
,
IntValue
>>
driver
=
new
GroupReduceDriver
<
Tuple2
<
StringValue
,
IntValue
>,
Tuple2
<
StringValue
,
IntValue
>>();
driver
.
setup
(
context
);
driver
.
prepare
();
driver
.
run
();
Object
[]
res
=
result
.
getList
().
toArray
();
Object
[]
expected
=
DriverTestData
.
createReduceMutableDataGroupedResult
().
toArray
();
DriverTestData
.
compareTupleArrays
(
expected
,
res
);
}
catch
(
Exception
e
)
{
System
.
err
.
println
(
e
.
getMessage
());
e
.
printStackTrace
();
Assert
.
fail
(
e
.
getMessage
());
}
}
// --------------------------------------------------------------------------------------------
// Test UDFs
...
...
@@ -178,4 +259,26 @@ public class GroupReduceDriverTest {
out
.
collect
(
current
);
}
}
public
static
final
class
ConcatSumMutableAccumulatingReducer
implements
GroupReduceFunction
<
Tuple2
<
StringValue
,
IntValue
>,
Tuple2
<
StringValue
,
IntValue
>>
{
@Override
public
void
reduce
(
Iterable
<
Tuple2
<
StringValue
,
IntValue
>>
values
,
Collector
<
Tuple2
<
StringValue
,
IntValue
>>
out
)
throws
Exception
{
List
<
Tuple2
<
StringValue
,
IntValue
>>
all
=
new
ArrayList
<
Tuple2
<
StringValue
,
IntValue
>>();
for
(
Tuple2
<
StringValue
,
IntValue
>
t
:
values
)
{
all
.
add
(
t
);
}
Tuple2
<
StringValue
,
IntValue
>
result
=
all
.
get
(
0
);
for
(
int
i
=
1
;
i
<
all
.
size
();
i
++)
{
Tuple2
<
StringValue
,
IntValue
>
e
=
all
.
get
(
i
);
result
.
f0
.
append
(
e
.
f0
);
result
.
f1
.
setValue
(
result
.
f1
.
getValue
()
+
e
.
f1
.
getValue
());
}
out
.
collect
(
result
);
}
}
}
flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
浏览文件 @
e7c4c858
...
...
@@ -119,6 +119,10 @@ public class TestTaskContext<S, T> implements PactTaskContext<S, T> {
this
.
config
.
setDriverStrategy
(
strategy
);
}
public
void
setMutableObjectMode
(
boolean
mutableObjectMode
)
{
this
.
config
.
setMutableObjectMode
(
mutableObjectMode
);
}
// --------------------------------------------------------------------------------------------
// Context Methods
// --------------------------------------------------------------------------------------------
...
...
stratosphere-runtime/src/test/java/eu/stratosphere/pact
/runtime/util/KeyGroupedIteratorImmutableTest.java
→
flink-runtime/src/test/java/org/apache/flink
/runtime/util/KeyGroupedIteratorImmutableTest.java
浏览文件 @
e7c4c858
/***********************************************************************************************************************
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
**********************************************************************************************************************/
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package
eu.stratosphere.pact
.runtime.util
;
package
org.apache.flink
.runtime.util
;
import
java.io.IOException
;
import
java.util.ArrayList
;
import
java.util.Iterator
;
import
java.util.NoSuchElementException
;
import
org.apache.flink.api.common.typeutils.record.RecordComparator
;
import
org.apache.flink.api.common.typeutils.record.RecordSerializer
;
import
org.apache.flink.types.IntValue
;
import
org.apache.flink.types.Record
;
import
org.apache.flink.types.StringValue
;
import
org.apache.flink.util.MutableObjectIterator
;
import
org.junit.Assert
;
import
org.junit.Before
;
import
org.junit.Test
;
import
eu.stratosphere.api.java.typeutils.runtime.record.RecordComparator
;
import
eu.stratosphere.api.java.typeutils.runtime.record.RecordSerializer
;
import
eu.stratosphere.types.IntValue
;
import
eu.stratosphere.types.Record
;
import
eu.stratosphere.types.StringValue
;
import
eu.stratosphere.util.MutableObjectIterator
;
/**
* Test for the safe key grouped iterator, which advances in windows containing the same key and provides a sub-iterator
* over the records with the same key.
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录