Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-2871] support outer join for hash on build side.
this commit support full outer join includes: 1. left outer join with REPARTITION_HASH_FIRST. 2. right outer join with REPARTITION_HASH_SECOND. 3. fullouter join with REPARTITION_HASH_FIRST and REPARTITION_HASH_SECOND. this close #1469
- Loading branch information
chengxiang li
committed
Jan 27, 2016
1 parent
a7d4334
commit eaf540f
Showing
37 changed files
with
1,754 additions
and
253 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
65 changes: 65 additions & 0 deletions
65
...main/java/org/apache/flink/optimizer/operators/HashFullOuterJoinBuildFirstDescriptor.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,65 @@ | ||
/* | ||
* 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. | ||
*/ | ||
package org.apache.flink.optimizer.operators; | ||
|
||
import org.apache.flink.api.common.operators.util.FieldList; | ||
import org.apache.flink.optimizer.dag.TwoInputNode; | ||
import org.apache.flink.optimizer.dataproperties.LocalProperties; | ||
import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; | ||
import org.apache.flink.optimizer.plan.Channel; | ||
import org.apache.flink.optimizer.plan.DualInputPlanNode; | ||
import org.apache.flink.runtime.operators.DriverStrategy; | ||
|
||
import java.util.Collections; | ||
import java.util.List; | ||
|
||
public class HashFullOuterJoinBuildFirstDescriptor extends AbstractJoinDescriptor { | ||
|
||
public HashFullOuterJoinBuildFirstDescriptor(FieldList keys1, FieldList keys2) { | ||
super(keys1, keys2, false, false, true); | ||
} | ||
|
||
@Override | ||
public DriverStrategy getStrategy() { | ||
return DriverStrategy.FULL_OUTER_HYBRIDHASH_BUILD_FIRST; | ||
} | ||
|
||
@Override | ||
protected List<LocalPropertiesPair> createPossibleLocalProperties() { | ||
// all properties are possible | ||
return Collections.singletonList(new LocalPropertiesPair(new RequestedLocalProperties(), new RequestedLocalProperties())); | ||
} | ||
|
||
@Override | ||
public boolean areCoFulfilled(RequestedLocalProperties requested1, RequestedLocalProperties requested2, | ||
LocalProperties produced1, LocalProperties produced2) { | ||
return true; | ||
} | ||
|
||
@Override | ||
public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { | ||
|
||
String nodeName = "FullOuterJoin("+node.getOperator().getName()+")"; | ||
return new DualInputPlanNode(node, nodeName, in1, in2, getStrategy(), this.keys1, this.keys2); | ||
} | ||
|
||
@Override | ||
public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { | ||
return new LocalProperties(); | ||
} | ||
} |
64 changes: 64 additions & 0 deletions
64
...ain/java/org/apache/flink/optimizer/operators/HashFullOuterJoinBuildSecondDescriptor.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,64 @@ | ||
/* | ||
* 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. | ||
*/ | ||
package org.apache.flink.optimizer.operators; | ||
|
||
import org.apache.flink.api.common.operators.util.FieldList; | ||
import org.apache.flink.optimizer.dag.TwoInputNode; | ||
import org.apache.flink.optimizer.dataproperties.LocalProperties; | ||
import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; | ||
import org.apache.flink.optimizer.plan.Channel; | ||
import org.apache.flink.optimizer.plan.DualInputPlanNode; | ||
import org.apache.flink.runtime.operators.DriverStrategy; | ||
|
||
import java.util.Collections; | ||
import java.util.List; | ||
|
||
public class HashFullOuterJoinBuildSecondDescriptor extends AbstractJoinDescriptor { | ||
public HashFullOuterJoinBuildSecondDescriptor(FieldList keys1, FieldList keys2) { | ||
super(keys1, keys2, false, false, true); | ||
} | ||
|
||
@Override | ||
public DriverStrategy getStrategy() { | ||
return DriverStrategy.FULL_OUTER_HYBRIDHASH_BUILD_SECOND; | ||
} | ||
|
||
@Override | ||
protected List<LocalPropertiesPair> createPossibleLocalProperties() { | ||
// all properties are possible | ||
return Collections.singletonList(new LocalPropertiesPair(new RequestedLocalProperties(), new RequestedLocalProperties())); | ||
} | ||
|
||
@Override | ||
public boolean areCoFulfilled(RequestedLocalProperties requested1, RequestedLocalProperties requested2, | ||
LocalProperties produced1, LocalProperties produced2) { | ||
return true; | ||
} | ||
|
||
@Override | ||
public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { | ||
|
||
String nodeName = "FullOuterJoin("+node.getOperator().getName()+")"; | ||
return new DualInputPlanNode(node, nodeName, in1, in2, getStrategy(), this.keys1, this.keys2); | ||
} | ||
|
||
@Override | ||
public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { | ||
return new LocalProperties(); | ||
} | ||
} |
65 changes: 65 additions & 0 deletions
65
...main/java/org/apache/flink/optimizer/operators/HashLeftOuterJoinBuildFirstDescriptor.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,65 @@ | ||
/* | ||
* 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. | ||
*/ | ||
package org.apache.flink.optimizer.operators; | ||
|
||
import org.apache.flink.api.common.operators.util.FieldList; | ||
import org.apache.flink.optimizer.dag.TwoInputNode; | ||
import org.apache.flink.optimizer.dataproperties.LocalProperties; | ||
import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; | ||
import org.apache.flink.optimizer.plan.Channel; | ||
import org.apache.flink.optimizer.plan.DualInputPlanNode; | ||
import org.apache.flink.runtime.operators.DriverStrategy; | ||
|
||
import java.util.Collections; | ||
import java.util.List; | ||
|
||
public class HashLeftOuterJoinBuildFirstDescriptor extends AbstractJoinDescriptor{ | ||
public HashLeftOuterJoinBuildFirstDescriptor(FieldList keys1, FieldList keys2, | ||
boolean broadcastSecondAllowed, boolean repartitionAllowed) { | ||
super(keys1, keys2, false, broadcastSecondAllowed, repartitionAllowed); | ||
} | ||
|
||
@Override | ||
public DriverStrategy getStrategy() { | ||
return DriverStrategy.LEFT_HYBRIDHASH_BUILD_FIRST; | ||
} | ||
|
||
@Override | ||
protected List<OperatorDescriptorDual.LocalPropertiesPair> createPossibleLocalProperties() { | ||
// all properties are possible | ||
return Collections.singletonList(new OperatorDescriptorDual.LocalPropertiesPair(new RequestedLocalProperties(), new RequestedLocalProperties())); | ||
} | ||
|
||
@Override | ||
public boolean areCoFulfilled(RequestedLocalProperties requested1, RequestedLocalProperties requested2, | ||
LocalProperties produced1, LocalProperties produced2) { | ||
return true; | ||
} | ||
|
||
@Override | ||
public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { | ||
|
||
String nodeName = "LeftOuterJoin("+node.getOperator().getName()+")"; | ||
return new DualInputPlanNode(node, nodeName, in1, in2, getStrategy(), this.keys1, this.keys2); | ||
} | ||
|
||
@Override | ||
public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { | ||
return new LocalProperties(); | ||
} | ||
} |
65 changes: 65 additions & 0 deletions
65
...in/java/org/apache/flink/optimizer/operators/HashRightOuterJoinBuildSecondDescriptor.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,65 @@ | ||
/* | ||
* 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. | ||
*/ | ||
package org.apache.flink.optimizer.operators; | ||
|
||
import org.apache.flink.api.common.operators.util.FieldList; | ||
import org.apache.flink.optimizer.dag.TwoInputNode; | ||
import org.apache.flink.optimizer.dataproperties.LocalProperties; | ||
import org.apache.flink.optimizer.dataproperties.RequestedLocalProperties; | ||
import org.apache.flink.optimizer.plan.Channel; | ||
import org.apache.flink.optimizer.plan.DualInputPlanNode; | ||
import org.apache.flink.runtime.operators.DriverStrategy; | ||
|
||
import java.util.Collections; | ||
import java.util.List; | ||
|
||
public class HashRightOuterJoinBuildSecondDescriptor extends AbstractJoinDescriptor { | ||
public HashRightOuterJoinBuildSecondDescriptor(FieldList keys1, FieldList keys2, | ||
boolean broadcastFirstAllowed, boolean repartitionAllowed) { | ||
super(keys1, keys2, broadcastFirstAllowed, false, repartitionAllowed); | ||
} | ||
|
||
@Override | ||
public DriverStrategy getStrategy() { | ||
return DriverStrategy.RIGHT_HYBRIDHASH_BUILD_SECOND; | ||
} | ||
|
||
@Override | ||
protected List<LocalPropertiesPair> createPossibleLocalProperties() { | ||
// all properties are possible | ||
return Collections.singletonList(new LocalPropertiesPair(new RequestedLocalProperties(), new RequestedLocalProperties())); | ||
} | ||
|
||
@Override | ||
public boolean areCoFulfilled(RequestedLocalProperties requested1, RequestedLocalProperties requested2, | ||
LocalProperties produced1, LocalProperties produced2) { | ||
return true; | ||
} | ||
|
||
@Override | ||
public DualInputPlanNode instantiate(Channel in1, Channel in2, TwoInputNode node) { | ||
|
||
String nodeName = "RightOuterJoin("+node.getOperator().getName()+")"; | ||
return new DualInputPlanNode(node, nodeName, in1, in2, getStrategy(), this.keys1, this.keys2); | ||
} | ||
|
||
@Override | ||
public LocalProperties computeLocalProperties(LocalProperties in1, LocalProperties in2) { | ||
return new LocalProperties(); | ||
} | ||
} |
Oops, something went wrong.