[
https://issues.apache.org/jira/browse/FLINK-2105?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14647522#comment-14647522
]
ASF GitHub Bot commented on FLINK-2105:
---------------------------------------
Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/907#discussion_r35863641
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java
---
@@ -0,0 +1,108 @@
+/*
+ * 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.sort;
+
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypePairComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
+import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.MutableObjectIterator;
+
+import java.util.Iterator;
+
+/**
+ * An implementation of the {@link
org.apache.flink.runtime.operators.util.JoinTaskIterator} that realizes the
+ * matching through a sort-merge join strategy.
+ */
+public abstract class AbstractMergeInnerJoinIterator<T1, T2, O> extends
AbstractMergeIterator<T1, T2, O> {
+
+ public AbstractMergeInnerJoinIterator(
+ MutableObjectIterator<T1> input1,
MutableObjectIterator<T2> input2,
+ TypeSerializer<T1> serializer1, TypeComparator<T1>
comparator1,
+ TypeSerializer<T2> serializer2, TypeComparator<T2>
comparator2,
+ TypePairComparator<T1, T2> pairComparator,
+ MemoryManager memoryManager,
+ IOManager ioManager,
+ int numMemoryPages,
+ AbstractInvokable parentTask)
+ throws MemoryAllocationException {
+ super(input1, input2, serializer1, comparator1, serializer2,
comparator2, pairComparator, memoryManager, ioManager, numMemoryPages,
parentTask);
+ }
+
+ /**
+ * Calls the <code>JoinFunction#match()</code> method for all two
key-value pairs that share the same key and come
--- End diff --
`match()` function has been renamed to `join()`. Please update, thanks!
> Implement Sort-Merge Outer Join algorithm
> -----------------------------------------
>
> Key: FLINK-2105
> URL: https://issues.apache.org/jira/browse/FLINK-2105
> Project: Flink
> Issue Type: Sub-task
> Components: Local Runtime
> Reporter: Fabian Hueske
> Assignee: Ricky Pogalz
> Priority: Minor
> Fix For: pre-apache
>
>
> Flink does not natively support outer joins at the moment.
> This issue proposes to implement a sort-merge outer join algorithm that can
> cover left, right, and full outer joins.
> The implementation can be based on the regular sort-merge join iterator
> ({{ReusingMergeMatchIterator}} and {{NonReusingMergeMatchIterator}}, see also
> {{MatchDriver}} class)
> The Reusing and NonReusing variants differ in whether object instances are
> reused or new objects are created. I would start with the NonReusing variant
> which is safer from a user's point of view and should also be easier to
> implement.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)