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

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_r37846982
  
    --- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
 ---
    @@ -0,0 +1,418 @@
    +/*
    + * 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.testutils;
    +
    +import org.apache.flink.api.common.ExecutionConfig;
    +import org.apache.flink.api.common.functions.Function;
    +import org.apache.flink.api.common.functions.util.FunctionUtils;
    +import org.apache.flink.api.common.typeutils.TypeComparator;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
    +import 
org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.runtime.io.disk.iomanager.IOManager;
    +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
    +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
    +import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
    +import org.apache.flink.runtime.memorymanager.MemoryManager;
    +import org.apache.flink.runtime.operators.PactDriver;
    +import org.apache.flink.runtime.operators.PactTaskContext;
    +import org.apache.flink.runtime.operators.ResettablePactDriver;
    +import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
    +import org.apache.flink.runtime.operators.util.TaskConfig;
    +import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.MutableObjectIterator;
    +import org.junit.After;
    +import org.junit.Assert;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.LinkedList;
    +import java.util.List;
    +
    +@RunWith(Parameterized.class)
    +public class BinaryOperatorTestBase<S extends Function, IN, OUT> 
implements PactTaskContext<S, OUT> {
    +
    +   protected static final int PAGE_SIZE = 32 * 1024;
    +
    +   private final IOManager ioManager;
    +
    +   private final MemoryManager memManager;
    +
    +   private final List<MutableObjectIterator<IN>> inputs;
    +
    +   private final List<TypeComparator<IN>> comparators;
    +
    +   private final List<UnilateralSortMerger<IN>> sorters;
    +
    +   private final AbstractInvokable owner;
    +
    +   private final TaskConfig taskConfig;
    +
    +   private final TaskManagerRuntimeInfo taskManageInfo;
    +
    +   protected final long perSortMem;
    +
    +   protected final double perSortFractionMem;
    +
    +   private Collector<OUT> output;
    +
    +   protected int numFileHandles;
    +
    +   private S stub;
    +
    +   private PactDriver<S, IN> driver;
    +
    +   private volatile boolean running;
    +
    +   private ExecutionConfig executionConfig;
    +
    +   private List<TypeSerializer<IN>> inputSerializers = new ArrayList<>();
    +
    +   protected BinaryOperatorTestBase(ExecutionConfig executionConfig, long 
memory, int maxNumSorters, long perSortMemory) {
    +           if (memory < 0 || maxNumSorters < 0 || perSortMemory < 0) {
    +                   throw new IllegalArgumentException();
    +           }
    +           
    +           final long totalMem = Math.max(memory, 0) + 
(Math.max(maxNumSorters, 0) * perSortMemory);
    +           
    +           this.perSortMem = perSortMemory;
    +           this.perSortFractionMem = (double)perSortMemory/totalMem;
    +           this.ioManager = new IOManagerAsync();
    +           this.memManager = totalMem > 0 ? new 
DefaultMemoryManager(totalMem,1) : null;
    +
    +           this.inputs = new ArrayList<>();
    +           this.comparators = new ArrayList<>();
    +           this.sorters = new ArrayList<>();
    +           
    +           this.owner = new DummyInvokable();
    +           this.taskConfig = new TaskConfig(new Configuration());
    +           this.executionConfig = executionConfig;
    +           this.taskManageInfo = new TaskManagerRuntimeInfo("localhost", 
new Configuration());
    +   }
    +
    +   @Parameterized.Parameters
    +   public static Collection<Object[]> getConfigurations() throws 
IOException {
    +
    +           LinkedList<Object[]> configs = new LinkedList<>();
    +
    +           ExecutionConfig withReuse = new ExecutionConfig();
    +           withReuse.enableObjectReuse();
    +
    +           ExecutionConfig withoutReuse = new ExecutionConfig();
    +           withoutReuse.disableObjectReuse();
    +
    +           Object[] a = { withoutReuse };
    +           configs.add(a);
    +           Object[] b = { withReuse };
    +           configs.add(b);
    +
    +           return configs;
    +   }
    +
    +   public void addInput(MutableObjectIterator<IN> input, 
TypeSerializer<IN> serializer) {
    +           this.inputs.add(input);
    +           this.sorters.add(null);
    +           this.inputSerializers.add(serializer);
    +   }
    +   
    +   public void addInputSorted(MutableObjectIterator<IN> input, 
TypeSerializer<IN> serializer,
    +                                                      TypeComparator<IN> 
comp) throws Exception {
    +           this.inputSerializers.add(serializer);
    +           UnilateralSortMerger<IN> sorter = new UnilateralSortMerger<>(
    +                           this.memManager, this.ioManager, input, 
this.owner,
    +                           new RuntimeSerializerFactory<>(serializer, 
(Class<IN>) serializer.createInstance().getClass()),
    +                           comp, this.perSortFractionMem, 32, 0.8f);
    --- End diff --
    
    What's your policy for line breaks 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)

Reply via email to