You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2015/03/20 11:06:44 UTC
[05/53] [abbrv] flink git commit: [optimizer] Rename optimizer
project to "flink-optimizer" (previously flink-compiler)
http://git-wip-us.apache.org/repos/asf/flink/blob/633b0d6a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReduceAllTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReduceAllTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReduceAllTest.java
new file mode 100644
index 0000000..f6885c5
--- /dev/null
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReduceAllTest.java
@@ -0,0 +1,61 @@
+/*
+ * 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.optimizer;
+
+import static org.junit.Assert.fail;
+
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.java.record.operators.FileDataSink;
+import org.apache.flink.api.java.record.operators.FileDataSource;
+import org.apache.flink.api.java.record.operators.ReduceOperator;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.apache.flink.optimizer.util.DummyInputFormat;
+import org.apache.flink.optimizer.util.DummyOutputFormat;
+import org.apache.flink.optimizer.util.IdentityReduce;
+import org.junit.Test;
+
+/**
+ * This test case has been created to validate a bug that occurred when
+ * the ReduceOperator was used without a grouping key.
+ */
+@SuppressWarnings({"serial", "deprecation"})
+public class ReduceAllTest extends CompilerTestBase {
+
+ @Test
+ public void testReduce() {
+ // construct the plan
+ FileDataSource source = new FileDataSource(new DummyInputFormat(), IN_FILE, "Source");
+ ReduceOperator reduce1 = ReduceOperator.builder(new IdentityReduce()).name("Reduce1").input(source).build();
+ FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, "Sink");
+ sink.setInput(reduce1);
+ Plan plan = new Plan(sink, "AllReduce Test");
+ plan.setDefaultParallelism(DEFAULT_PARALLELISM);
+
+
+ try {
+ OptimizedPlan oPlan = compileNoStats(plan);
+ JobGraphGenerator jobGen = new JobGraphGenerator();
+ jobGen.compileJobGraph(oPlan);
+ } catch(CompilerException ce) {
+ ce.printStackTrace();
+ fail("The pact compiler is unable to compile this plan correctly");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/633b0d6a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java
new file mode 100644
index 0000000..da44b59
--- /dev/null
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/ReplicatingDataSourceTest.java
@@ -0,0 +1,495 @@
+/*
+ * 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.optimizer;
+
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.MapPartitionFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.io.ReplicatingInputFormat;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.CsvInputFormat;
+import org.apache.flink.api.java.operators.DataSink;
+import org.apache.flink.api.java.operators.translation.JavaPlan;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.optimizer.plan.DualInputPlanNode;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plan.SinkPlanNode;
+import org.apache.flink.core.fs.FileInputSplit;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
+import org.apache.flink.util.Collector;
+import org.junit.Assert;
+import org.junit.Test;
+
+@SuppressWarnings({"serial", "deprecation"})
+public class ReplicatingDataSourceTest extends CompilerTestBase {
+
+ /**
+ * Tests join program with replicated data source.
+ */
+ @Test
+ public void checkJoinWithReplicatedSourceInput() {
+
+ ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+ env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+
+ ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
+ new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
+
+ DataSet<Tuple1<String>> source1 = env.createInput(rif, new TupleTypeInfo<Tuple1<String>>(BasicTypeInfo.STRING_TYPE_INFO));
+ DataSet<Tuple1<String>> source2 = env.readCsvFile("/some/otherpath").types(String.class);
+
+ DataSink<Tuple2<Tuple1<String>, Tuple1<String>>> out = source1
+ .join(source2).where("*").equalTo("*")
+ .writeAsText("/some/newpath");
+
+ JavaPlan plan = env.createProgramPlan();
+
+ // submit the plan to the compiler
+ OptimizedPlan oPlan = compileNoStats(plan);
+
+ // check the optimized Plan
+ // when join should have forward strategy on both sides
+ SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
+ DualInputPlanNode joinNode = (DualInputPlanNode) sinkNode.getPredecessor();
+
+ ShipStrategyType joinIn1 = joinNode.getInput1().getShipStrategy();
+ ShipStrategyType joinIn2 = joinNode.getInput2().getShipStrategy();
+
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn1);
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn2);
+ }
+
+ /**
+ * Tests join program with replicated data source behind map.
+ */
+ @Test
+ public void checkJoinWithReplicatedSourceInputBehindMap() {
+
+ ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+ env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+
+ ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
+ new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
+
+ DataSet<Tuple1<String>> source1 = env.createInput(rif, new TupleTypeInfo<Tuple1<String>>(BasicTypeInfo.STRING_TYPE_INFO));
+ DataSet<Tuple1<String>> source2 = env.readCsvFile("/some/otherpath").types(String.class);
+
+ DataSink<Tuple2<Tuple1<String>, Tuple1<String>>> out = source1
+ .map(new IdMap())
+ .join(source2).where("*").equalTo("*")
+ .writeAsText("/some/newpath");
+
+ JavaPlan plan = env.createProgramPlan();
+
+ // submit the plan to the compiler
+ OptimizedPlan oPlan = compileNoStats(plan);
+
+ // check the optimized Plan
+ // when join should have forward strategy on both sides
+ SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
+ DualInputPlanNode joinNode = (DualInputPlanNode) sinkNode.getPredecessor();
+
+ ShipStrategyType joinIn1 = joinNode.getInput1().getShipStrategy();
+ ShipStrategyType joinIn2 = joinNode.getInput2().getShipStrategy();
+
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn1);
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn2);
+ }
+
+ /**
+ * Tests join program with replicated data source behind filter.
+ */
+ @Test
+ public void checkJoinWithReplicatedSourceInputBehindFilter() {
+
+ ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+ env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+
+ ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
+ new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
+
+ DataSet<Tuple1<String>> source1 = env.createInput(rif, new TupleTypeInfo<Tuple1<String>>(BasicTypeInfo.STRING_TYPE_INFO));
+ DataSet<Tuple1<String>> source2 = env.readCsvFile("/some/otherpath").types(String.class);
+
+ DataSink<Tuple2<Tuple1<String>, Tuple1<String>>> out = source1
+ .filter(new NoFilter())
+ .join(source2).where("*").equalTo("*")
+ .writeAsText("/some/newpath");
+
+ JavaPlan plan = env.createProgramPlan();
+
+ // submit the plan to the compiler
+ OptimizedPlan oPlan = compileNoStats(plan);
+
+ // check the optimized Plan
+ // when join should have forward strategy on both sides
+ SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
+ DualInputPlanNode joinNode = (DualInputPlanNode) sinkNode.getPredecessor();
+
+ ShipStrategyType joinIn1 = joinNode.getInput1().getShipStrategy();
+ ShipStrategyType joinIn2 = joinNode.getInput2().getShipStrategy();
+
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn1);
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn2);
+ }
+
+ /**
+ * Tests join program with replicated data source behind flatMap.
+ */
+ @Test
+ public void checkJoinWithReplicatedSourceInputBehindFlatMap() {
+
+ ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+ env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+
+ ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
+ new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
+
+ DataSet<Tuple1<String>> source1 = env.createInput(rif, new TupleTypeInfo<Tuple1<String>>(BasicTypeInfo.STRING_TYPE_INFO));
+ DataSet<Tuple1<String>> source2 = env.readCsvFile("/some/otherpath").types(String.class);
+
+ DataSink<Tuple2<Tuple1<String>, Tuple1<String>>> out = source1
+ .flatMap(new IdFlatMap())
+ .join(source2).where("*").equalTo("*")
+ .writeAsText("/some/newpath");
+
+ JavaPlan plan = env.createProgramPlan();
+
+ // submit the plan to the compiler
+ OptimizedPlan oPlan = compileNoStats(plan);
+
+ // check the optimized Plan
+ // when join should have forward strategy on both sides
+ SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
+ DualInputPlanNode joinNode = (DualInputPlanNode) sinkNode.getPredecessor();
+
+ ShipStrategyType joinIn1 = joinNode.getInput1().getShipStrategy();
+ ShipStrategyType joinIn2 = joinNode.getInput2().getShipStrategy();
+
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn1);
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn2);
+ }
+
+ /**
+ * Tests join program with replicated data source behind map partition.
+ */
+ @Test
+ public void checkJoinWithReplicatedSourceInputBehindMapPartition() {
+
+ ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+ env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+
+ ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
+ new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
+
+ DataSet<Tuple1<String>> source1 = env.createInput(rif, new TupleTypeInfo<Tuple1<String>>(BasicTypeInfo.STRING_TYPE_INFO));
+ DataSet<Tuple1<String>> source2 = env.readCsvFile("/some/otherpath").types(String.class);
+
+ DataSink<Tuple2<Tuple1<String>, Tuple1<String>>> out = source1
+ .mapPartition(new IdPMap())
+ .join(source2).where("*").equalTo("*")
+ .writeAsText("/some/newpath");
+
+ JavaPlan plan = env.createProgramPlan();
+
+ // submit the plan to the compiler
+ OptimizedPlan oPlan = compileNoStats(plan);
+
+ // check the optimized Plan
+ // when join should have forward strategy on both sides
+ SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
+ DualInputPlanNode joinNode = (DualInputPlanNode) sinkNode.getPredecessor();
+
+ ShipStrategyType joinIn1 = joinNode.getInput1().getShipStrategy();
+ ShipStrategyType joinIn2 = joinNode.getInput2().getShipStrategy();
+
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn1);
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn2);
+ }
+
+ /**
+ * Tests join program with replicated data source behind multiple map ops.
+ */
+ @Test
+ public void checkJoinWithReplicatedSourceInputBehindMultiMaps() {
+
+ ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+ env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+
+ ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
+ new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
+
+ DataSet<Tuple1<String>> source1 = env.createInput(rif, new TupleTypeInfo<Tuple1<String>>(BasicTypeInfo.STRING_TYPE_INFO));
+ DataSet<Tuple1<String>> source2 = env.readCsvFile("/some/otherpath").types(String.class);
+
+ DataSink<Tuple2<Tuple1<String>, Tuple1<String>>> out = source1
+ .filter(new NoFilter())
+ .mapPartition(new IdPMap())
+ .flatMap(new IdFlatMap())
+ .map(new IdMap())
+ .join(source2).where("*").equalTo("*")
+ .writeAsText("/some/newpath");
+
+ JavaPlan plan = env.createProgramPlan();
+
+ // submit the plan to the compiler
+ OptimizedPlan oPlan = compileNoStats(plan);
+
+ // check the optimized Plan
+ // when join should have forward strategy on both sides
+ SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
+ DualInputPlanNode joinNode = (DualInputPlanNode) sinkNode.getPredecessor();
+
+ ShipStrategyType joinIn1 = joinNode.getInput1().getShipStrategy();
+ ShipStrategyType joinIn2 = joinNode.getInput2().getShipStrategy();
+
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn1);
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, joinIn2);
+ }
+
+ /**
+ * Tests cross program with replicated data source.
+ */
+ @Test
+ public void checkCrossWithReplicatedSourceInput() {
+
+ ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+ env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+
+ ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
+ new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
+
+ DataSet<Tuple1<String>> source1 = env.createInput(rif, new TupleTypeInfo<Tuple1<String>>(BasicTypeInfo.STRING_TYPE_INFO));
+ DataSet<Tuple1<String>> source2 = env.readCsvFile("/some/otherpath").types(String.class);
+
+ DataSink<Tuple2<Tuple1<String>, Tuple1<String>>> out = source1
+ .cross(source2)
+ .writeAsText("/some/newpath");
+
+ JavaPlan plan = env.createProgramPlan();
+
+ // submit the plan to the compiler
+ OptimizedPlan oPlan = compileNoStats(plan);
+
+ // check the optimized Plan
+ // when cross should have forward strategy on both sides
+ SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
+ DualInputPlanNode crossNode = (DualInputPlanNode) sinkNode.getPredecessor();
+
+ ShipStrategyType crossIn1 = crossNode.getInput1().getShipStrategy();
+ ShipStrategyType crossIn2 = crossNode.getInput2().getShipStrategy();
+
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, crossIn1);
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, crossIn2);
+ }
+
+ /**
+ * Tests cross program with replicated data source behind map and filter.
+ */
+ @Test
+ public void checkCrossWithReplicatedSourceInputBehindMap() {
+
+ ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+ env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+
+ ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
+ new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
+
+ DataSet<Tuple1<String>> source1 = env.createInput(rif, new TupleTypeInfo<Tuple1<String>>(BasicTypeInfo.STRING_TYPE_INFO));
+ DataSet<Tuple1<String>> source2 = env.readCsvFile("/some/otherpath").types(String.class);
+
+ DataSink<Tuple2<Tuple1<String>, Tuple1<String>>> out = source1
+ .map(new IdMap())
+ .filter(new NoFilter())
+ .cross(source2)
+ .writeAsText("/some/newpath");
+
+ JavaPlan plan = env.createProgramPlan();
+
+ // submit the plan to the compiler
+ OptimizedPlan oPlan = compileNoStats(plan);
+
+ // check the optimized Plan
+ // when cross should have forward strategy on both sides
+ SinkPlanNode sinkNode = oPlan.getDataSinks().iterator().next();
+ DualInputPlanNode crossNode = (DualInputPlanNode) sinkNode.getPredecessor();
+
+ ShipStrategyType crossIn1 = crossNode.getInput1().getShipStrategy();
+ ShipStrategyType crossIn2 = crossNode.getInput2().getShipStrategy();
+
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, crossIn1);
+ Assert.assertEquals("Invalid ship strategy for an operator.", ShipStrategyType.FORWARD, crossIn2);
+ }
+
+ /**
+ * Tests compiler fail for join program with replicated data source and changing DOP.
+ */
+ @Test(expected = CompilerException.class)
+ public void checkJoinWithReplicatedSourceInputChangingDOP() {
+
+ ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+ env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+
+ ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
+ new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
+
+ DataSet<Tuple1<String>> source1 = env.createInput(rif, new TupleTypeInfo<Tuple1<String>>(BasicTypeInfo.STRING_TYPE_INFO));
+ DataSet<Tuple1<String>> source2 = env.readCsvFile("/some/otherpath").types(String.class);
+
+ DataSink<Tuple2<Tuple1<String>, Tuple1<String>>> out = source1
+ .join(source2).where("*").equalTo("*").setParallelism(DEFAULT_PARALLELISM+2)
+ .writeAsText("/some/newpath");
+
+ JavaPlan plan = env.createProgramPlan();
+
+ // submit the plan to the compiler
+ OptimizedPlan oPlan = compileNoStats(plan);
+ }
+
+ /**
+ * Tests compiler fail for join program with replicated data source behind map and changing DOP.
+ */
+ @Test(expected = CompilerException.class)
+ public void checkJoinWithReplicatedSourceInputBehindMapChangingDOP() {
+
+ ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+ env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+
+ ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
+ new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
+
+ DataSet<Tuple1<String>> source1 = env.createInput(rif, new TupleTypeInfo<Tuple1<String>>(BasicTypeInfo.STRING_TYPE_INFO));
+ DataSet<Tuple1<String>> source2 = env.readCsvFile("/some/otherpath").types(String.class);
+
+ DataSink<Tuple2<Tuple1<String>, Tuple1<String>>> out = source1
+ .map(new IdMap()).setParallelism(DEFAULT_PARALLELISM+1)
+ .join(source2).where("*").equalTo("*")
+ .writeAsText("/some/newpath");
+
+ JavaPlan plan = env.createProgramPlan();
+
+ // submit the plan to the compiler
+ OptimizedPlan oPlan = compileNoStats(plan);
+
+ }
+
+ /**
+ * Tests compiler fail for join program with replicated data source behind reduce.
+ */
+ @Test(expected = CompilerException.class)
+ public void checkJoinWithReplicatedSourceInputBehindReduce() {
+ ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+ env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+
+ ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
+ new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
+
+ DataSet<Tuple1<String>> source1 = env.createInput(rif, new TupleTypeInfo<Tuple1<String>>(BasicTypeInfo.STRING_TYPE_INFO));
+ DataSet<Tuple1<String>> source2 = env.readCsvFile("/some/otherpath").types(String.class);
+
+ DataSink<Tuple2<Tuple1<String>, Tuple1<String>>> out = source1
+ .reduce(new LastReduce())
+ .join(source2).where("*").equalTo("*")
+ .writeAsText("/some/newpath");
+
+ JavaPlan plan = env.createProgramPlan();
+
+ // submit the plan to the compiler
+ OptimizedPlan oPlan = compileNoStats(plan);
+ }
+
+ /**
+ * Tests compiler fail for join program with replicated data source behind rebalance.
+ */
+ @Test(expected = CompilerException.class)
+ public void checkJoinWithReplicatedSourceInputBehindRebalance() {
+ ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
+ env.setDegreeOfParallelism(DEFAULT_PARALLELISM);
+
+ ReplicatingInputFormat<Tuple1<String>, FileInputSplit> rif =
+ new ReplicatingInputFormat<Tuple1<String>, FileInputSplit>(new CsvInputFormat<Tuple1<String>>(new Path("/some/path"), String.class));
+
+ DataSet<Tuple1<String>> source1 = env.createInput(rif, new TupleTypeInfo<Tuple1<String>>(BasicTypeInfo.STRING_TYPE_INFO));
+ DataSet<Tuple1<String>> source2 = env.readCsvFile("/some/otherpath").types(String.class);
+
+ DataSink<Tuple2<Tuple1<String>, Tuple1<String>>> out = source1
+ .rebalance()
+ .join(source2).where("*").equalTo("*")
+ .writeAsText("/some/newpath");
+
+ JavaPlan plan = env.createProgramPlan();
+
+ // submit the plan to the compiler
+ OptimizedPlan oPlan = compileNoStats(plan);
+ }
+
+
+ public static class IdMap<T> implements MapFunction<T,T> {
+
+ @Override
+ public T map(T value) throws Exception {
+ return value;
+ }
+ }
+
+ public static class NoFilter<T> implements FilterFunction<T> {
+
+ @Override
+ public boolean filter(T value) throws Exception {
+ return false;
+ }
+ }
+
+ public static class IdFlatMap<T> implements FlatMapFunction<T,T> {
+
+ @Override
+ public void flatMap(T value, Collector<T> out) throws Exception {
+ out.collect(value);
+ }
+ }
+
+ public static class IdPMap<T> implements MapPartitionFunction<T,T> {
+
+ @Override
+ public void mapPartition(Iterable<T> values, Collector<T> out) throws Exception {
+ for(T v : values) {
+ out.collect(v);
+ }
+ }
+ }
+
+ public static class LastReduce<T> implements ReduceFunction<T> {
+
+ @Override
+ public T reduce(T value1, T value2) throws Exception {
+ return value2;
+ }
+ }
+
+
+}
+
+
http://git-wip-us.apache.org/repos/asf/flink/blob/633b0d6a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java
new file mode 100644
index 0000000..1fe16bb
--- /dev/null
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java
@@ -0,0 +1,173 @@
+/**
+ * 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.optimizer;
+
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.operators.base.JoinOperatorBase;
+import org.apache.flink.api.common.operators.base.MapOperatorBase;
+import org.apache.flink.api.common.operators.base.ReduceOperatorBase;
+import org.apache.flink.api.common.operators.util.FieldSet;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.translation.JavaPlan;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.optimizer.dataproperties.GlobalProperties;
+import org.apache.flink.optimizer.dataproperties.LocalProperties;
+import org.apache.flink.optimizer.dataproperties.PartitioningProperty;
+import org.apache.flink.optimizer.plan.Channel;
+import org.apache.flink.optimizer.plan.DualInputPlanNode;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plan.PlanNode;
+import org.apache.flink.optimizer.plan.SingleInputPlanNode;
+import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
+import org.apache.flink.util.Visitor;
+import org.junit.Assert;
+import org.junit.Test;
+
+@SuppressWarnings("serial")
+public class SemanticPropertiesAPIToPlanTest extends CompilerTestBase {
+
+ @Test
+ public void forwardFieldsTestMapReduce() {
+ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ DataSet<Tuple3<Integer, Integer, Integer>> set = env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class);
+ set = set.map(new MockMapper()).withForwardedFields("*")
+ .groupBy(0)
+ .reduce(new MockReducer()).withForwardedFields("f0->f1")
+ .map(new MockMapper()).withForwardedFields("*")
+ .groupBy(1)
+ .reduce(new MockReducer()).withForwardedFields("*");
+
+ set.print();
+ JavaPlan plan = env.createProgramPlan();
+ OptimizedPlan oPlan = compileWithStats(plan);
+
+ oPlan.accept(new Visitor<PlanNode>() {
+ @Override
+ public boolean preVisit(PlanNode visitable) {
+ if (visitable instanceof SingleInputPlanNode && visitable.getProgramOperator() instanceof ReduceOperatorBase) {
+ for (Channel input: visitable.getInputs()) {
+ GlobalProperties gprops = visitable.getGlobalProperties();
+ LocalProperties lprops = visitable.getLocalProperties();
+
+ Assert.assertTrue("Reduce should just forward the input if it is already partitioned",
+ input.getShipStrategy() == ShipStrategyType.FORWARD);
+ Assert.assertTrue("Wrong GlobalProperties on Reducer",
+ gprops.isPartitionedOnFields(new FieldSet(1)));
+ Assert.assertTrue("Wrong GlobalProperties on Reducer",
+ gprops.getPartitioning() == PartitioningProperty.HASH_PARTITIONED);
+ Assert.assertTrue("Wrong LocalProperties on Reducer",
+ lprops.getGroupedFields().contains(1));
+ }
+ }
+ if (visitable instanceof SingleInputPlanNode && visitable.getProgramOperator() instanceof MapOperatorBase) {
+ for (Channel input: visitable.getInputs()) {
+ GlobalProperties gprops = visitable.getGlobalProperties();
+ LocalProperties lprops = visitable.getLocalProperties();
+
+ Assert.assertTrue("Map should just forward the input if it is already partitioned",
+ input.getShipStrategy() == ShipStrategyType.FORWARD);
+ Assert.assertTrue("Wrong GlobalProperties on Mapper",
+ gprops.isPartitionedOnFields(new FieldSet(1)));
+ Assert.assertTrue("Wrong GlobalProperties on Mapper",
+ gprops.getPartitioning() == PartitioningProperty.HASH_PARTITIONED);
+ Assert.assertTrue("Wrong LocalProperties on Mapper",
+ lprops.getGroupedFields().contains(1));
+ }
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public void postVisit(PlanNode visitable) {
+
+ }
+ });
+ }
+
+ @Test
+ public void forwardFieldsTestJoin() {
+ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ DataSet<Tuple3<Integer, Integer, Integer>> in1 = env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class);
+ DataSet<Tuple3<Integer, Integer, Integer>> in2 = env.readCsvFile(IN_FILE).types(Integer.class, Integer.class, Integer.class);
+ in1 = in1.map(new MockMapper()).withForwardedFields("*")
+ .groupBy(0)
+ .reduce(new MockReducer()).withForwardedFields("f0->f1");
+ in2 = in2.map(new MockMapper()).withForwardedFields("*")
+ .groupBy(1)
+ .reduce(new MockReducer()).withForwardedFields("f1->f2");
+ DataSet<Tuple3<Integer, Integer, Integer>> out = in1.join(in2).where(1).equalTo(2).with(new MockJoin());
+
+ out.print();
+ JavaPlan plan = env.createProgramPlan();
+ OptimizedPlan oPlan = compileWithStats(plan);
+
+ oPlan.accept(new Visitor<PlanNode>() {
+ @Override
+ public boolean preVisit(PlanNode visitable) {
+ if (visitable instanceof DualInputPlanNode && visitable.getProgramOperator() instanceof JoinOperatorBase) {
+ DualInputPlanNode node = ((DualInputPlanNode) visitable);
+
+ final Channel inConn1 = node.getInput1();
+ final Channel inConn2 = node.getInput2();
+
+ Assert.assertTrue("Join should just forward the input if it is already partitioned",
+ inConn1.getShipStrategy() == ShipStrategyType.FORWARD);
+ Assert.assertTrue("Join should just forward the input if it is already partitioned",
+ inConn2.getShipStrategy() == ShipStrategyType.FORWARD);
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public void postVisit(PlanNode visitable) {
+
+ }
+ });
+ }
+
+ public static class MockMapper implements MapFunction<Tuple3<Integer, Integer, Integer>, Tuple3<Integer, Integer, Integer>> {
+ @Override
+ public Tuple3<Integer, Integer, Integer> map(Tuple3<Integer, Integer, Integer> value) throws Exception {
+ return null;
+ }
+ }
+
+ public static class MockReducer implements ReduceFunction<Tuple3<Integer, Integer, Integer>> {
+ @Override
+ public Tuple3<Integer, Integer, Integer> reduce(Tuple3<Integer, Integer, Integer> value1, Tuple3<Integer, Integer, Integer> value2) throws Exception {
+ return null;
+ }
+ }
+
+ public static class MockJoin implements JoinFunction<Tuple3<Integer, Integer, Integer>,
+ Tuple3<Integer, Integer, Integer>, Tuple3<Integer, Integer, Integer>> {
+
+ @Override
+ public Tuple3<Integer, Integer, Integer> join(Tuple3<Integer, Integer, Integer> first, Tuple3<Integer, Integer, Integer> second) throws Exception {
+ return null;
+ }
+ }
+
+}
+
http://git-wip-us.apache.org/repos/asf/flink/blob/633b0d6a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SortPartialReuseTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SortPartialReuseTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/SortPartialReuseTest.java
new file mode 100644
index 0000000..40b54e0
--- /dev/null
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/SortPartialReuseTest.java
@@ -0,0 +1,130 @@
+/*
+ * 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.optimizer;
+
+import static org.junit.Assert.*;
+
+import org.junit.Test;
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plan.SingleInputPlanNode;
+import org.apache.flink.optimizer.plan.SinkPlanNode;
+import org.apache.flink.optimizer.testfunctions.IdentityGroupReducer;
+import org.apache.flink.optimizer.testfunctions.IdentityMapper;
+import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
+import org.apache.flink.runtime.operators.util.LocalStrategy;
+
+@SuppressWarnings("serial")
+public class SortPartialReuseTest extends CompilerTestBase {
+
+ @Test
+ public void testPartialPartitioningReuse() {
+ try {
+ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ @SuppressWarnings("unchecked")
+ DataSet<Tuple3<Long, Long, Long>> input = env.fromElements(new Tuple3<Long, Long, Long>(0L, 0L, 0L));
+
+ input
+ .partitionByHash(0)
+ .map(new IdentityMapper<Tuple3<Long,Long,Long>>()).withForwardedFields("0", "1", "2")
+
+ .groupBy(0, 1)
+ .reduceGroup(new IdentityGroupReducer<Tuple3<Long,Long,Long>>()).withForwardedFields("0", "1", "2")
+
+ .groupBy(0)
+ .reduceGroup(new IdentityGroupReducer<Tuple3<Long,Long,Long>>())
+
+ .print();
+
+ Plan p = env.createProgramPlan();
+ OptimizedPlan op = compileNoStats(p);
+
+ SinkPlanNode sink = op.getDataSinks().iterator().next();
+ SingleInputPlanNode reducer2 = (SingleInputPlanNode) sink.getInput().getSource();
+ SingleInputPlanNode reducer1 = (SingleInputPlanNode) reducer2.getInput().getSource();
+
+ assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy());
+
+ // should be locally forwarding, reusing sort and partitioning
+ assertEquals(ShipStrategyType.FORWARD, reducer2.getInput().getShipStrategy());
+ assertEquals(LocalStrategy.NONE, reducer2.getInput().getLocalStrategy());
+
+ assertEquals(ShipStrategyType.FORWARD, reducer1.getInput().getShipStrategy());
+ assertEquals(LocalStrategy.COMBININGSORT, reducer1.getInput().getLocalStrategy());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testCustomPartitioningNotReused() {
+ try {
+ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ @SuppressWarnings("unchecked")
+ DataSet<Tuple3<Long, Long, Long>> input = env.fromElements(new Tuple3<Long, Long, Long>(0L, 0L, 0L));
+
+ input
+ .partitionCustom(new Partitioner<Long>() {
+ @Override
+ public int partition(Long key, int numPartitions) { return 0; }
+ }, 0)
+ .map(new IdentityMapper<Tuple3<Long,Long,Long>>()).withForwardedFields("0", "1", "2")
+
+ .groupBy(0, 1)
+ .reduceGroup(new IdentityGroupReducer<Tuple3<Long,Long,Long>>()).withForwardedFields("0", "1", "2")
+
+ .groupBy(1)
+ .reduceGroup(new IdentityGroupReducer<Tuple3<Long,Long,Long>>())
+
+ .print();
+
+ Plan p = env.createProgramPlan();
+ OptimizedPlan op = compileNoStats(p);
+
+ SinkPlanNode sink = op.getDataSinks().iterator().next();
+ SingleInputPlanNode reducer2 = (SingleInputPlanNode) sink.getInput().getSource();
+ SingleInputPlanNode combiner = (SingleInputPlanNode) reducer2.getInput().getSource();
+ SingleInputPlanNode reducer1 = (SingleInputPlanNode) combiner.getInput().getSource();
+
+ assertEquals(ShipStrategyType.FORWARD, sink.getInput().getShipStrategy());
+
+ // should be locally forwarding, reusing sort and partitioning
+ assertEquals(ShipStrategyType.PARTITION_HASH, reducer2.getInput().getShipStrategy());
+ assertEquals(LocalStrategy.COMBININGSORT, reducer2.getInput().getLocalStrategy());
+
+ assertEquals(ShipStrategyType.FORWARD, combiner.getInput().getShipStrategy());
+ assertEquals(LocalStrategy.NONE, combiner.getInput().getLocalStrategy());
+
+ assertEquals(ShipStrategyType.FORWARD, reducer1.getInput().getShipStrategy());
+ assertEquals(LocalStrategy.COMBININGSORT, reducer1.getInput().getLocalStrategy());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/633b0d6a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionBetweenDynamicAndStaticPathTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionBetweenDynamicAndStaticPathTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionBetweenDynamicAndStaticPathTest.java
new file mode 100644
index 0000000..92b4fc5
--- /dev/null
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionBetweenDynamicAndStaticPathTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.optimizer;
+
+import static org.junit.Assert.*;
+
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.IterativeDataSet;
+import org.apache.flink.optimizer.plan.BinaryUnionPlanNode;
+import org.apache.flink.optimizer.plan.BulkIterationPlanNode;
+import org.apache.flink.optimizer.plan.Channel;
+import org.apache.flink.optimizer.plan.NAryUnionPlanNode;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plan.SingleInputPlanNode;
+import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.junit.Test;
+
+@SuppressWarnings("serial")
+public class UnionBetweenDynamicAndStaticPathTest extends CompilerTestBase {
+
+ @Test
+ public void testUnionStaticFirst() {
+ try {
+ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ DataSet<Long> input1 = env.generateSequence(1, 10);
+ DataSet<Long> input2 = env.generateSequence(1, 10);
+
+ IterativeDataSet<Long> iteration = input1.iterate(10);
+
+ DataSet<Long> result = iteration.closeWith(
+ input2.union(input2).union(iteration.union(iteration)));
+
+ result.print();
+ result.print();
+
+ Plan p = env.createProgramPlan();
+ OptimizedPlan op = compileNoStats(p);
+
+ assertEquals(2, op.getDataSinks().size());
+
+ BulkIterationPlanNode iterPlan = (BulkIterationPlanNode) op.getDataSinks().iterator().next().getInput().getSource();
+
+ SingleInputPlanNode noopNode = (SingleInputPlanNode) iterPlan.getRootOfStepFunction();
+ BinaryUnionPlanNode mixedUnion = (BinaryUnionPlanNode) noopNode.getInput().getSource();
+ NAryUnionPlanNode staticUnion = (NAryUnionPlanNode) mixedUnion.getInput1().getSource();
+ NAryUnionPlanNode dynamicUnion = (NAryUnionPlanNode) mixedUnion.getInput2().getSource();
+
+ assertTrue(mixedUnion.unionsStaticAndDynamicPath());
+ assertFalse(mixedUnion.getInput1().isOnDynamicPath());
+ assertTrue(mixedUnion.getInput2().isOnDynamicPath());
+ assertTrue(mixedUnion.getInput1().getTempMode().isCached());
+
+ for (Channel c : staticUnion.getInputs()) {
+ assertFalse(c.isOnDynamicPath());
+ }
+ for (Channel c : dynamicUnion.getInputs()) {
+ assertTrue(c.isOnDynamicPath());
+ }
+
+ assertEquals(0.5, iterPlan.getRelativeMemoryPerSubTask(), 0.0);
+ assertEquals(0.5, mixedUnion.getInput1().getRelativeTempMemory(), 0.0);
+ assertEquals(0.0, mixedUnion.getInput2().getRelativeTempMemory(), 0.0);
+
+ new JobGraphGenerator().compileJobGraph(op);
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testUnionStaticSecond() {
+ try {
+ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ DataSet<Long> input1 = env.generateSequence(1, 10);
+ DataSet<Long> input2 = env.generateSequence(1, 10);
+
+ IterativeDataSet<Long> iteration = input1.iterate(10);
+
+ DataSet<Long> iterResult = iteration
+ .closeWith(iteration.union(iteration).union(input2.union(input2)));
+
+ iterResult.print();
+ iterResult.print();
+
+
+ Plan p = env.createProgramPlan();
+ OptimizedPlan op = compileNoStats(p);
+
+ assertEquals(2, op.getDataSinks().size());
+
+ BulkIterationPlanNode iterPlan = (BulkIterationPlanNode) op.getDataSinks().iterator().next().getInput().getSource();
+
+ SingleInputPlanNode noopNode = (SingleInputPlanNode) iterPlan.getRootOfStepFunction();
+ BinaryUnionPlanNode mixedUnion = (BinaryUnionPlanNode) noopNode.getInput().getSource();
+ NAryUnionPlanNode staticUnion = (NAryUnionPlanNode) mixedUnion.getInput1().getSource();
+ NAryUnionPlanNode dynamicUnion = (NAryUnionPlanNode) mixedUnion.getInput2().getSource();
+
+ assertTrue(mixedUnion.unionsStaticAndDynamicPath());
+ assertFalse(mixedUnion.getInput1().isOnDynamicPath());
+ assertTrue(mixedUnion.getInput2().isOnDynamicPath());
+ assertTrue(mixedUnion.getInput1().getTempMode().isCached());
+
+ assertEquals(0.5, iterPlan.getRelativeMemoryPerSubTask(), 0.0);
+ assertEquals(0.5, mixedUnion.getInput1().getRelativeTempMemory(), 0.0);
+ assertEquals(0.0, mixedUnion.getInput2().getRelativeTempMemory(), 0.0);
+
+ for (Channel c : staticUnion.getInputs()) {
+ assertFalse(c.isOnDynamicPath());
+ }
+ for (Channel c : dynamicUnion.getInputs()) {
+ assertTrue(c.isOnDynamicPath());
+ }
+
+ new JobGraphGenerator().compileJobGraph(op);
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/633b0d6a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionPropertyPropagationTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionPropertyPropagationTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionPropertyPropagationTest.java
new file mode 100644
index 0000000..5d15ed8
--- /dev/null
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionPropertyPropagationTest.java
@@ -0,0 +1,186 @@
+/*
+ * 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.optimizer;
+
+import java.util.Iterator;
+
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.operators.base.FlatMapOperatorBase;
+import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase;
+import org.apache.flink.api.java.aggregation.Aggregations;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.java.record.operators.FileDataSink;
+import org.apache.flink.api.java.record.operators.FileDataSource;
+import org.apache.flink.api.java.record.operators.ReduceOperator;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
+import org.apache.flink.types.IntValue;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.Visitor;
+import org.junit.Assert;
+import org.junit.Test;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.optimizer.plan.Channel;
+import org.apache.flink.optimizer.plan.NAryUnionPlanNode;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plan.PlanNode;
+import org.apache.flink.optimizer.plan.SingleInputPlanNode;
+import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.apache.flink.optimizer.util.DummyInputFormat;
+import org.apache.flink.optimizer.util.DummyOutputFormat;
+import org.apache.flink.optimizer.util.IdentityReduce;
+
+
+@SuppressWarnings({"serial", "deprecation"})
+public class UnionPropertyPropagationTest extends CompilerTestBase {
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void testUnionPropertyOldApiPropagation() {
+ // construct the plan
+
+ FileDataSource sourceA = new FileDataSource(new DummyInputFormat(), IN_FILE);
+ FileDataSource sourceB = new FileDataSource(new DummyInputFormat(), IN_FILE);
+
+ ReduceOperator redA = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0)
+ .input(sourceA)
+ .build();
+ ReduceOperator redB = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0)
+ .input(sourceB)
+ .build();
+
+ ReduceOperator globalRed = ReduceOperator.builder(new IdentityReduce(), IntValue.class, 0).build();
+ globalRed.addInput(redA);
+ globalRed.addInput(redB);
+
+ FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, globalRed);
+
+ // return the plan
+ Plan plan = new Plan(sink, "Union Property Propagation");
+
+ OptimizedPlan oPlan = compileNoStats(plan);
+
+ JobGraphGenerator jobGen = new JobGraphGenerator();
+
+ // Compile plan to verify that no error is thrown
+ jobGen.compileJobGraph(oPlan);
+
+ oPlan.accept(new Visitor<PlanNode>() {
+
+ @Override
+ public boolean preVisit(PlanNode visitable) {
+ if (visitable instanceof SingleInputPlanNode && visitable.getProgramOperator() instanceof ReduceOperator) {
+ for (Channel inConn : visitable.getInputs()) {
+ Assert.assertTrue("Reduce should just forward the input if it is already partitioned",
+ inConn.getShipStrategy() == ShipStrategyType.FORWARD);
+ }
+ //just check latest ReduceNode
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public void postVisit(PlanNode visitable) {
+ // DO NOTHING
+ }
+ });
+ }
+
+ @Test
+ public void testUnionNewApiAssembly() {
+ final int NUM_INPUTS = 4;
+
+ // construct the plan it will be multiple flat maps, all unioned
+ // and the "unioned" dataSet will be grouped
+ final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ DataSet<String> source = env.readTextFile(IN_FILE);
+ DataSet<Tuple2<String, Integer>> lastUnion = source.flatMap(new DummyFlatMap());
+
+ for (int i = 1; i< NUM_INPUTS; i++){
+ lastUnion = lastUnion.union(source.flatMap(new DummyFlatMap()));
+ }
+
+ DataSet<Tuple2<String, Integer>> result = lastUnion.groupBy(0).aggregate(Aggregations.SUM, 1);
+ result.writeAsText(OUT_FILE);
+
+ // return the plan
+ Plan plan = env.createProgramPlan("Test union on new java-api");
+ OptimizedPlan oPlan = compileNoStats(plan);
+ JobGraphGenerator jobGen = new JobGraphGenerator();
+
+ // Compile plan to verify that no error is thrown
+ jobGen.compileJobGraph(oPlan);
+
+ oPlan.accept(new Visitor<PlanNode>() {
+
+ @Override
+ public boolean preVisit(PlanNode visitable) {
+
+ /* Test on the union output connections
+ * It must be under the GroupOperator and the strategy should be forward
+ */
+ if (visitable instanceof SingleInputPlanNode && visitable.getProgramOperator() instanceof GroupReduceOperatorBase){
+ final Channel inConn = ((SingleInputPlanNode) visitable).getInput();
+ Assert.assertTrue("Union should just forward the Partitioning",
+ inConn.getShipStrategy() == ShipStrategyType.FORWARD );
+ Assert.assertTrue("Union Node should be under Group operator",
+ inConn.getSource() instanceof NAryUnionPlanNode );
+ }
+
+ /* Test on the union input connections
+ * Must be NUM_INPUTS input connections, all FlatMapOperators with a own partitioning strategy(propably hash)
+ */
+ if (visitable instanceof NAryUnionPlanNode) {
+ int numberInputs = 0;
+ for (Iterator<Channel> inputs = visitable.getInputs().iterator(); inputs.hasNext(); numberInputs++) {
+ final Channel inConn = inputs.next();
+ PlanNode inNode = inConn.getSource();
+ Assert.assertTrue("Input of Union should be FlatMapOperators",
+ inNode.getProgramOperator() instanceof FlatMapOperatorBase);
+ Assert.assertTrue("Shipment strategy under union should partition the data",
+ inConn.getShipStrategy() == ShipStrategyType.PARTITION_HASH);
+ }
+
+ Assert.assertTrue("NAryUnion should have " + NUM_INPUTS + " inputs", numberInputs == NUM_INPUTS);
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public void postVisit(PlanNode visitable) {
+ // DO NOTHING
+ }
+ });
+ }
+
+ public static final class DummyFlatMap extends RichFlatMapFunction<String, Tuple2<String, Integer>> {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void flatMap(String value, Collector<Tuple2<String, Integer>> out) {
+ out.collect(new Tuple2<String, Integer>(value, 0));
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/633b0d6a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java
new file mode 100644
index 0000000..1e4124c
--- /dev/null
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/UnionReplacementTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.optimizer;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.junit.Test;
+
+import static org.junit.Assert.fail;
+
+@SuppressWarnings("serial")
+public class UnionReplacementTest extends CompilerTestBase {
+
+ @Test
+ public void testUnionReplacement() {
+ try {
+ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ DataSet<String> input1 = env.fromElements("test1");
+ DataSet<String> input2 = env.fromElements("test2");
+
+ DataSet<String> union = input1.union(input2);
+
+ union.print();
+ union.print();
+
+ Plan plan = env.createProgramPlan();
+ OptimizedPlan oPlan = compileNoStats(plan);
+ JobGraphGenerator jobGen = new JobGraphGenerator();
+ jobGen.compileJobGraph(oPlan);
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/633b0d6a/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationCornerCasesTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationCornerCasesTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationCornerCasesTest.java
new file mode 100644
index 0000000..80c0bda
--- /dev/null
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationCornerCasesTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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.optimizer;
+
+import static org.junit.Assert.*;
+
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.DeltaIteration;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plan.WorksetIterationPlanNode;
+import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.junit.Test;
+
+@SuppressWarnings("serial")
+public class WorksetIterationCornerCasesTest extends CompilerTestBase {
+
+ @Test
+ public void testWorksetIterationNotDependingOnSolutionSet() {
+ try {
+ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ DataSet<Tuple2<Long, Long>> input = env.generateSequence(1, 100).map(new Duplicator<Long>());
+
+ DeltaIteration<Tuple2<Long, Long>, Tuple2<Long, Long>> iteration = input.iterateDelta(input, 100, 1);
+
+ DataSet<Tuple2<Long, Long>> iterEnd = iteration.getWorkset().map(new TestMapper<Tuple2<Long,Long>>());
+ iteration.closeWith(iterEnd, iterEnd).print();
+
+ Plan p = env.createProgramPlan();
+ OptimizedPlan op = compileNoStats(p);
+
+ WorksetIterationPlanNode wipn = (WorksetIterationPlanNode) op.getDataSinks().iterator().next().getInput().getSource();
+ assertTrue(wipn.getSolutionSetPlanNode().getOutgoingChannels().isEmpty());
+
+ JobGraphGenerator jgg = new JobGraphGenerator();
+ jgg.compileJobGraph(op);
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ private static final class Duplicator<T> implements MapFunction<T, Tuple2<T, T>> {
+ @Override
+ public Tuple2<T, T> map(T value) {
+ return new Tuple2<T, T>(value, value);
+ }
+ }
+
+ private static final class TestMapper<T> implements MapFunction<T, T> {
+ @Override
+ public T map(T value) {
+ return value;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/633b0d6a/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationsRecordApiCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationsRecordApiCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationsRecordApiCompilerTest.java
new file mode 100644
index 0000000..6e7c0a3
--- /dev/null
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/WorksetIterationsRecordApiCompilerTest.java
@@ -0,0 +1,247 @@
+/*
+ * 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.optimizer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.operators.util.FieldList;
+import org.apache.flink.api.java.record.operators.DeltaIteration;
+import org.apache.flink.api.java.record.operators.FileDataSink;
+import org.apache.flink.api.java.record.operators.FileDataSource;
+import org.apache.flink.api.java.record.operators.JoinOperator;
+import org.apache.flink.api.java.record.operators.MapOperator;
+import org.apache.flink.api.java.record.operators.ReduceOperator;
+import org.apache.flink.optimizer.plan.DualInputPlanNode;
+import org.apache.flink.optimizer.plan.OptimizedPlan;
+import org.apache.flink.optimizer.plan.SingleInputPlanNode;
+import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
+import org.apache.flink.optimizer.util.DummyInputFormat;
+import org.apache.flink.optimizer.util.DummyMatchStub;
+import org.apache.flink.optimizer.util.DummyNonPreservingMatchStub;
+import org.apache.flink.optimizer.util.DummyOutputFormat;
+import org.apache.flink.optimizer.util.IdentityMap;
+import org.apache.flink.optimizer.util.IdentityReduce;
+import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
+import org.apache.flink.types.LongValue;
+import org.junit.Test;
+
+
+/**
+* Tests that validate optimizer choices when using operators that are requesting certain specific execution
+* strategies.
+*/
+@SuppressWarnings("deprecation")
+public class WorksetIterationsRecordApiCompilerTest extends CompilerTestBase {
+
+ private static final long serialVersionUID = 1L;
+
+ private static final String ITERATION_NAME = "Test Workset Iteration";
+ private static final String JOIN_WITH_INVARIANT_NAME = "Test Join Invariant";
+ private static final String JOIN_WITH_SOLUTION_SET = "Test Join SolutionSet";
+ private static final String NEXT_WORKSET_REDUCER_NAME = "Test Reduce Workset";
+ private static final String SOLUTION_DELTA_MAPPER_NAME = "Test Map Delta";
+
+ private final FieldList list0 = new FieldList(0);
+
+ @Test
+ public void testRecordApiWithDeferredSoltionSetUpdateWithMapper() {
+ Plan plan = getRecordTestPlan(false, true);
+
+ OptimizedPlan oPlan;
+ try {
+ oPlan = compileNoStats(plan);
+ } catch(CompilerException ce) {
+ ce.printStackTrace();
+ fail("The pact compiler is unable to compile this plan correctly.");
+ return; // silence the compiler
+ }
+
+ OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan);
+ DualInputPlanNode joinWithInvariantNode = resolver.getNode(JOIN_WITH_INVARIANT_NAME);
+ DualInputPlanNode joinWithSolutionSetNode = resolver.getNode(JOIN_WITH_SOLUTION_SET);
+ SingleInputPlanNode worksetReducer = resolver.getNode(NEXT_WORKSET_REDUCER_NAME);
+ SingleInputPlanNode deltaMapper = resolver.getNode(SOLUTION_DELTA_MAPPER_NAME);
+
+ // iteration preserves partitioning in reducer, so the first partitioning is out of the loop,
+ // the in-loop partitioning is before the final reducer
+
+ // verify joinWithInvariant
+ assertEquals(ShipStrategyType.FORWARD, joinWithInvariantNode.getInput1().getShipStrategy());
+ assertEquals(ShipStrategyType.PARTITION_HASH, joinWithInvariantNode.getInput2().getShipStrategy());
+ assertEquals(list0, joinWithInvariantNode.getKeysForInput1());
+ assertEquals(list0, joinWithInvariantNode.getKeysForInput2());
+
+ // verify joinWithSolutionSet
+ assertEquals(ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput1().getShipStrategy());
+ assertEquals(ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput2().getShipStrategy());
+
+ // verify reducer
+ assertEquals(ShipStrategyType.PARTITION_HASH, worksetReducer.getInput().getShipStrategy());
+ assertEquals(list0, worksetReducer.getKeys(0));
+
+ // currently, the system may partition before or after the mapper
+ ShipStrategyType ss1 = deltaMapper.getInput().getShipStrategy();
+ ShipStrategyType ss2 = deltaMapper.getOutgoingChannels().get(0).getShipStrategy();
+
+ assertTrue( (ss1 == ShipStrategyType.FORWARD && ss2 == ShipStrategyType.PARTITION_HASH) ||
+ (ss2 == ShipStrategyType.FORWARD && ss1 == ShipStrategyType.PARTITION_HASH) );
+
+ new JobGraphGenerator().compileJobGraph(oPlan);
+ }
+
+ @Test
+ public void testRecordApiWithDeferredSoltionSetUpdateWithNonPreservingJoin() {
+ Plan plan = getRecordTestPlan(false, false);
+
+ OptimizedPlan oPlan;
+ try {
+ oPlan = compileNoStats(plan);
+ } catch(CompilerException ce) {
+ ce.printStackTrace();
+ fail("The pact compiler is unable to compile this plan correctly.");
+ return; // silence the compiler
+ }
+
+ OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan);
+ DualInputPlanNode joinWithInvariantNode = resolver.getNode(JOIN_WITH_INVARIANT_NAME);
+ DualInputPlanNode joinWithSolutionSetNode = resolver.getNode(JOIN_WITH_SOLUTION_SET);
+ SingleInputPlanNode worksetReducer = resolver.getNode(NEXT_WORKSET_REDUCER_NAME);
+
+ // iteration preserves partitioning in reducer, so the first partitioning is out of the loop,
+ // the in-loop partitioning is before the final reducer
+
+ // verify joinWithInvariant
+ assertEquals(ShipStrategyType.FORWARD, joinWithInvariantNode.getInput1().getShipStrategy());
+ assertEquals(ShipStrategyType.PARTITION_HASH, joinWithInvariantNode.getInput2().getShipStrategy());
+ assertEquals(list0, joinWithInvariantNode.getKeysForInput1());
+ assertEquals(list0, joinWithInvariantNode.getKeysForInput2());
+
+ // verify joinWithSolutionSet
+ assertEquals(ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput1().getShipStrategy());
+ assertEquals(ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput2().getShipStrategy());
+
+ // verify reducer
+ assertEquals(ShipStrategyType.PARTITION_HASH, worksetReducer.getInput().getShipStrategy());
+ assertEquals(list0, worksetReducer.getKeys(0));
+
+
+ // verify solution delta
+ assertEquals(2, joinWithSolutionSetNode.getOutgoingChannels().size());
+ assertEquals(ShipStrategyType.PARTITION_HASH, joinWithSolutionSetNode.getOutgoingChannels().get(0).getShipStrategy());
+ assertEquals(ShipStrategyType.PARTITION_HASH, joinWithSolutionSetNode.getOutgoingChannels().get(1).getShipStrategy());
+
+ new JobGraphGenerator().compileJobGraph(oPlan);
+ }
+
+ @Test
+ public void testRecordApiWithDirectSoltionSetUpdate() {
+ Plan plan = getRecordTestPlan(true, false);
+
+ OptimizedPlan oPlan;
+ try {
+ oPlan = compileNoStats(plan);
+ } catch(CompilerException ce) {
+ ce.printStackTrace();
+ fail("The pact compiler is unable to compile this plan correctly.");
+ return; // silence the compiler
+ }
+
+ OptimizerPlanNodeResolver resolver = getOptimizerPlanNodeResolver(oPlan);
+ DualInputPlanNode joinWithInvariantNode = resolver.getNode(JOIN_WITH_INVARIANT_NAME);
+ DualInputPlanNode joinWithSolutionSetNode = resolver.getNode(JOIN_WITH_SOLUTION_SET);
+ SingleInputPlanNode worksetReducer = resolver.getNode(NEXT_WORKSET_REDUCER_NAME);
+
+ // iteration preserves partitioning in reducer, so the first partitioning is out of the loop,
+ // the in-loop partitioning is before the final reducer
+
+ // verify joinWithInvariant
+ assertEquals(ShipStrategyType.FORWARD, joinWithInvariantNode.getInput1().getShipStrategy());
+ assertEquals(ShipStrategyType.PARTITION_HASH, joinWithInvariantNode.getInput2().getShipStrategy());
+ assertEquals(list0, joinWithInvariantNode.getKeysForInput1());
+ assertEquals(list0, joinWithInvariantNode.getKeysForInput2());
+
+ // verify joinWithSolutionSet
+ assertEquals(ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput1().getShipStrategy());
+ assertEquals(ShipStrategyType.FORWARD, joinWithSolutionSetNode.getInput2().getShipStrategy());
+
+ // verify reducer
+ assertEquals(ShipStrategyType.FORWARD, worksetReducer.getInput().getShipStrategy());
+ assertEquals(list0, worksetReducer.getKeys(0));
+
+
+ // verify solution delta
+ assertEquals(1, joinWithSolutionSetNode.getOutgoingChannels().size());
+ assertEquals(ShipStrategyType.FORWARD, joinWithSolutionSetNode.getOutgoingChannels().get(0).getShipStrategy());
+
+ new JobGraphGenerator().compileJobGraph(oPlan);
+ }
+
+ private Plan getRecordTestPlan(boolean joinPreservesSolutionSet, boolean mapBeforeSolutionDelta) {
+ FileDataSource solutionSetInput = new FileDataSource(new DummyInputFormat(), IN_FILE, "Solution Set");
+ FileDataSource worksetInput = new FileDataSource(new DummyInputFormat(), IN_FILE, "Workset");
+
+ FileDataSource invariantInput = new FileDataSource(new DummyInputFormat(), IN_FILE, "Invariant Input");
+
+ DeltaIteration iteration = new DeltaIteration(0, ITERATION_NAME);
+ iteration.setInitialSolutionSet(solutionSetInput);
+ iteration.setInitialWorkset(worksetInput);
+ iteration.setMaximumNumberOfIterations(100);
+
+ JoinOperator joinWithInvariant = JoinOperator.builder(new DummyMatchStub(), LongValue.class, 0, 0)
+ .input1(iteration.getWorkset())
+ .input2(invariantInput)
+ .name(JOIN_WITH_INVARIANT_NAME)
+ .build();
+
+ JoinOperator joinWithSolutionSet = JoinOperator.builder(
+ joinPreservesSolutionSet ? new DummyMatchStub() : new DummyNonPreservingMatchStub(), LongValue.class, 0, 0)
+ .input1(iteration.getSolutionSet())
+ .input2(joinWithInvariant)
+ .name(JOIN_WITH_SOLUTION_SET)
+ .build();
+
+ ReduceOperator nextWorkset = ReduceOperator.builder(new IdentityReduce(), LongValue.class, 0)
+ .input(joinWithSolutionSet)
+ .name(NEXT_WORKSET_REDUCER_NAME)
+ .build();
+
+ if (mapBeforeSolutionDelta) {
+ MapOperator mapper = MapOperator.builder(new IdentityMap())
+ .input(joinWithSolutionSet)
+ .name(SOLUTION_DELTA_MAPPER_NAME)
+ .build();
+ iteration.setSolutionSetDelta(mapper);
+ } else {
+ iteration.setSolutionSetDelta(joinWithSolutionSet);
+ }
+
+ iteration.setNextWorkset(nextWorkset);
+
+ FileDataSink sink = new FileDataSink(new DummyOutputFormat(), OUT_FILE, iteration, "Sink");
+
+ Plan plan = new Plan(sink);
+ plan.setDefaultParallelism(DEFAULT_PARALLELISM);
+ return plan;
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/flink/blob/633b0d6a/flink-optimizer/src/test/java/org/apache/flink/optimizer/costs/DefaultCostEstimatorTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/costs/DefaultCostEstimatorTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/costs/DefaultCostEstimatorTest.java
new file mode 100644
index 0000000..af03eec
--- /dev/null
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/costs/DefaultCostEstimatorTest.java
@@ -0,0 +1,428 @@
+/*
+ * 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.optimizer.costs;
+
+import static org.junit.Assert.*;
+
+import org.apache.flink.optimizer.dag.EstimateProvider;
+import org.junit.Test;
+
+/**
+ * Tests for the cost formulas in the {@link DefaultCostEstimator}. Most of the tests establish relative
+ * relationships.
+ */
+public class DefaultCostEstimatorTest {
+
+ // estimates
+
+ private static final long SMALL_DATA_SIZE = 10000;
+ private static final long SMALL_RECORD_COUNT = 100;
+
+ private static final long MEDIUM_DATA_SIZE = 500000000L;
+ private static final long MEDIUM_RECORD_COUNT = 500000L;
+
+ private static final long BIG_DATA_SIZE = 100000000000L;
+ private static final long BIG_RECORD_COUNT = 100000000L;
+
+ private static final EstimateProvider UNKNOWN_ESTIMATES = new UnknownEstimates();
+ private static final EstimateProvider ZERO_ESTIMATES = new Estimates(0, 0);
+ private static final EstimateProvider SMALL_ESTIMATES = new Estimates(SMALL_DATA_SIZE, SMALL_RECORD_COUNT);
+ private static final EstimateProvider MEDIUM_ESTIMATES = new Estimates(MEDIUM_DATA_SIZE, MEDIUM_RECORD_COUNT);
+ private static final EstimateProvider BIG_ESTIMATES = new Estimates(BIG_DATA_SIZE, BIG_RECORD_COUNT);
+
+ private final CostEstimator costEstimator = new DefaultCostEstimator();
+
+ // --------------------------------------------------------------------------------------------
+
+ @Test
+ public void testShipStrategiesIsolated() {
+ testShipStrategiesIsolated(UNKNOWN_ESTIMATES, 1);
+ testShipStrategiesIsolated(UNKNOWN_ESTIMATES, 10);
+ testShipStrategiesIsolated(ZERO_ESTIMATES, 1);
+ testShipStrategiesIsolated(ZERO_ESTIMATES, 10);
+ testShipStrategiesIsolated(SMALL_ESTIMATES, 1);
+ testShipStrategiesIsolated(SMALL_ESTIMATES, 10);
+ testShipStrategiesIsolated(BIG_ESTIMATES, 1);
+ testShipStrategiesIsolated(BIG_ESTIMATES, 10);
+ }
+
+ private void testShipStrategiesIsolated(EstimateProvider estimates, int targetParallelism) {
+ Costs random = new Costs();
+ costEstimator.addRandomPartitioningCost(estimates, random);
+
+ Costs hash = new Costs();
+ costEstimator.addHashPartitioningCost(estimates, hash);
+
+ Costs range = new Costs();
+ costEstimator.addRangePartitionCost(estimates, range);
+
+ Costs broadcast = new Costs();
+ costEstimator.addBroadcastCost(estimates, targetParallelism, broadcast);
+
+ int randomVsHash = random.compareTo(hash);
+ int hashVsRange = hash.compareTo(range);
+ int hashVsBroadcast = hash.compareTo(broadcast);
+ int rangeVsBroadcast = range.compareTo(broadcast);
+
+ // repartition random is at most as expensive as hash partitioning
+ assertTrue(randomVsHash <= 0);
+
+ // range partitioning is always more expensive than hash partitioning
+ assertTrue(hashVsRange < 0);
+
+ // broadcasting is always more expensive than hash partitioning
+ if (targetParallelism > 1) {
+ assertTrue(hashVsBroadcast < 0);
+ } else {
+ assertTrue(hashVsBroadcast <= 0);
+ }
+
+ // range partitioning is not more expensive than broadcasting
+ if (targetParallelism > 1) {
+ assertTrue(rangeVsBroadcast < 0);
+ }
+ }
+
+ // --------------------------------------------------------------------------------------------
+
+ @Test
+ public void testShipStrategyCombinationsPlain() {
+ Costs hashBothSmall = new Costs();
+ Costs hashSmallAndLarge = new Costs();
+ Costs hashBothLarge = new Costs();
+
+ Costs hashSmallBcLarge10 = new Costs();
+ Costs hashLargeBcSmall10 = new Costs();
+
+ Costs hashSmallBcLarge1000 = new Costs();
+ Costs hashLargeBcSmall1000 = new Costs();
+
+ Costs forwardSmallBcLarge10 = new Costs();
+ Costs forwardLargeBcSmall10 = new Costs();
+
+ Costs forwardSmallBcLarge1000 = new Costs();
+ Costs forwardLargeBcSmall1000 = new Costs();
+
+ costEstimator.addHashPartitioningCost(MEDIUM_ESTIMATES, hashBothSmall);
+ costEstimator.addHashPartitioningCost(MEDIUM_ESTIMATES, hashBothSmall);
+
+ costEstimator.addHashPartitioningCost(MEDIUM_ESTIMATES, hashSmallAndLarge);
+ costEstimator.addHashPartitioningCost(BIG_ESTIMATES, hashSmallAndLarge);
+
+ costEstimator.addHashPartitioningCost(BIG_ESTIMATES, hashBothLarge);
+ costEstimator.addHashPartitioningCost(BIG_ESTIMATES, hashBothLarge);
+
+ costEstimator.addHashPartitioningCost(MEDIUM_ESTIMATES, hashSmallBcLarge10);
+ costEstimator.addBroadcastCost(BIG_ESTIMATES, 10, hashSmallBcLarge10);
+
+ costEstimator.addHashPartitioningCost(BIG_ESTIMATES, hashLargeBcSmall10);
+ costEstimator.addBroadcastCost(MEDIUM_ESTIMATES, 10, hashLargeBcSmall10);
+
+ costEstimator.addHashPartitioningCost(MEDIUM_ESTIMATES, hashSmallBcLarge1000);
+ costEstimator.addBroadcastCost(BIG_ESTIMATES, 1000, hashSmallBcLarge1000);
+
+ costEstimator.addHashPartitioningCost(BIG_ESTIMATES, hashLargeBcSmall1000);
+ costEstimator.addBroadcastCost(MEDIUM_ESTIMATES, 1000, hashLargeBcSmall1000);
+
+ costEstimator.addBroadcastCost(BIG_ESTIMATES, 10, forwardSmallBcLarge10);
+
+ costEstimator.addBroadcastCost(MEDIUM_ESTIMATES, 10, forwardLargeBcSmall10);
+
+ costEstimator.addBroadcastCost(BIG_ESTIMATES, 1000, forwardSmallBcLarge1000);
+
+ costEstimator.addBroadcastCost(MEDIUM_ESTIMATES, 1000, forwardLargeBcSmall1000);
+
+ // hash cost is roughly monotonous
+ assertTrue(hashBothSmall.compareTo(hashSmallAndLarge) < 0);
+ assertTrue(hashSmallAndLarge.compareTo(hashBothLarge) < 0);
+
+ // broadcast the smaller is better
+ assertTrue(hashLargeBcSmall10.compareTo(hashSmallBcLarge10) < 0);
+ assertTrue(forwardLargeBcSmall10.compareTo(forwardSmallBcLarge10) < 0);
+ assertTrue(hashLargeBcSmall1000.compareTo(hashSmallBcLarge1000) < 0);
+ assertTrue(forwardLargeBcSmall1000.compareTo(forwardSmallBcLarge1000) < 0);
+
+ // broadcasting small and forwarding large is better than partition both, given size difference
+ assertTrue(forwardLargeBcSmall10.compareTo(hashSmallAndLarge) < 0);
+
+ // broadcasting too far is expensive again
+ assertTrue(forwardLargeBcSmall1000.compareTo(hashSmallAndLarge) > 0);
+
+ // assert weight is respected
+ assertTrue(hashSmallBcLarge10.compareTo(hashSmallBcLarge1000) < 0);
+ assertTrue(hashLargeBcSmall10.compareTo(hashLargeBcSmall1000) < 0);
+ assertTrue(forwardSmallBcLarge10.compareTo(forwardSmallBcLarge1000) < 0);
+ assertTrue(forwardLargeBcSmall10.compareTo(forwardLargeBcSmall1000) < 0);
+
+ // forward versus hash
+ assertTrue(forwardSmallBcLarge10.compareTo(hashSmallBcLarge10) < 0);
+ assertTrue(forwardSmallBcLarge1000.compareTo(hashSmallBcLarge1000) < 0);
+ assertTrue(forwardLargeBcSmall10.compareTo(hashLargeBcSmall10) < 0);
+ assertTrue(forwardLargeBcSmall1000.compareTo(hashLargeBcSmall1000) < 0);
+ }
+
+ // --------------------------------------------------------------------------------------------
+
+ @Test
+ public void testShipStrategyCombinationsWithUnknowns() {
+ testShipStrategyCombinationsWithUnknowns(UNKNOWN_ESTIMATES);
+ testShipStrategyCombinationsWithUnknowns(ZERO_ESTIMATES);
+ testShipStrategyCombinationsWithUnknowns(SMALL_ESTIMATES);
+ testShipStrategyCombinationsWithUnknowns(MEDIUM_ESTIMATES);
+ testShipStrategyCombinationsWithUnknowns(BIG_ESTIMATES);
+ }
+
+ private void testShipStrategyCombinationsWithUnknowns(EstimateProvider knownEstimates) {
+ Costs hashBoth = new Costs();
+ Costs bcKnown10 = new Costs();
+ Costs bcUnknown10 = new Costs();
+ Costs bcKnown1000 = new Costs();
+ Costs bcUnknown1000 = new Costs();
+
+ costEstimator.addHashPartitioningCost(knownEstimates, hashBoth);
+ costEstimator.addHashPartitioningCost(UNKNOWN_ESTIMATES, hashBoth);
+
+ costEstimator.addBroadcastCost(knownEstimates, 10, bcKnown10);
+
+ costEstimator.addBroadcastCost(UNKNOWN_ESTIMATES, 10, bcUnknown10);
+
+ costEstimator.addBroadcastCost(knownEstimates, 1000, bcKnown1000);
+
+ costEstimator.addBroadcastCost(UNKNOWN_ESTIMATES, 1000, bcUnknown1000);
+
+ // if we do not know one of them, hashing both should be cheaper than anything
+ assertTrue(hashBoth.compareTo(bcKnown10) < 0);
+ assertTrue(hashBoth.compareTo(bcUnknown10) < 0);
+ assertTrue(hashBoth.compareTo(bcKnown1000) < 0);
+ assertTrue(hashBoth.compareTo(bcUnknown1000) < 0);
+
+ // there should be no bias in broadcasting a known or unknown size input
+ assertTrue(bcKnown10.compareTo(bcUnknown10) == 0);
+ assertTrue(bcKnown1000.compareTo(bcUnknown1000) == 0);
+
+ // replication factor does matter
+ assertTrue(bcKnown10.compareTo(bcKnown1000) < 0);
+ assertTrue(bcUnknown10.compareTo(bcUnknown1000) < 0);
+ }
+
+ // --------------------------------------------------------------------------------------------
+
+ @Test
+ public void testJoinCostFormulasPlain() {
+
+ // hash join costs
+
+ Costs hashBothSmall = new Costs();
+ Costs hashBothLarge = new Costs();
+ Costs hashSmallBuild = new Costs();
+ Costs hashLargeBuild = new Costs();
+
+ costEstimator.addHybridHashCosts(SMALL_ESTIMATES, BIG_ESTIMATES, hashSmallBuild, 1);
+ costEstimator.addHybridHashCosts(BIG_ESTIMATES, SMALL_ESTIMATES, hashLargeBuild, 1);
+ costEstimator.addHybridHashCosts(SMALL_ESTIMATES, SMALL_ESTIMATES, hashBothSmall, 1);
+ costEstimator.addHybridHashCosts(BIG_ESTIMATES, BIG_ESTIMATES, hashBothLarge, 1);
+
+ assertTrue(hashBothSmall.compareTo(hashSmallBuild) < 0);
+ assertTrue(hashSmallBuild.compareTo(hashLargeBuild) < 0);
+ assertTrue(hashLargeBuild.compareTo(hashBothLarge) < 0);
+
+ // merge join costs
+
+ Costs mergeBothSmall = new Costs();
+ Costs mergeBothLarge = new Costs();
+ Costs mergeSmallFirst = new Costs();
+ Costs mergeSmallSecond = new Costs();
+
+ costEstimator.addLocalSortCost(SMALL_ESTIMATES, mergeSmallFirst);
+ costEstimator.addLocalSortCost(BIG_ESTIMATES, mergeSmallFirst);
+ costEstimator.addLocalMergeCost(SMALL_ESTIMATES, BIG_ESTIMATES, mergeSmallFirst, 1);
+
+ costEstimator.addLocalSortCost(BIG_ESTIMATES, mergeSmallSecond);
+ costEstimator.addLocalSortCost(SMALL_ESTIMATES, mergeSmallSecond);
+ costEstimator.addLocalMergeCost(BIG_ESTIMATES, SMALL_ESTIMATES, mergeSmallSecond, 1);
+
+ costEstimator.addLocalSortCost(SMALL_ESTIMATES, mergeBothSmall);
+ costEstimator.addLocalSortCost(SMALL_ESTIMATES, mergeBothSmall);
+ costEstimator.addLocalMergeCost(SMALL_ESTIMATES, SMALL_ESTIMATES, mergeBothSmall, 1);
+
+ costEstimator.addLocalSortCost(BIG_ESTIMATES, mergeBothLarge);
+ costEstimator.addLocalSortCost(BIG_ESTIMATES, mergeBothLarge);
+ costEstimator.addLocalMergeCost(BIG_ESTIMATES, BIG_ESTIMATES, mergeBothLarge, 1);
+
+
+ assertTrue(mergeBothSmall.compareTo(mergeSmallFirst) < 0);
+ assertTrue(mergeBothSmall.compareTo(mergeSmallSecond) < 0);
+ assertTrue(mergeSmallFirst.compareTo(mergeSmallSecond) == 0);
+ assertTrue(mergeSmallFirst.compareTo(mergeBothLarge) < 0);
+ assertTrue(mergeSmallSecond.compareTo(mergeBothLarge) < 0);
+
+ // compare merge join and hash join costs
+
+ assertTrue(hashBothSmall.compareTo(mergeBothSmall) < 0);
+ assertTrue(hashBothLarge.compareTo(mergeBothLarge) < 0);
+ assertTrue(hashSmallBuild.compareTo(mergeSmallFirst) < 0);
+ assertTrue(hashSmallBuild.compareTo(mergeSmallSecond) < 0);
+ assertTrue(hashLargeBuild.compareTo(mergeSmallFirst) < 0);
+ assertTrue(hashLargeBuild.compareTo(mergeSmallSecond) < 0);
+ }
+
+ // --------------------------------------------------------------------------------------------
+
+ @Test
+ public void testJoinCostFormulasWithWeights() {
+ testJoinCostFormulasWithWeights(UNKNOWN_ESTIMATES, SMALL_ESTIMATES);
+ testJoinCostFormulasWithWeights(SMALL_ESTIMATES, UNKNOWN_ESTIMATES);
+ testJoinCostFormulasWithWeights(UNKNOWN_ESTIMATES, MEDIUM_ESTIMATES);
+ testJoinCostFormulasWithWeights(MEDIUM_ESTIMATES, UNKNOWN_ESTIMATES);
+ testJoinCostFormulasWithWeights(BIG_ESTIMATES, MEDIUM_ESTIMATES);
+ testJoinCostFormulasWithWeights(MEDIUM_ESTIMATES, BIG_ESTIMATES);
+ }
+
+ private void testJoinCostFormulasWithWeights(EstimateProvider e1, EstimateProvider e2) {
+ Costs hf1 = new Costs();
+ Costs hf5 = new Costs();
+ Costs hs1 = new Costs();
+ Costs hs5 = new Costs();
+ Costs mm1 = new Costs();
+ Costs mm5 = new Costs();
+
+ costEstimator.addHybridHashCosts(e1, e2, hf1, 1);
+ costEstimator.addHybridHashCosts(e1, e2, hf5, 5);
+ costEstimator.addHybridHashCosts(e2, e1, hs1, 1);
+ costEstimator.addHybridHashCosts(e2, e1, hs5, 5);
+
+ costEstimator.addLocalSortCost(e1, mm1);
+ costEstimator.addLocalSortCost(e2, mm1);
+ costEstimator.addLocalMergeCost(e1, e2, mm1, 1);
+
+ costEstimator.addLocalSortCost(e1, mm5);
+ costEstimator.addLocalSortCost(e2, mm5);
+ mm5.multiplyWith(5);
+ costEstimator.addLocalMergeCost(e1, e2, mm5, 5);
+
+ // weight 1 versus weight 5
+ assertTrue(hf1.compareTo(hf5) < 0);
+ assertTrue(hs1.compareTo(hs5) < 0);
+ assertTrue(mm1.compareTo(mm5) < 0);
+
+ // hash versus merge
+ assertTrue(hf1.compareTo(mm1) < 0);
+ assertTrue(hs1.compareTo(mm1) < 0);
+ assertTrue(hf5.compareTo(mm5) < 0);
+ assertTrue(hs5.compareTo(mm5) < 0);
+ }
+
+ // --------------------------------------------------------------------------------------------
+
+ @Test
+ public void testHashJoinCostFormulasWithCaches() {
+
+ Costs hashBothUnknown10 = new Costs();
+ Costs hashBothUnknownCached10 = new Costs();
+
+ Costs hashBothSmall10 = new Costs();
+ Costs hashBothSmallCached10 = new Costs();
+
+ Costs hashSmallLarge10 = new Costs();
+ Costs hashSmallLargeCached10 = new Costs();
+
+ Costs hashLargeSmall10 = new Costs();
+ Costs hashLargeSmallCached10 = new Costs();
+
+ Costs hashLargeSmall1 = new Costs();
+ Costs hashLargeSmallCached1 = new Costs();
+
+ costEstimator.addHybridHashCosts(UNKNOWN_ESTIMATES, UNKNOWN_ESTIMATES, hashBothUnknown10, 10);
+ costEstimator.addCachedHybridHashCosts(UNKNOWN_ESTIMATES, UNKNOWN_ESTIMATES, hashBothUnknownCached10, 10);
+
+ costEstimator.addHybridHashCosts(MEDIUM_ESTIMATES, MEDIUM_ESTIMATES, hashBothSmall10, 10);
+ costEstimator.addCachedHybridHashCosts(MEDIUM_ESTIMATES, MEDIUM_ESTIMATES, hashBothSmallCached10, 10);
+
+ costEstimator.addHybridHashCosts(MEDIUM_ESTIMATES, BIG_ESTIMATES, hashSmallLarge10, 10);
+ costEstimator.addCachedHybridHashCosts(MEDIUM_ESTIMATES, BIG_ESTIMATES, hashSmallLargeCached10, 10);
+
+ costEstimator.addHybridHashCosts(BIG_ESTIMATES, MEDIUM_ESTIMATES, hashLargeSmall10, 10);
+ costEstimator.addCachedHybridHashCosts(BIG_ESTIMATES, MEDIUM_ESTIMATES, hashLargeSmallCached10, 10);
+
+ costEstimator.addHybridHashCosts(BIG_ESTIMATES, MEDIUM_ESTIMATES, hashLargeSmall1, 1);
+ costEstimator.addCachedHybridHashCosts(BIG_ESTIMATES, MEDIUM_ESTIMATES, hashLargeSmallCached1, 1);
+
+ // cached variant is always cheaper
+ assertTrue(hashBothUnknown10.compareTo(hashBothUnknownCached10) > 0);
+ assertTrue(hashBothSmall10.compareTo(hashBothSmallCached10) > 0);
+ assertTrue(hashSmallLarge10.compareTo(hashSmallLargeCached10) > 0);
+ assertTrue(hashLargeSmall10.compareTo(hashLargeSmallCached10) > 0);
+
+ // caching the large side is better, because then the small one is the one with additional I/O
+ assertTrue(hashLargeSmallCached10.compareTo(hashSmallLargeCached10) < 0);
+
+ // a weight of one makes the caching the same as the non-cached variant
+ assertTrue(hashLargeSmall1.compareTo(hashLargeSmallCached1) == 0);
+ }
+
+
+ // --------------------------------------------------------------------------------------------
+ // Estimate providers
+ // --------------------------------------------------------------------------------------------
+
+ private static final class UnknownEstimates implements EstimateProvider {
+
+ @Override
+ public long getEstimatedOutputSize() { return -1; }
+
+ @Override
+ public long getEstimatedNumRecords() { return -1; }
+
+ @Override
+ public float getEstimatedAvgWidthPerOutputRecord() { return -1.0f; }
+ }
+
+ private static final class Estimates implements EstimateProvider {
+
+ private final long size;
+ private final long records;
+ private final float width;
+
+ public Estimates(long size, long records) {
+ this(size, records, -1.0f);
+ }
+
+ public Estimates(long size, long records, float width) {
+ this.size = size;
+ this.records = records;
+ this.width = width;
+ }
+
+ @Override
+ public long getEstimatedOutputSize() {
+ return this.size;
+ }
+
+ @Override
+ public long getEstimatedNumRecords() {
+ return this.records;
+ }
+
+ @Override
+ public float getEstimatedAvgWidthPerOutputRecord() {
+ return this.width;
+ }
+ }
+}