Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
0954c4e5
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,发现更多精彩内容 >>
提交
0954c4e5
编写于
1月 08, 2015
作者:
S
Stephan Ewen
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-333] [apis] Forward crossWithSmall and crossWithLarge hints to optimizer.
上级
39fb7c94
变更
7
展开全部
隐藏空白更改
内联
并排
Showing
7 changed file
with
137 addition
and
78 deletion
+137
-78
flink-compiler/src/main/java/org/apache/flink/compiler/dag/CrossNode.java
...rc/main/java/org/apache/flink/compiler/dag/CrossNode.java
+9
-13
flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java
...he/flink/api/common/operators/base/CrossOperatorBase.java
+33
-0
flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
...java/src/main/java/org/apache/flink/api/java/DataSet.java
+4
-3
flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java
...va/org/apache/flink/api/java/operators/CrossOperator.java
+68
-43
flink-java/src/main/java/org/apache/flink/api/java/tuple/TupleGenerator.java
.../java/org/apache/flink/api/java/tuple/TupleGenerator.java
+7
-10
flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
...a/src/main/scala/org/apache/flink/api/scala/DataSet.scala
+7
-8
flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
.../main/scala/org/apache/flink/api/scala/crossDataSet.scala
+9
-1
未找到文件。
flink-compiler/src/main/java/org/apache/flink/compiler/dag/CrossNode.java
浏览文件 @
0954c4e5
...
...
@@ -23,6 +23,7 @@ import java.util.Collections;
import
java.util.List
;
import
org.apache.flink.api.common.operators.base.CrossOperatorBase
;
import
org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
;
import
org.apache.flink.compiler.CompilerException
;
import
org.apache.flink.compiler.DataStatistics
;
import
org.apache.flink.compiler.PactCompiler
;
...
...
@@ -48,21 +49,16 @@ public class CrossNode extends TwoInputNode {
public
CrossNode
(
CrossOperatorBase
<?,
?,
?,
?>
operation
)
{
super
(
operation
);
// check small / large hints to decide upon which side is to be broadcasted
boolean
allowBCfirst
=
true
;
boolean
allowBCsecond
=
true
;
if
(
operation
instanceof
CrossOperatorBase
.
CrossWithSmall
)
{
allowBCfirst
=
false
;
}
else
if
(
operation
instanceof
CrossOperatorBase
.
CrossWithLarge
)
{
allowBCsecond
=
false
;
}
Configuration
conf
=
operation
.
getParameters
();
String
localStrategy
=
conf
.
getString
(
PactCompiler
.
HINT_LOCAL_STRATEGY
,
null
);
CrossHint
hint
=
operation
.
getCrossHint
();
if
(
localStrategy
!=
null
)
{
final
boolean
allowBCfirst
=
hint
!=
CrossHint
.
SECOND_IS_SMALL
;
final
boolean
allowBCsecond
=
hint
!=
CrossHint
.
FIRST_IS_SMALL
;
final
OperatorDescriptorDual
fixedDriverStrat
;
if
(
PactCompiler
.
HINT_LOCAL_STRATEGY_NESTEDLOOP_BLOCKED_OUTER_FIRST
.
equals
(
localStrategy
))
{
fixedDriverStrat
=
new
CrossBlockOuterFirstDescriptor
(
allowBCfirst
,
allowBCsecond
);
...
...
@@ -78,13 +74,13 @@ public class CrossNode extends TwoInputNode {
this
.
dataProperties
=
Collections
.
singletonList
(
fixedDriverStrat
);
}
else
if
(
operation
instanceof
CrossOperatorBase
.
CrossWithSmall
)
{
else
if
(
hint
==
CrossHint
.
SECOND_IS_SMALL
)
{
ArrayList
<
OperatorDescriptorDual
>
list
=
new
ArrayList
<
OperatorDescriptorDual
>();
list
.
add
(
new
CrossBlockOuterSecondDescriptor
(
false
,
true
));
list
.
add
(
new
CrossStreamOuterFirstDescriptor
(
false
,
true
));
this
.
dataProperties
=
list
;
}
else
if
(
operation
instanceof
CrossOperatorBase
.
CrossWithLarge
)
{
else
if
(
hint
==
CrossHint
.
FIRST_IS_SMALL
)
{
ArrayList
<
OperatorDescriptorDual
>
list
=
new
ArrayList
<
OperatorDescriptorDual
>();
list
.
add
(
new
CrossBlockOuterFirstDescriptor
(
true
,
false
));
list
.
add
(
new
CrossStreamOuterSecondDescriptor
(
true
,
false
));
...
...
flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java
浏览文件 @
0954c4e5
...
...
@@ -36,8 +36,32 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
*/
public
class
CrossOperatorBase
<
IN1
,
IN2
,
OUT
,
FT
extends
CrossFunction
<
IN1
,
IN2
,
OUT
>>
extends
DualInputOperator
<
IN1
,
IN2
,
OUT
,
FT
>
{
/**
* The cross hint tells the system which sizes to expect from the data sets
*/
public
static
enum
CrossHint
{
OPTIMIZER_CHOOSES
,
FIRST_IS_SMALL
,
SECOND_IS_SMALL
}
// --------------------------------------------------------------------------------------------
private
CrossHint
hint
=
CrossHint
.
OPTIMIZER_CHOOSES
;
public
CrossOperatorBase
(
UserCodeWrapper
<
FT
>
udf
,
BinaryOperatorInformation
<
IN1
,
IN2
,
OUT
>
operatorInfo
,
String
name
)
{
super
(
udf
,
operatorInfo
,
name
);
if
(
this
instanceof
CrossWithSmall
)
{
setCrossHint
(
CrossHint
.
SECOND_IS_SMALL
);
}
else
if
(
this
instanceof
CrossWithLarge
)
{
setCrossHint
(
CrossHint
.
FIRST_IS_SMALL
);
}
}
public
CrossOperatorBase
(
FT
udf
,
BinaryOperatorInformation
<
IN1
,
IN2
,
OUT
>
operatorInfo
,
String
name
)
{
...
...
@@ -48,6 +72,15 @@ public class CrossOperatorBase<IN1, IN2, OUT, FT extends CrossFunction<IN1, IN2,
this
(
new
UserCodeClassWrapper
<
FT
>(
udf
),
operatorInfo
,
name
);
}
public
void
setCrossHint
(
CrossHint
hint
)
{
this
.
hint
=
hint
==
null
?
CrossHint
.
OPTIMIZER_CHOOSES
:
hint
;
}
public
CrossHint
getCrossHint
()
{
return
hint
;
}
// --------------------------------------------------------------------------------------------
@Override
...
...
flink-java/src/main/java/org/apache/flink/api/java/DataSet.java
浏览文件 @
0954c4e5
...
...
@@ -31,6 +31,7 @@ import org.apache.flink.api.common.functions.ReduceFunction;
import
org.apache.flink.api.common.io.FileOutputFormat
;
import
org.apache.flink.api.common.io.OutputFormat
;
import
org.apache.flink.api.common.typeinfo.TypeInformation
;
import
org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
;
import
org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint
;
import
org.apache.flink.api.common.operators.base.PartitionOperatorBase.PartitionMethod
;
import
org.apache.flink.api.java.aggregation.Aggregations
;
...
...
@@ -791,7 +792,7 @@ public abstract class DataSet<T> {
* @see Tuple2
*/
public
<
R
>
CrossOperator
.
DefaultCross
<
T
,
R
>
cross
(
DataSet
<
R
>
other
)
{
return
new
CrossOperator
.
DefaultCross
<
T
,
R
>(
this
,
other
,
Utils
.
getCallLocationName
());
return
new
CrossOperator
.
DefaultCross
<
T
,
R
>(
this
,
other
,
CrossHint
.
OPTIMIZER_CHOOSES
,
Utils
.
getCallLocationName
());
}
/**
...
...
@@ -821,7 +822,7 @@ public abstract class DataSet<T> {
* @see Tuple2
*/
public
<
R
>
CrossOperator
.
DefaultCross
<
T
,
R
>
crossWithTiny
(
DataSet
<
R
>
other
)
{
return
new
CrossOperator
.
DefaultCross
<
T
,
R
>(
this
,
other
,
Utils
.
getCallLocationName
());
return
new
CrossOperator
.
DefaultCross
<
T
,
R
>(
this
,
other
,
CrossHint
.
SECOND_IS_SMALL
,
Utils
.
getCallLocationName
());
}
/**
...
...
@@ -851,7 +852,7 @@ public abstract class DataSet<T> {
* @see Tuple2
*/
public
<
R
>
CrossOperator
.
DefaultCross
<
T
,
R
>
crossWithHuge
(
DataSet
<
R
>
other
)
{
return
new
CrossOperator
.
DefaultCross
<
T
,
R
>(
this
,
other
,
Utils
.
getCallLocationName
());
return
new
CrossOperator
.
DefaultCross
<
T
,
R
>(
this
,
other
,
CrossHint
.
FIRST_IS_SMALL
,
Utils
.
getCallLocationName
());
}
// --------------------------------------------------------------------------------------------
...
...
flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java
浏览文件 @
0954c4e5
此差异已折叠。
点击以展开。
flink-java/src/main/java/org/apache/flink/api/java/tuple/TupleGenerator.java
浏览文件 @
0954c4e5
...
...
@@ -166,11 +166,10 @@ class TupleGenerator {
// method comment
sb
.
append
(
"\t\t/**\n"
);
sb
.
append
(
"\t\t * Chooses a projectTupleX according to the length of {@link CrossProjection#fieldIndexes} \n"
);
sb
.
append
(
"\t\t * Chooses a projectTupleX according to the length of\n"
);
sb
.
append
(
"\t\t * {@link org.apache.flink.api.java.operators.CrossOperator.CrossProjection#fieldIndexes} \n"
);
sb
.
append
(
"\t\t * \n"
);
sb
.
append
(
"\t\t * @return The projected DataSet.\n"
);
sb
.
append
(
"\t\t * \n"
);
sb
.
append
(
"\t\t * @see Projection\n"
);
sb
.
append
(
"\t\t */\n"
);
// method signature
...
...
@@ -224,7 +223,7 @@ class TupleGenerator {
// create and return new project operator
sb
.
append
(
"\t\t\treturn new ProjectCross<I1, I2, Tuple"
+
numFields
+
"<"
);
appendTupleTypeGenerics
(
sb
,
numFields
);
sb
.
append
(
">>(this.ds1, this.ds2, this.fieldIndexes, this.isFieldInFirst, tType, this);\n"
);
sb
.
append
(
">>(this.ds1, this.ds2, this.fieldIndexes, this.isFieldInFirst, tType, this
, hint
);\n"
);
// method end
sb
.
append
(
"\t\t}\n"
);
...
...
@@ -246,11 +245,10 @@ class TupleGenerator {
// method comment
sb
.
append
(
"\t\t/**\n"
);
sb
.
append
(
"\t\t * Chooses a projectTupleX according to the length of {@link Projection#fieldIndexes} \n"
);
sb
.
append
(
"\t\t * Chooses a projectTupleX according to the length of\n"
);
sb
.
append
(
"\t\t * {@link org.apache.flink.api.java.operators.ProjectOperator.Projection#fieldIndexes} \n"
);
sb
.
append
(
"\t\t * \n"
);
sb
.
append
(
"\t\t * @return The projected DataSet.\n"
);
sb
.
append
(
"\t\t * \n"
);
sb
.
append
(
"\t\t * @see Projection\n"
);
sb
.
append
(
"\t\t */\n"
);
// method signature
...
...
@@ -326,11 +324,10 @@ class TupleGenerator {
// method comment
sb
.
append
(
"\t\t/**\n"
);
sb
.
append
(
"\t\t * Chooses a projectTupleX according to the length of {@link JoinProjection#fieldIndexes} \n"
);
sb
.
append
(
"\t\t * Chooses a projectTupleX according to the length of\n"
);
sb
.
append
(
"\t\t * {@link org.apache.flink.api.java.operators.JoinOperator.JoinProjection#fieldIndexes}\n"
);
sb
.
append
(
"\t\t * \n"
);
sb
.
append
(
"\t\t * @return The projected DataSet.\n"
);
sb
.
append
(
"\t\t * \n"
);
sb
.
append
(
"\t\t * @see Projection\n"
);
sb
.
append
(
"\t\t */\n"
);
// method signature
...
...
flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala
浏览文件 @
0954c4e5
...
...
@@ -26,7 +26,8 @@ import org.apache.flink.api.common.operators.base.PartitionOperatorBase.Partitio
import
org.apache.flink.api.java.aggregation.Aggregations
import
org.apache.flink.api.java.functions.
{
FirstReducer
,
KeySelector
}
import
org.apache.flink.api.java.io.
{
PrintingOutputFormat
,
TextOutputFormat
}
import
org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint
;
import
org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint
import
org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
import
org.apache.flink.api.java.operators.Keys.ExpressionKeys
import
org.apache.flink.api.java.operators._
import
org.apache.flink.api.java.
{
DataSet
=>
JavaDataSet
}
...
...
@@ -36,7 +37,6 @@ import org.apache.flink.core.fs.FileSystem.WriteMode
import
org.apache.flink.core.fs.
{
FileSystem
,
Path
}
import
org.apache.flink.api.common.typeinfo.TypeInformation
import
org.apache.flink.util.Collector
import
scala.collection.JavaConverters._
import
scala.reflect.ClassTag
...
...
@@ -254,10 +254,9 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) {
javaSet
match
{
case
udfOp
:
UdfOperator
[
_
]
=>
udfOp
.
withParameters
(
parameters
)
case
source
:
DataSource
[
_
]
=>
source
.
withParameters
(
parameters
)
case
sink
:
DataSink
[
_
]
=>
sink
.
withParameters
(
parameters
)
case
_
=>
throw
new
UnsupportedOperationException
(
"Operator "
+
javaSet
.
toString
+
" cannot have "
+
"
parameters"
)
throw
new
UnsupportedOperationException
(
"Operator "
+
javaSet
.
toString
+
" cannot have
parameters"
)
}
this
}
...
...
@@ -836,21 +835,21 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) {
* }}}
*/
def
cross
[
O
](
other
:
DataSet
[
O
])
:
CrossDataSet
[
T
,
O
]
=
CrossDataSet
.
createCrossOperator
(
this
,
other
)
CrossDataSet
.
createCrossOperator
(
this
,
other
,
CrossHint
.
OPTIMIZER_CHOOSES
)
/**
* Special [[cross]] operation for explicitly telling the system that the right side is assumed
* to be a lot smaller than the left side of the cartesian product.
*/
def
crossWithTiny
[
O
](
other
:
DataSet
[
O
])
:
CrossDataSet
[
T
,
O
]
=
CrossDataSet
.
createCrossOperator
(
this
,
other
)
CrossDataSet
.
createCrossOperator
(
this
,
other
,
CrossHint
.
SECOND_IS_SMALL
)
/**
* Special [[cross]] operation for explicitly telling the system that the left side is assumed
* to be a lot smaller than the right side of the cartesian product.
*/
def
crossWithHuge
[
O
](
other
:
DataSet
[
O
])
:
CrossDataSet
[
T
,
O
]
=
CrossDataSet
.
createCrossOperator
(
this
,
other
)
CrossDataSet
.
createCrossOperator
(
this
,
other
,
CrossHint
.
FIRST_IS_SMALL
)
// --------------------------------------------------------------------------------------------
// Iterations
...
...
flink-scala/src/main/scala/org/apache/flink/api/scala/crossDataSet.scala
浏览文件 @
0954c4e5
...
...
@@ -25,6 +25,7 @@ import org.apache.flink.api.java.{DataSet => JavaDataSet}
import
org.apache.flink.api.scala.typeutils.
{
CaseClassSerializer
,
CaseClassTypeInfo
}
import
org.apache.flink.api.common.typeinfo.TypeInformation
import
org.apache.flink.util.Collector
import
org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint
import
scala.reflect.ClassTag
...
...
@@ -69,6 +70,7 @@ class CrossDataSet[L, R](
rightInput
.
javaSet
,
crosser
,
implicitly
[
TypeInformation
[
O
]],
defaultCross
.
getCrossHint
(),
getCallLocationName
())
wrap
(
crossOperator
)
}
...
...
@@ -88,6 +90,7 @@ class CrossDataSet[L, R](
rightInput
.
javaSet
,
crosser
,
implicitly
[
TypeInformation
[
O
]],
defaultCross
.
getCrossHint
(),
getCallLocationName
())
wrap
(
crossOperator
)
}
...
...
@@ -98,7 +101,11 @@ private[flink] object CrossDataSet {
/**
* Creates a default cross operation with Tuple2 as result.
*/
def
createCrossOperator
[
L
,
R
](
leftInput
:
DataSet
[
L
],
rightInput
:
DataSet
[
R
])
=
{
def
createCrossOperator
[
L
,
R
](
leftInput
:
DataSet
[
L
],
rightInput
:
DataSet
[
R
],
crossHint
:
CrossHint
)
=
{
val
crosser
=
new
CrossFunction
[
L
,
R
,
(
L
,
R
)]
{
def
cross
(
left
:
L
,
right
:
R
)
=
{
(
left
,
right
)
...
...
@@ -125,6 +132,7 @@ private[flink] object CrossDataSet {
rightInput
.
javaSet
,
crosser
,
returnType
,
crossHint
,
getCallLocationName
())
new
CrossDataSet
(
crossOperator
,
leftInput
,
rightInput
)
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录