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