[ 
https://issues.apache.org/jira/browse/FLINK-2105?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14639621#comment-14639621
 ] 

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_r35379539
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java
 ---
    @@ -0,0 +1,189 @@
    +/*
    + * 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
    + * outer join through a sort-merge join strategy.
    + * */
    +public abstract class AbstractMergeOuterJoinIterator<T1, T2, O> extends 
AbstractMergeIterator<T1, T2, O> {
    +
    +   public static enum OuterJoinType {LEFT, RIGHT, FULL}
    +
    +   private Boolean it1Empty;
    +   private Boolean it2Empty;
    +
    +   private OuterJoinType outerJoinType;
    +
    +
    +   public AbstractMergeOuterJoinIterator(
    +                   OuterJoinType outerJoinType,
    +                   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);
    +
    +           this.outerJoinType = outerJoinType;
    +   }
    +
    +   /**
    +    * Calls the <code>JoinFunction#match()</code> method for all two 
key-value pairs that share the same key and come
    +    * from different inputs. Furthermore, depending on the outer join type 
(LEFT, RIGHT, FULL), all key-value pairs where no
    +    * matching partner from the other input exists are joined with null.
    +    * The output of the <code>match()</code> method is forwarded.
    +    *
    +    * @throws Exception Forwards all exceptions from the user code and the 
I/O system.
    +    * @see 
org.apache.flink.runtime.operators.util.JoinTaskIterator#callWithNextKey(org.apache.flink.api.common.functions.FlatJoinFunction,
 org.apache.flink.util.Collector)
    +    */
    +   @Override
    +   public boolean callWithNextKey(final FlatJoinFunction<T1, T2, O> 
matchFunction, final Collector<O> collector) throws Exception {
    +           if(it1Empty == null && it2Empty == null) {
    +                   //first run, set iterators to first elements
    +                   it1Empty = !this.iterator1.nextKey();
    +                   it2Empty = !this.iterator2.nextKey();
    +           }
    +
    +           if (it1Empty && it2Empty) {
    +                   return false;
    +           }
    +           if (it2Empty) {
    +                   if(outerJoinType == OuterJoinType.LEFT || outerJoinType 
== OuterJoinType.FULL) {
    +                           
joinLeftKeyValuesWithNull(iterator1.getValues(), matchFunction, collector);
    +                           it1Empty = !iterator1.nextKey();
    +                           return true;
    +                   }else{
    +                           //consume rest of left side
    +                           while(iterator1.nextKey());
    +                           it1Empty = true;
    +                           return false;
    +                   }
    +           }
    +           if (it1Empty) {
    +                   if(outerJoinType == OuterJoinType.RIGHT || 
outerJoinType == OuterJoinType.FULL) {
    +                           
joinRightKeyValuesWithNull(iterator2.getValues(), matchFunction, collector);
    +                           it2Empty = !iterator2.nextKey();
    +                           return true;
    +                   }else{
    +                           //consume rest of right side
    +                           while(iterator2.nextKey());
    +                           it2Empty = true;
    +                           return false;
    +                   }
    +           }
    +
    +           final TypePairComparator<T1, T2> comparator = 
super.pairComparator;
    --- End diff --
    
    Please put this in an `else` block.


> 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)

Reply via email to