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

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_r37843780
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
 ---
    @@ -0,0 +1,166 @@
    +/*
    + * 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.ExecutionConfig;
    +import org.apache.flink.api.common.functions.FlatJoinFunction;
    +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.util.JoinTaskIterator;
    +import org.apache.flink.runtime.operators.util.TaskConfig;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.MutableObjectIterator;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * The abstract 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 joining 
records with null depending
    + * on the outer join type.
    + * 
    + * @see FlatJoinFunction
    + */
    +public abstract class AbstractOuterJoinDriver<IT1, IT2, OT> implements 
PactDriver<FlatJoinFunction<IT1, IT2, OT>, OT> {
    +
    +   protected static final Logger LOG = 
LoggerFactory.getLogger(AbstractOuterJoinDriver.class);
    +
    +   protected PactTaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> 
taskContext;
    +
    +   protected volatile JoinTaskIterator<IT1, IT2, OT> outerJoinIterator; // 
the iterator that does the actual outer join
    +   protected volatile boolean running;
    +
    +   // 
------------------------------------------------------------------------
    +
    +   @Override
    +   public void setup(PactTaskContext<FlatJoinFunction<IT1, IT2, OT>, OT> 
context) {
    +           this.taskContext = context;
    +           this.running = true;
    +   }
    +
    +   @Override
    +   public int getNumberOfInputs() {
    +           return 2;
    +   }
    +
    +   @Override
    +   public Class<FlatJoinFunction<IT1, IT2, OT>> getStubType() {
    +           @SuppressWarnings("unchecked")
    +           final Class<FlatJoinFunction<IT1, IT2, OT>> clazz = 
(Class<FlatJoinFunction<IT1, IT2, OT>>) (Class<?>) FlatJoinFunction.class;
    +           return clazz;
    +   }
    +   
    +   @Override
    +   public int getNumberOfDriverComparators() {
    +           return 2;
    +   }
    +
    +   @Override
    +   public void prepare() throws Exception{
    +           final TaskConfig config = this.taskContext.getTaskConfig();
    +           
    +           // obtain task manager's memory manager and I/O manager
    +           final MemoryManager memoryManager = 
this.taskContext.getMemoryManager();
    +           final IOManager ioManager = this.taskContext.getIOManager();
    +           
    +           // set up memory and I/O parameters
    +           final double fractionAvailableMemory = 
config.getRelativeMemoryDriver();
    +           final int numPages = 
memoryManager.computeNumberOfPages(fractionAvailableMemory);
    +           
    +           // test minimum memory requirements
    +           final DriverStrategy ls = config.getDriverStrategy();
    +           
    +           final MutableObjectIterator<IT1> in1 = 
this.taskContext.getInput(0);
    +           final MutableObjectIterator<IT2> in2 = 
this.taskContext.getInput(1);
    +
    +           // get the key positions and types
    +           final TypeSerializer<IT1> serializer1 = 
this.taskContext.<IT1>getInputSerializer(0).getSerializer();
    +           final TypeSerializer<IT2> serializer2 = 
this.taskContext.<IT2>getInputSerializer(1).getSerializer();
    +           final TypeComparator<IT1> comparator1 = 
this.taskContext.getDriverComparator(0);
    +           final TypeComparator<IT2> comparator2 = 
this.taskContext.getDriverComparator(1);
    +           
    +           final TypePairComparatorFactory<IT1, IT2> pairComparatorFactory 
= config.getPairComparatorFactory(
    +                           this.taskContext.getUserCodeClassLoader());
    +           if (pairComparatorFactory == null) {
    +                   throw new Exception("Missing pair comparator factory 
for outer join driver");
    +           }
    +
    +           ExecutionConfig executionConfig = 
taskContext.getExecutionConfig();
    +           boolean objectReuseEnabled = 
executionConfig.isObjectReuseEnabled();
    +
    +           if (LOG.isDebugEnabled()) {
    +                   LOG.debug("Outer Join Driver object reuse: " + 
(objectReuseEnabled ? "ENABLED" : "DISABLED") + ".");
    +           }
    +
    +           // create and return outer join iterator according to provided 
local strategy.
    +           if (objectReuseEnabled) {
    +                   setReusingOuterJoinIterator(ls, in1, in2, serializer1, 
comparator1, serializer2, comparator2, pairComparatorFactory,
    +                                   memoryManager, ioManager, numPages);
    +           } else {
    +                   setNonReusingOuterJoinIterator(ls, in1, in2, 
serializer1, comparator1, serializer2, comparator2, pairComparatorFactory,
    +                                   memoryManager, ioManager, numPages);
    +           }
    +           
    +           this.outerJoinIterator.open();
    --- End diff --
    
    It might be clearer if you let the `setReusingOuterJoinIterator` and 
`setNonReusingOuterJoinIterator` method return the iterator and assign it in 
this method to `this.outerJoinIterator`. Otherwise, it might be a little bit 
surprising where this variable was actually set.


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

Reply via email to