From 5228916f453f67e418f373c4de3129ea8656d6ea Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 19 Sep 2016 22:24:26 +0900 Subject: [PATCH] STORM-2100 Fix Trident SQL join tests to not rely on ordering * join doesn't guarantee preserving order (Trident and SQL itself) --- external/sql/storm-sql-core/pom.xml | 6 +++ .../backends/trident/TestPlanCompiler.java | 37 +++++++++++-------- 2 files changed, 27 insertions(+), 16 deletions(-) diff --git a/external/sql/storm-sql-core/pom.xml b/external/sql/storm-sql-core/pom.xml index d9a6098ce82..39591ae7a04 100644 --- a/external/sql/storm-sql-core/pom.xml +++ b/external/sql/storm-sql-core/pom.xml @@ -111,6 +111,12 @@ mockito-all test + + org.apache.commons + commons-collections4 + 4.1 + test + src/jvm diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java index 0ad0a46e14d..35d1364c086 100644 --- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java +++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java @@ -19,34 +19,34 @@ */ package org.apache.storm.sql.compiler.backends.trident; +import com.google.common.collect.Lists; +import org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.commons.collections4.CollectionUtils; import org.apache.storm.Config; import org.apache.storm.ILocalCluster; import org.apache.storm.LocalCluster; import org.apache.storm.generated.StormTopology; -import org.apache.storm.sql.runtime.AbstractValuesProcessor; -import org.apache.storm.tuple.Fields; -import org.apache.storm.tuple.Values; -import org.apache.storm.utils.Utils; -import org.apache.calcite.adapter.java.JavaTypeFactory; -import org.apache.calcite.jdbc.JavaTypeFactoryImpl; -import org.apache.calcite.rel.type.RelDataTypeSystem; import org.apache.storm.sql.TestUtils; import org.apache.storm.sql.TestUtils.MockSqlTridentDataSource.CollectDataFunction; import org.apache.storm.sql.compiler.TestCompilerUtils; import org.apache.storm.sql.runtime.ISqlTridentDataSource; import org.apache.storm.sql.runtime.trident.AbstractTridentProcessor; +import org.apache.storm.trident.TridentTopology; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Values; +import org.apache.storm.utils.Utils; import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.apache.storm.trident.TridentTopology; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.Callable; -import static org.apache.storm.sql.TestUtils.MockSqlTridentDataSource.CollectDataFunction.*; +import static org.apache.storm.sql.TestUtils.MockSqlTridentDataSource.CollectDataFunction.getCollectedValues; public class TestPlanCompiler { private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl( @@ -122,7 +122,7 @@ public void testCompileEquiJoinAndGroupBy() throws Exception { proc.outputStream().each(f, new CollectDataFunction(), new Fields()).toStream(); runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo); - Assert.assertArrayEquals(new Values[] { new Values(1, 2L), new Values(0, 2L)}, getCollectedValues().toArray()); + assertListsAreEqualIgnoringOrder(Lists.newArrayList(new Values(1, 2L), new Values(0, 2L)), getCollectedValues()); } @Test @@ -140,7 +140,7 @@ public void testCompileEquiJoinWithLeftOuterJoin() throws Exception { proc.outputStream().each(f, new CollectDataFunction(), new Fields()).toStream(); runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo); - Assert.assertArrayEquals(new Values[] { new Values(2, null), new Values(3, null), new Values(4, null)}, getCollectedValues().toArray()); + assertListsAreEqualIgnoringOrder(Lists.newArrayList(new Values(2, null), new Values(3, null), new Values(4, null)), getCollectedValues()); } @Test @@ -158,7 +158,7 @@ public void testCompileEquiJoinWithRightOuterJoin() throws Exception { proc.outputStream().each(f, new CollectDataFunction(), new Fields()).toStream(); runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo); - Assert.assertArrayEquals(new Values[] { new Values(2, null), new Values(3, null), new Values(4, null)}, getCollectedValues().toArray()); + assertListsAreEqualIgnoringOrder(Lists.newArrayList(new Values(2, null), new Values(3, null), new Values(4, null)), getCollectedValues()); } @Test @@ -176,9 +176,9 @@ public void testCompileEquiJoinWithFullOuterJoin() throws Exception { proc.outputStream().each(f, new CollectDataFunction(), new Fields()).toStream(); runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo); - Assert.assertArrayEquals(new Values[] { new Values(null, "dept-2"), new Values(null, "dept-3"), new Values(null, "dept-4"), - new Values(10, null), new Values(11, null), new Values(12, null), new Values(13, null), new Values(14, null)}, - getCollectedValues().toArray()); + assertListsAreEqualIgnoringOrder(Lists.newArrayList(new Values(null, "dept-2"), new Values(null, "dept-3"), new Values(null, "dept-4"), + new Values(10, null), new Values(11, null), new Values(12, null), new Values(13, null), new Values(14, null)), + getCollectedValues()); } @Test @@ -274,4 +274,9 @@ private void waitForCompletion(long timeout, Callable cond) throws Exce Thread.sleep(100); } } + + private void assertListsAreEqualIgnoringOrder(List expected, List> actual) { + Assert.assertTrue("Two lists are not same (even ignoring order)!\n"+ "Expected: " + expected + "\n" + "Actual: " + actual, + CollectionUtils.isEqualCollection(expected, actual)); + } } \ No newline at end of file