[
https://issues.apache.org/jira/browse/FLINK-2106?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14710900#comment-14710900
]
ASF GitHub Bot commented on FLINK-2106:
---------------------------------------
Github user tillrohrmann commented on a diff in the pull request:
https://github.com/apache/flink/pull/1052#discussion_r37844335
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java
---
@@ -0,0 +1,76 @@
+/*
+ * 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.runtime.operators;
+
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.memorymanager.MemoryManager;
+import
org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
+import
org.apache.flink.runtime.operators.sort.NonReusingMergeOuterJoinIterator;
+import
org.apache.flink.runtime.operators.sort.ReusingMergeOuterJoinIterator;
+import org.apache.flink.util.MutableObjectIterator;
+
+/**
+ * The full outer join driver implements the logic of an outer join
operator at runtime. It instantiates a
+ * sort-merge based strategy to find joining pairs of records or joins
records with null if no match is found.
+ *
+ */
+public class FullOuterJoinDriver<IT1, IT2, OT> extends
AbstractOuterJoinDriver<IT1, IT2, OT> {
+
+ @Override
+ protected void setReusingOuterJoinIterator(DriverStrategy
driverStrategy, MutableObjectIterator<IT1> in1,
+
MutableObjectIterator<IT2> in2, TypeSerializer<IT1> serializer1,
+
TypeComparator<IT1> comparator1, TypeSerializer<IT2> serializer2,
+
TypeComparator<IT2> comparator2, TypePairComparatorFactory<IT1, IT2>
pairComparatorFactory,
+
MemoryManager memoryManager, IOManager ioManager, int numPages) throws
Exception {
+ switch (driverStrategy) {
+ case FULL_OUTER_MERGE:
+ super.outerJoinIterator = new
ReusingMergeOuterJoinIterator<>(OuterJoinType.FULL, in1, in2,
+ serializer1, comparator1,
+ serializer2, comparator2,
+
pairComparatorFactory.createComparator12(comparator1, comparator2),
+ memoryManager, ioManager,
numPages, this.taskContext.getOwningNepheleTask());
+ break;
+ default:
+ throw new Exception("Unsupported driver
strategy for full outer join driver: " + driverStrategy.name());
+ }
+ }
+
+ @Override
+ protected void setNonReusingOuterJoinIterator(DriverStrategy
driverStrategy, MutableObjectIterator<IT1> in1,
+
MutableObjectIterator<IT2> in2, TypeSerializer<IT1> serializer1,
+
TypeComparator<IT1> comparator1, TypeSerializer<IT2>
serializer2,
+
TypeComparator<IT2> comparator2, TypePairComparatorFactory<IT1,
IT2> pairComparatorFactory,
+
MemoryManager memoryManager, IOManager ioManager, int numPages)
throws Exception {
+ switch (driverStrategy) {
+ case FULL_OUTER_MERGE:
+ this.outerJoinIterator = new
NonReusingMergeOuterJoinIterator<>(OuterJoinType.FULL, in1, in2,
+ serializer1, comparator1,
+ serializer2, comparator2,
+
pairComparatorFactory.createComparator12(comparator1, comparator2),
+ memoryManager, ioManager,
numPages, this.taskContext.getOwningNepheleTask());
--- End diff --
Same here.
> Add outer joins to API, Optimizer, and Runtime
> ----------------------------------------------
>
> Key: FLINK-2106
> URL: https://issues.apache.org/jira/browse/FLINK-2106
> Project: Flink
> Issue Type: Sub-task
> Components: Java API, Local Runtime, Optimizer, Scala API
> Reporter: Fabian Hueske
> Assignee: Ricky Pogalz
> Priority: Minor
> Fix For: pre-apache
>
>
> Add left/right/full outer join methods to the DataSet APIs (Java, Scala), to
> the optimizer, and the runtime of Flink.
> Initially, the execution strategy should be a sort-merge outer join
> (FLINK-2105) but can later be extended to hash joins for left/right outer
> joins.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)