Author: xuefu
Date: Tue Jun 30 03:16:20 2015
New Revision: 1688347
URL: http://svn.apache.org/r1688347
Log:
PIG-4607: Enable TestRank1/TestRank3 unit tests in spark mode (Xianda via Xuefu)
Modified:
pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkCompiler.java
pig/branches/spark/test/org/apache/pig/test/TestRank1.java
pig/branches/spark/test/org/apache/pig/test/TestRank2.java
pig/branches/spark/test/org/apache/pig/test/TestRank3.java
Modified:
pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkCompiler.java
URL:
http://svn.apache.org/viewvc/pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkCompiler.java?rev=1688347&r1=1688346&r2=1688347&view=diff
==============================================================================
---
pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkCompiler.java
(original)
+++
pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkCompiler.java
Tue Jun 30 03:16:20 2015
@@ -57,6 +57,8 @@ import org.apache.pig.backend.hadoop.exe
import
org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStream;
import
org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POUnion;
import
org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.Packager;
+import
org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POCounter;
+import
org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.PORank;
import
org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.PlanHelper;
import org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher;
import
org.apache.pig.backend.hadoop.executionengine.spark.operator.NativeSparkOperator;
@@ -558,6 +560,32 @@ public class SparkCompiler extends PhyPl
phyToSparkOpMap.put(op, curSparkOp);
} catch (Exception e) {
int errCode = 2034;
+ String msg = "Error compiling operator "
+ + op.getClass().getSimpleName();
+ throw new SparkCompilerException(msg, errCode,
PigException.BUG, e);
+ }
+ }
+
+ @Override
+ public void visitCounter(POCounter op) throws VisitorException {
+ try {
+ addToPlan(op);
+ phyToSparkOpMap.put(op, curSparkOp);
+ } catch (Exception e) {
+ int errCode = 2034;
+ String msg = "Error compiling operator "
+ + op.getClass().getSimpleName();
+ throw new SparkCompilerException(msg, errCode,
PigException.BUG, e);
+ }
+ }
+
+ @Override
+ public void visitRank(PORank op) throws VisitorException {
+ try {
+ addToPlan(op);
+ phyToSparkOpMap.put(op, curSparkOp);
+ } catch (Exception e) {
+ int errCode = 2034;
String msg = "Error compiling operator "
+ op.getClass().getSimpleName();
throw new SparkCompilerException(msg, errCode,
PigException.BUG, e);
Modified: pig/branches/spark/test/org/apache/pig/test/TestRank1.java
URL:
http://svn.apache.org/viewvc/pig/branches/spark/test/org/apache/pig/test/TestRank1.java?rev=1688347&r1=1688346&r2=1688347&view=diff
==============================================================================
--- pig/branches/spark/test/org/apache/pig/test/TestRank1.java (original)
+++ pig/branches/spark/test/org/apache/pig/test/TestRank1.java Tue Jun 30
03:16:20 2015
@@ -19,11 +19,9 @@ package org.apache.pig.test;
import static org.apache.pig.builtin.mock.Storage.resetData;
import static org.apache.pig.builtin.mock.Storage.tuple;
-import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.List;
-import java.util.Set;
import org.apache.pig.PigServer;
import org.apache.pig.builtin.mock.Storage.Data;
@@ -32,9 +30,6 @@ import org.apache.pig.data.TupleFactory;
import org.junit.Before;
import org.junit.Test;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
public class TestRank1 {
private static TupleFactory tf = TupleFactory.getInstance();
private static PigServer pigServer;
@@ -75,21 +70,20 @@ public class TestRank1 {
+ "store C into 'result' using mock.Storage();";
Util.registerMultiLineQuery(pigServer, query);
-
- Set<Tuple> expected = ImmutableSet.of(
- tf.newTuple(ImmutableList.of((long) 1, "A", 1, "N")),
- tf.newTuple(ImmutableList.of((long) 2, "B", 2, "N")),
- tf.newTuple(ImmutableList.of((long) 3, "C", 3, "M")),
- tf.newTuple(ImmutableList.of((long) 4, "D", 4, "P")),
- tf.newTuple(ImmutableList.of((long) 5, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 6, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 7, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 8, "F", 7, "Q")),
- tf.newTuple(ImmutableList.of((long) 9, "F", 8, "T")),
- tf.newTuple(ImmutableList.of((long) 10, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 11, "G", 10, "V")));
-
- verifyExpected(data.get("result"), expected);
+ List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new
String[]{
+ "(1L,'A',1,'N')",
+ "(2L,'B',2,'N')",
+ "(3L,'C',3,'M')",
+ "(4L,'D',4,'P')",
+ "(5L,'E',4,'Q')",
+ "(6L,'E',4,'Q')",
+ "(7L,'F',8,'Q')",
+ "(8L,'F',7,'Q')",
+ "(9L,'F',8,'T')",
+ "(10L,'F',8,'Q')",
+ "(11L,'G',10,'V')"
+ });
+ Util.checkQueryOutputsAfterSort(data.get("result"), expected);
}
@Test
@@ -99,23 +93,22 @@ public class TestRank1 {
+ "store B into 'result' using mock.Storage();";
Util.registerMultiLineQuery(pigServer, query);
-
- Set<Tuple> expected = ImmutableSet.of(
- tf.newTuple(ImmutableList.of((long) 1, "Michael", "Blythe",
1,1, 1, 1, 4557045.046, 98027)),
- tf.newTuple(ImmutableList.of((long) 2, "Linda","Mitchell", 2,
1, 1, 1, 5200475.231, 98027)),
- tf.newTuple(ImmutableList.of((long) 3, "Jillian", "Carson",
3,1, 1, 1, 3857163.633, 98027)),
- tf.newTuple(ImmutableList.of((long) 4, "Garrett","Vargas", 4,
1, 1, 1, 1764938.986, 98027)),
- tf.newTuple(ImmutableList.of((long) 5, "Tsvi", "Reiter",5, 1,
1, 2, 2811012.715, 98027)),
- tf.newTuple(ImmutableList.of((long) 6, "Shu", "Ito", 6,6, 2,
2, 3018725.486, 98055)),
- tf.newTuple(ImmutableList.of((long) 7, "Jose", "Saraiva",7, 6,
2, 2, 3189356.247, 98055)),
- tf.newTuple(ImmutableList.of((long) 8, "David","Campbell", 8,
6, 2, 3, 3587378.426, 98055)),
- tf.newTuple(ImmutableList.of((long) 9, "Tete",
"Mensa-Annan",9, 6, 2, 3, 1931620.184, 98055)),
- tf.newTuple(ImmutableList.of((long) 10, "Lynn","Tsoflias", 10,
6, 2, 3, 1758385.926, 98055)),
- tf.newTuple(ImmutableList.of((long) 11, "Rachel", "Valdez",
11,6, 2, 4, 2241204.042, 98055)),
- tf.newTuple(ImmutableList.of((long) 12, "Jae", "Pak", 12,6, 2,
4, 5015682.375, 98055)),
- tf.newTuple(ImmutableList.of((long) 13, "Ranjit","Varkey
Chudukatil", 13, 6, 2, 4, 3827950.238,98055)));
-
- verifyExpected(data.get("result"), expected);
+ List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new
String[]{
+ "(1L,'Michael', 'Blythe', 1,1, 1, 1, 4557045.046, 98027)",
+ "(2L,'Linda','Mitchell', 2, 1, 1, 1, 5200475.231, 98027)",
+ "(3L,'Jillian', 'Carson', 3,1, 1, 1, 3857163.633, 98027)",
+ "(4L,'Garrett','Vargas', 4, 1, 1, 1, 1764938.986, 98027)",
+ "(5L,'Tsvi', 'Reiter',5, 1, 1, 2, 2811012.715, 98027)",
+ "(6L,'Shu', 'Ito', 6,6, 2, 2, 3018725.486, 98055)",
+ "(7L,'Jose', 'Saraiva',7, 6, 2, 2, 3189356.247, 98055)",
+ "(8L,'David','Campbell', 8, 6, 2, 3, 3587378.426, 98055)",
+ "(9L,'Tete', 'Mensa-Annan',9, 6, 2, 3, 1931620.184, 98055)",
+ "(10L, 'Lynn','Tsoflias', 10, 6, 2, 3, 1758385.926, 98055)",
+ "(11L, 'Rachel', 'Valdez', 11,6, 2, 4, 2241204.042, 98055)",
+ "(12L, 'Jae', 'Pak', 12,6, 2, 4, 5015682.375, 98055)",
+ "(13L, 'Ranjit','Varkey Chudukatil', 13, 6, 2, 4,
3827950.238,98055)"
+ });
+ Util.checkQueryOutputsAfterSort(data.get("result"), expected);
}
@Test
@@ -125,21 +118,20 @@ public class TestRank1 {
+ "store C into 'result' using mock.Storage();";
Util.registerMultiLineQuery(pigServer, query);
-
- Set<Tuple> expected = ImmutableSet.of(
- tf.newTuple(ImmutableList.of((long) 1, "C", 3, "M")),
- tf.newTuple(ImmutableList.of((long) 2, "A", 1, "N")),
- tf.newTuple(ImmutableList.of((long) 2, "B", 2, "N")),
- tf.newTuple(ImmutableList.of((long) 4, "D", 4, "P")),
- tf.newTuple(ImmutableList.of((long) 5, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 5, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 5, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 5, "F", 7, "Q")),
- tf.newTuple(ImmutableList.of((long) 5, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 10, "F", 8, "T")),
- tf.newTuple(ImmutableList.of((long) 11, "G", 10, "V")));
-
- verifyExpected(data.get("result"), expected);
+ List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new
String[]{
+ "(1L,'C',3,'M')",
+ "(2L,'A',1,'N')",
+ "(2L,'B',2,'N')",
+ "(4L,'D',4,'P')",
+ "(5L,'E',4,'Q')",
+ "(5L,'E',4,'Q')",
+ "(5L,'F',8,'Q')",
+ "(5L,'F',7,'Q')",
+ "(5L,'F',8,'Q')",
+ "(10L,'F',8,'T')",
+ "(11L,'G',10,'V')"
+ });
+ Util.checkQueryOutputsAfterSort(data.get("result"), expected);
}
@Test
@@ -149,21 +141,20 @@ public class TestRank1 {
+ "store C into 'result' using mock.Storage();";
Util.registerMultiLineQuery(pigServer, query);
-
- Set<Tuple> expected = ImmutableSet.of(
- tf.newTuple(ImmutableList.of((long) 1, "A", 1, "N")),
- tf.newTuple(ImmutableList.of((long) 2, "B", 2, "N")),
- tf.newTuple(ImmutableList.of((long) 3, "C", 3, "M")),
- tf.newTuple(ImmutableList.of((long) 4, "D", 4, "P")),
- tf.newTuple(ImmutableList.of((long) 4, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 4, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 7, "F", 7, "Q")),
- tf.newTuple(ImmutableList.of((long) 8, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 8, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 8, "F", 8, "T")),
- tf.newTuple(ImmutableList.of((long) 11, "G", 10, "V")));
-
- verifyExpected(data.get("result"), expected);
+ List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new
String[]{
+ "(1L,'A',1,'N')",
+ "(2L,'B',2,'N')",
+ "(3L,'C',3,'M')",
+ "(4L,'D',4,'P')",
+ "(4L,'E',4,'Q')",
+ "(4L,'E',4,'Q')",
+ "(7L,'F',7,'Q')",
+ "(8L,'F',8,'Q')",
+ "(8L,'F',8,'Q')",
+ "(8L,'F',8,'T')",
+ "(11L,'G',10,'V')"
+ });
+ Util.checkQueryOutputsAfterSort(data.get("result"), expected);
}
@Test
@@ -173,21 +164,20 @@ public class TestRank1 {
+ "store C into 'result' using mock.Storage();";
Util.registerMultiLineQuery(pigServer, query);
-
- Set<Tuple> expected = ImmutableSet.of(
- tf.newTuple(ImmutableList.of((long) 1, "G", 10, "V")),
- tf.newTuple(ImmutableList.of((long) 2, "F", 8, "T")),
- tf.newTuple(ImmutableList.of((long) 2, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 2, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 2, "F", 7, "Q")),
- tf.newTuple(ImmutableList.of((long) 6, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 6, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 8, "D", 4, "P")),
- tf.newTuple(ImmutableList.of((long) 9, "C", 3, "M")),
- tf.newTuple(ImmutableList.of((long) 10, "B", 2, "N")),
- tf.newTuple(ImmutableList.of((long) 11, "A", 1, "N")));
-
- verifyExpected(data.get("result"), expected);
+ List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new
String[]{
+ "(1L,'G',10,'V')",
+ "(2L,'F',8,'T')",
+ "(2L,'F',8,'Q')",
+ "(2L,'F',8,'Q')",
+ "(2L,'F',7,'Q')",
+ "(6L,'E',4,'Q')",
+ "(6L,'E',4,'Q')",
+ "(8L,'D',4,'P')",
+ "(9L,'C',3,'M')",
+ "(10L,'B',2,'N')",
+ "(11L,'A',1,'N')"
+ });
+ Util.checkQueryOutputsAfterSort(data.get("result"), expected);
}
@Test
@@ -197,23 +187,22 @@ public class TestRank1 {
+ "store C into 'result' using mock.Storage();";
Util.registerMultiLineQuery(pigServer, query);
-
- Set<Tuple> expected = ImmutableSet.of(
- tf.newTuple(ImmutableList.of((long) 1, "Michael", "Blythe",
1,1, 1, 1, 4557045.046, 98027)),
- tf.newTuple(ImmutableList.of((long) 1, "Linda","Mitchell", 2,
1, 1, 1, 5200475.231, 98027)),
- tf.newTuple(ImmutableList.of((long) 1, "Jillian", "Carson",
3,1, 1, 1, 3857163.633, 98027)),
- tf.newTuple(ImmutableList.of((long) 1, "Garrett","Vargas", 4,
1, 1, 1, 1764938.986, 98027)),
- tf.newTuple(ImmutableList.of((long) 1, "Tsvi", "Reiter",5, 1,
1, 2, 2811012.715, 98027)),
- tf.newTuple(ImmutableList.of((long) 6, "Shu", "Ito", 6,6, 2,
2, 3018725.486, 98055)),
- tf.newTuple(ImmutableList.of((long) 6, "Jose", "Saraiva",7, 6,
2, 2, 3189356.247, 98055)),
- tf.newTuple(ImmutableList.of((long) 6, "David","Campbell", 8,
6, 2, 3, 3587378.426, 98055)),
- tf.newTuple(ImmutableList.of((long) 6, "Tete",
"Mensa-Annan",9, 6, 2, 3, 1931620.184, 98055)),
- tf.newTuple(ImmutableList.of((long) 6, "Lynn","Tsoflias", 10,
6, 2, 3, 1758385.926, 98055)),
- tf.newTuple(ImmutableList.of((long) 6, "Rachel", "Valdez",
11,6, 2, 4, 2241204.042, 98055)),
- tf.newTuple(ImmutableList.of((long) 6, "Jae", "Pak", 12,6, 2,
4, 5015682.375, 98055)),
- tf.newTuple(ImmutableList.of((long) 6, "Ranjit","Varkey
Chudukatil", 13, 6, 2, 4, 3827950.238,98055)));
-
- verifyExpected(data.get("result"), expected);
+ List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new
String[]{
+ "(1L,'Michael','Blythe',1,1,1,1,4557045.046,98027)",
+ "(1L,'Linda','Mitchell',2,1,1,1,5200475.231,98027)",
+ "(1L,'Jillian','Carson',3,1,1,1,3857163.633,98027)",
+ "(1L,'Garrett','Vargas',4,1,1,1,1764938.986,98027)",
+ "(1L,'Tsvi','Reiter',5,1,1,2,2811012.715,98027)",
+ "(6L,'Shu','Ito',6,6,2,2,3018725.486,98055)",
+ "(6L,'Jose','Saraiva',7,6,2,2,3189356.247,98055)",
+ "(6L,'David','Campbell',8,6,2,3,3587378.426,98055)",
+ "(6L,'Tete','Mensa-Annan',9,6,2,3,1931620.184,98055)",
+ "(6L,'Lynn','Tsoflias',10,6,2,3,1758385.926,98055)",
+ "(6L,'Rachel','Valdez',11,6,2,4,2241204.042,98055)",
+ "(6L,'Jae','Pak',12,6,2,4,5015682.375,98055)",
+ "(6L,'Ranjit','Varkey Chudukatil',13,6,2,4,3827950.238,98055)",
+ });
+ Util.checkQueryOutputsAfterSort(data.get("result"), expected);
}
@Test
@@ -223,23 +212,22 @@ public class TestRank1 {
+ "store C into 'result' using mock.Storage();";
Util.registerMultiLineQuery(pigServer, query);
-
- Set<Tuple> expected = ImmutableSet.of(
- tf.newTuple(ImmutableList.of((long) 1, "David", "Campbell",
8,6, 2, 3, 3587378.426, 98055)),
- tf.newTuple(ImmutableList.of((long) 2, "Garrett","Vargas", 4,
1, 1, 1, 1764938.986, 98027)),
- tf.newTuple(ImmutableList.of((long) 3, "Jae", "Pak", 12,6, 2,
4, 5015682.375, 98055)),
- tf.newTuple(ImmutableList.of((long) 4, "Jillian","Carson", 3,
1, 1, 1, 3857163.633, 98027)),
- tf.newTuple(ImmutableList.of((long) 5, "Jose", "Saraiva",7, 6,
2, 2, 3189356.247, 98055)),
- tf.newTuple(ImmutableList.of((long) 6, "Linda","Mitchell", 2,
1, 1, 1, 5200475.231, 98027)),
- tf.newTuple(ImmutableList.of((long) 7, "Lynn", "Tsoflias",
10,6, 2, 3, 1758385.926, 98055)),
- tf.newTuple(ImmutableList.of((long) 8, "Michael","Blythe", 1,
1, 1, 1, 4557045.046, 98027)),
- tf.newTuple(ImmutableList.of((long) 9, "Rachel","Valdez", 11,
6, 2, 4, 2241204.042, 98055)),
- tf.newTuple(ImmutableList.of((long) 10, "Ranjit","Varkey
Chudukatil", 13, 6, 2, 4, 3827950.238, 98055)),
- tf.newTuple(ImmutableList.of((long) 11, "Shu", "Ito", 6, 6, 2,
2, 3018725.486,98055)),
- tf.newTuple(ImmutableList.of((long) 12, "Tete", "Mensa-Annan",
9, 6, 2, 3,1931620.184, 98055)),
- tf.newTuple(ImmutableList.of((long) 13, "Tsvi", "Reiter", 5,
1, 1, 2, 2811012.715,98027)));
-
- verifyExpected(data.get("result"), expected);
+ List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new
String[]{
+ "(1L,'David','Campbell',8,6,2,3,3587378.426,98055)",
+ "(2L,'Garrett','Vargas',4,1,1,1,1764938.986,98027)",
+ "(3L,'Jae','Pak',12,6,2,4,5015682.375,98055)",
+ "(4L,'Jillian','Carson',3,1,1,1,3857163.633,98027)",
+ "(5L,'Jose','Saraiva',7,6,2,2,3189356.247,98055)",
+ "(6L,'Linda','Mitchell',2,1,1,1,5200475.231,98027)",
+ "(7L,'Lynn','Tsoflias',10,6,2,3,1758385.926,98055)",
+ "(8L,'Michael','Blythe',1,1,1,1,4557045.046,98027)",
+ "(9L,'Rachel','Valdez',11,6,2,4,2241204.042,98055)",
+ "(10L,'Ranjit','Varkey
Chudukatil',13,6,2,4,3827950.238,98055)",
+ "(11L,'Shu','Ito',6,6,2,2,3018725.486,98055)",
+ "(12L,'Tete','Mensa-Annan',9,6,2,3,1931620.184,98055)",
+ "(13L,'Tsvi','Reiter',5,1,1,2,2811012.715,98027)"
+ });
+ Util.checkQueryOutputsAfterSort(data.get("result"), expected);
}
@Test
@@ -249,23 +237,22 @@ public class TestRank1 {
+ "store C into 'result' using mock.Storage();";
Util.registerMultiLineQuery(pigServer, query);
-
- Set<Tuple> expected = ImmutableSet.of(
- tf.newTuple(ImmutableList.of((long) 1, "David", "Campbell", 8,
6, 2, 3, 3587378.426, 98055)),
- tf.newTuple(ImmutableList.of((long) 2, "Garrett","Vargas", 4,
1, 1, 1, 1764938.986, 98027)),
- tf.newTuple(ImmutableList.of((long) 3, "Jae", "Pak", 12,6, 2,
4, 5015682.375, 98055)),
- tf.newTuple(ImmutableList.of((long) 4, "Jillian","Carson", 3,
1, 1, 1, 3857163.633, 98027)),
- tf.newTuple(ImmutableList.of((long) 5, "Jose", "Saraiva",7, 6,
2, 2, 3189356.247, 98055)),
- tf.newTuple(ImmutableList.of((long) 6, "Linda","Mitchell", 2,
1, 1, 1, 5200475.231, 98027)),
- tf.newTuple(ImmutableList.of((long) 7, "Lynn", "Tsoflias",
10,6, 2, 3, 1758385.926, 98055)),
- tf.newTuple(ImmutableList.of((long) 8, "Michael","Blythe", 1,
1, 1, 1, 4557045.046, 98027)),
- tf.newTuple(ImmutableList.of((long) 9, "Rachel","Valdez", 11,
6, 2, 4, 2241204.042, 98055)),
- tf.newTuple(ImmutableList.of((long) 10, "Ranjit","Varkey
Chudukatil", 13, 6, 2, 4, 3827950.238, 98055)),
- tf.newTuple(ImmutableList.of((long) 11, "Shu", "Ito", 6, 6, 2,
2, 3018725.486,98055)),
- tf.newTuple(ImmutableList.of((long) 12, "Tete", "Mensa-Annan",
9, 6, 2, 3,1931620.184, 98055)),
- tf.newTuple(ImmutableList.of((long) 13, "Tsvi", "Reiter", 5,
1, 1, 2, 2811012.715,98027)));
-
- verifyExpected(data.get("result"), expected);
+ List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new
String[]{
+ "(1L,'David','Campbell',8,6,2,3,3587378.426,98055)",
+ "(2L,'Garrett','Vargas',4,1,1,1,1764938.986,98027)",
+ "(3L,'Jae','Pak',12,6,2,4,5015682.375,98055)",
+ "(4L,'Jillian','Carson',3,1,1,1,3857163.633,98027)",
+ "(5L,'Jose','Saraiva',7,6,2,2,3189356.247,98055)",
+ "(6L,'Linda','Mitchell',2,1,1,1,5200475.231,98027)",
+ "(7L,'Lynn','Tsoflias',10,6,2,3,1758385.926,98055)",
+ "(8L,'Michael','Blythe',1,1,1,1,4557045.046,98027)",
+ "(9L,'Rachel','Valdez',11,6,2,4,2241204.042,98055)",
+ "(10L,'Ranjit','Varkey
Chudukatil',13,6,2,4,3827950.238,98055)",
+ "(11L,'Shu','Ito',6,6,2,2,3018725.486,98055)",
+ "(12L,'Tete','Mensa-Annan',9,6,2,3,1931620.184,98055)",
+ "(13L,'Tsvi','Reiter',5,1,1,2,2811012.715,98027)"
+ });
+ Util.checkQueryOutputsAfterSort(data.get("result"), expected);
}
@Test
@@ -275,27 +262,20 @@ public class TestRank1 {
+ "store C into 'result' using mock.Storage();";
Util.registerMultiLineQuery(pigServer, query);
-
- Set<Tuple> expected = ImmutableSet.of(
- tf.newTuple(ImmutableList.of((long) 1, "A", 1, "N")),
- tf.newTuple(ImmutableList.of((long) 2, "B", 2, "N")),
- tf.newTuple(ImmutableList.of((long) 3, "C", 3, "M")),
- tf.newTuple(ImmutableList.of((long) 4, "D", 4, "P")),
- tf.newTuple(ImmutableList.of((long) 5, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 5, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 7, "F", 7, "Q")),
- tf.newTuple(ImmutableList.of((long) 8, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 8, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 10, "F", 8, "T")),
- tf.newTuple(ImmutableList.of((long) 11, "G", 10, "V")));
-
- verifyExpected(data.get("result"), expected);
- }
-
- public void verifyExpected(List<Tuple> out, Set<Tuple> expected) {
- for (Tuple tup : out) {
- assertTrue(expected + " contains " + tup, expected.contains(tup));
- }
+ List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new
String[]{
+ "(1L,'A',1,'N')",
+ "(2L,'B',2,'N')",
+ "(3L,'C',3,'M')",
+ "(4L,'D',4,'P')",
+ "(5L,'E',4,'Q')",
+ "(5L,'E',4,'Q')",
+ "(7L,'F',7,'Q')",
+ "(8L,'F',8,'Q')",
+ "(8L,'F',8,'Q')",
+ "(10L,'F',8,'T')",
+ "(11L,'G',10,'V')"
+ });
+ Util.checkQueryOutputsAfterSort(data.get("result"), expected);
}
}
Modified: pig/branches/spark/test/org/apache/pig/test/TestRank2.java
URL:
http://svn.apache.org/viewvc/pig/branches/spark/test/org/apache/pig/test/TestRank2.java?rev=1688347&r1=1688346&r2=1688347&view=diff
==============================================================================
--- pig/branches/spark/test/org/apache/pig/test/TestRank2.java (original)
+++ pig/branches/spark/test/org/apache/pig/test/TestRank2.java Tue Jun 30
03:16:20 2015
@@ -19,11 +19,9 @@ package org.apache.pig.test;
import static org.apache.pig.builtin.mock.Storage.resetData;
import static org.apache.pig.builtin.mock.Storage.tuple;
-import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.List;
-import java.util.Set;
import org.apache.pig.PigServer;
import org.apache.pig.builtin.mock.Storage.Data;
@@ -32,8 +30,6 @@ import org.apache.pig.data.TupleFactory;
import org.junit.Before;
import org.junit.Test;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
public class TestRank2 {
private static PigServer pigServer;
@@ -75,21 +71,20 @@ public class TestRank2 {
+ "store C into 'result' using mock.Storage();";
Util.registerMultiLineQuery(pigServer, query);
-
- Set<Tuple> expected = ImmutableSet.of(
- tf.newTuple(ImmutableList.of((long) 1, "C", 3, "M")),
- tf.newTuple(ImmutableList.of((long) 2, "A", 1, "N")),
- tf.newTuple(ImmutableList.of((long) 2, "B", 2, "N")),
- tf.newTuple(ImmutableList.of((long) 3, "D", 4, "P")),
- tf.newTuple(ImmutableList.of((long) 4, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 4, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 4, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 4, "F", 7, "Q")),
- tf.newTuple(ImmutableList.of((long) 4, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 5, "F", 8, "T")),
- tf.newTuple(ImmutableList.of((long) 6, "G", 10, "V")));
-
- verifyExpected(data.get("result"), expected);
+ List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new
String[]{
+ "(1L,'C',3,'M')",
+ "(2L,'A',1,'N')",
+ "(2L,'B',2,'N')",
+ "(3L,'D',4,'P')",
+ "(4L,'E',4,'Q')",
+ "(4L,'E',4,'Q')",
+ "(4L,'F',8,'Q')",
+ "(4L,'F',7,'Q')",
+ "(4L,'F',8,'Q')",
+ "(5L,'F',8,'T')",
+ "(6L,'G',10,'V')"
+ });
+ Util.checkQueryOutputsAfterSort(data.get("result"), expected);
}
@Test
@@ -99,21 +94,20 @@ public class TestRank2 {
+ "store C into 'result' using mock.Storage();";
Util.registerMultiLineQuery(pigServer, query);
-
- Set<Tuple> expected = ImmutableSet.of(
- tf.newTuple(ImmutableList.of((long) 1, "A", 1, "N")),
- tf.newTuple(ImmutableList.of((long) 2, "B", 2, "N")),
- tf.newTuple(ImmutableList.of((long) 3, "C", 3, "M")),
- tf.newTuple(ImmutableList.of((long) 4, "D", 4, "P")),
- tf.newTuple(ImmutableList.of((long) 4, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 4, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 5, "F", 7, "Q")),
- tf.newTuple(ImmutableList.of((long) 6, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 6, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 6, "F", 8, "T")),
- tf.newTuple(ImmutableList.of((long) 7, "G", 10, "V")));
-
- verifyExpected(data.get("result"), expected);
+ List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new
String[]{
+ "(1L,'A',1,'N')",
+ "(2L,'B',2,'N')",
+ "(3L,'C',3,'M')",
+ "(4L,'D',4,'P')",
+ "(4L,'E',4,'Q')",
+ "(4L,'E',4,'Q')",
+ "(5L,'F',7,'Q')",
+ "(6L,'F',8,'Q')",
+ "(6L,'F',8,'Q')",
+ "(6L,'F',8,'T')",
+ "(7L,'G',10,'V')"
+ });
+ Util.checkQueryOutputsAfterSort(data.get("result"), expected);
}
@Test
@@ -123,21 +117,20 @@ public class TestRank2 {
+ "store C into 'result' using mock.Storage();";
Util.registerMultiLineQuery(pigServer, query);
-
- Set<Tuple> expected = ImmutableSet.of(
- tf.newTuple(ImmutableList.of((long) 1, "G", 10, "V")),
- tf.newTuple(ImmutableList.of((long) 2, "F", 8, "T")),
- tf.newTuple(ImmutableList.of((long) 2, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 2, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 2, "F", 7, "Q")),
- tf.newTuple(ImmutableList.of((long) 3, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 3, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 4, "D", 4, "P")),
- tf.newTuple(ImmutableList.of((long) 5, "C", 3, "M")),
- tf.newTuple(ImmutableList.of((long) 6, "B", 2, "N")),
- tf.newTuple(ImmutableList.of((long) 7, "A", 1, "N")));
-
- verifyExpected(data.get("result"), expected);
+ List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new
String[]{
+ "(1L,'G',10,'V')",
+ "(2L,'F',8,'T')",
+ "(2L,'F',8,'Q')",
+ "(2L,'F',8,'Q')",
+ "(2L,'F',7,'Q')",
+ "(3L,'E',4,'Q')",
+ "(3L,'E',4,'Q')",
+ "(4L,'D',4,'P')",
+ "(5L,'C',3,'M')",
+ "(6L,'B',2,'N')",
+ "(7L,'A',1,'N')"
+ });
+ Util.checkQueryOutputsAfterSort(data.get("result"), expected);
}
@Test
@@ -147,26 +140,20 @@ public class TestRank2 {
+ "store C into 'result' using mock.Storage();";
Util.registerMultiLineQuery(pigServer, query);
-
- Set<Tuple> expected = ImmutableSet.of(
- tf.newTuple(ImmutableList.of((long) 1, "A", 1, "N")),
- tf.newTuple(ImmutableList.of((long) 2, "B", 2, "N")),
- tf.newTuple(ImmutableList.of((long) 3, "C", 3, "M")),
- tf.newTuple(ImmutableList.of((long) 4, "D", 4, "P")),
- tf.newTuple(ImmutableList.of((long) 5, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 5, "E", 4, "Q")),
- tf.newTuple(ImmutableList.of((long) 6, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 6, "F", 8, "Q")),
- tf.newTuple(ImmutableList.of((long) 6, "F", 8, "T")),
- tf.newTuple(ImmutableList.of((long) 7, "F", 7, "Q")),
- tf.newTuple(ImmutableList.of((long) 8, "G", 10, "V")));
-
- verifyExpected(data.get("result"), expected);
- }
-
- public void verifyExpected(List<Tuple> out, Set<Tuple> expected) {
- for (Tuple tup : out) {
- assertTrue(expected + " contains " + tup, expected.contains(tup));
- }
+ List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new
String[]{
+ "(1L,'A',1,'N')",
+ "(2L,'B',2,'N')",
+ "(3L,'C',3,'M')",
+ "(4L,'D',4,'P')",
+ "(5L,'E',4,'Q')",
+ "(5L,'E',4,'Q')",
+ "(6L,'F',8,'Q')",
+ "(6L,'F',8,'Q')",
+ "(6L,'F',8,'T')",
+ "(7L,'F',7,'Q')",
+ "(8L,'G',10,'V')"
+ });
+ Util.checkQueryOutputsAfterSort(data.get("result"), expected);
}
+
}
\ No newline at end of file
Modified: pig/branches/spark/test/org/apache/pig/test/TestRank3.java
URL:
http://svn.apache.org/viewvc/pig/branches/spark/test/org/apache/pig/test/TestRank3.java?rev=1688347&r1=1688346&r2=1688347&view=diff
==============================================================================
--- pig/branches/spark/test/org/apache/pig/test/TestRank3.java (original)
+++ pig/branches/spark/test/org/apache/pig/test/TestRank3.java Tue Jun 30
03:16:20 2015
@@ -19,11 +19,9 @@ package org.apache.pig.test;
import static org.apache.pig.builtin.mock.Storage.resetData;
import static org.apache.pig.builtin.mock.Storage.tuple;
-import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.List;
-import java.util.Set;
import org.apache.pig.PigServer;
import org.apache.pig.backend.executionengine.ExecException;
@@ -34,8 +32,6 @@ import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
public class TestRank3 {
private static PigServer pigServer;
@@ -112,41 +108,39 @@ public class TestRank3 {
+ "store R8 into 'result' using mock.Storage();";
Util.registerMultiLineQuery(pigServer, query);
-
- Set<Tuple> expected = ImmutableSet.of(
- tf.newTuple(ImmutableList.of(1L,21L,5L,7L,1L,1L,0L,8L,8L)),
- tf.newTuple(ImmutableList.of(2L,26L,2L,3L,2L,5L,1L,9L,10L)),
- tf.newTuple(ImmutableList.of(3L,30L,24L,21L,2L,3L,1L,3L,10L)),
- tf.newTuple(ImmutableList.of(4L,6L,10L,8L,3L,4L,1L,7L,2L)),
- tf.newTuple(ImmutableList.of(5L,8L,28L,25L,3L,2L,1L,0L,2L)),
- tf.newTuple(ImmutableList.of(6L,28L,11L,12L,4L,6L,2L,7L,10L)),
- tf.newTuple(ImmutableList.of(7L,9L,26L,22L,5L,7L,3L,2L,3L)),
- tf.newTuple(ImmutableList.of(8L,5L,6L,5L,6L,8L,3L,8L,1L)),
- tf.newTuple(ImmutableList.of(9L,29L,16L,15L,7L,9L,4L,6L,10L)),
- tf.newTuple(ImmutableList.of(10L,18L,12L,10L,8L,11L,5L,7L,6L)),
- tf.newTuple(ImmutableList.of(11L,14L,17L,14L,9L,10L,5L,6L,5L)),
- tf.newTuple(ImmutableList.of(12L,6L,12L,8L,10L,11L,5L,7L,2L)),
- tf.newTuple(ImmutableList.of(13L,2L,17L,13L,11L,10L,5L,6L,0L)),
- tf.newTuple(ImmutableList.of(14L,26L,3L,3L,12L,14L,6L,9L,10L)),
-
tf.newTuple(ImmutableList.of(15L,15L,20L,18L,13L,13L,6L,4L,5L)),
- tf.newTuple(ImmutableList.of(16L,3L,29L,24L,14L,12L,6L,0L,0L)),
-
tf.newTuple(ImmutableList.of(17L,23L,21L,19L,15L,16L,7L,4L,8L)),
-
tf.newTuple(ImmutableList.of(18L,19L,19L,16L,16L,17L,7L,5L,6L)),
-
tf.newTuple(ImmutableList.of(19L,20L,30L,26L,16L,15L,7L,0L,6L)),
-
tf.newTuple(ImmutableList.of(20L,12L,21L,17L,17L,16L,7L,4L,4L)),
- tf.newTuple(ImmutableList.of(21L,4L,1L,1L,18L,19L,7L,10L,1L)),
- tf.newTuple(ImmutableList.of(22L,1L,7L,4L,19L,18L,7L,8L,0L)),
-
tf.newTuple(ImmutableList.of(23L,24L,14L,11L,20L,21L,8L,7L,9L)),
-
tf.newTuple(ImmutableList.of(24L,16L,25L,20L,21L,20L,8L,3L,5L)),
-
tf.newTuple(ImmutableList.of(25L,25L,27L,23L,22L,22L,9L,1L,9L)),
- tf.newTuple(ImmutableList.of(26L,21L,8L,7L,23L,25L,9L,8L,8L)),
- tf.newTuple(ImmutableList.of(27L,17L,4L,2L,24L,26L,9L,9L,6L)),
- tf.newTuple(ImmutableList.of(28L,10L,8L,6L,25L,25L,9L,8L,4L)),
- tf.newTuple(ImmutableList.of(29L,11L,15L,9L,25L,24L,9L,7L,4L)),
- tf.newTuple(ImmutableList.of(30L,12L,23L,17L,25L,23L,9L,4L,4L))
- );
-
- verifyExpected(data.get("result"), expected);
+ List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new
String[]{
+ "(1L,21L,5L,7L,1L,1L,0L,8L,8L)",
+ "(2L,26L,2L,3L,2L,5L,1L,9L,10L)",
+ "(3L,30L,24L,21L,2L,3L,1L,3L,10L)",
+ "(4L,6L,10L,8L,3L,4L,1L,7L,2L)",
+ "(5L,8L,28L,25L,3L,2L,1L,0L,2L)",
+ "(6L,28L,11L,12L,4L,6L,2L,7L,10L)",
+ "(7L,9L,26L,22L,5L,7L,3L,2L,3L)",
+ "(8L,5L,6L,5L,6L,8L,3L,8L,1L)",
+ "(9L,29L,16L,15L,7L,9L,4L,6L,10L)",
+ "(10L,18L,12L,10L,8L,11L,5L,7L,6L)",
+ "(11L,14L,17L,14L,9L,10L,5L,6L,5L)",
+ "(12L,6L,12L,8L,10L,11L,5L,7L,2L)",
+ "(13L,2L,17L,13L,11L,10L,5L,6L,0L)",
+ "(14L,26L,3L,3L,12L,14L,6L,9L,10L)",
+ "(15L,15L,20L,18L,13L,13L,6L,4L,5L)",
+ "(16L,3L,29L,24L,14L,12L,6L,0L,0L)",
+ "(17L,23L,21L,19L,15L,16L,7L,4L,8L)",
+ "(18L,19L,19L,16L,16L,17L,7L,5L,6L)",
+ "(19L,20L,30L,26L,16L,15L,7L,0L,6L)",
+ "(20L,12L,21L,17L,17L,16L,7L,4L,4L)",
+ "(21L,4L,1L,1L,18L,19L,7L,10L,1L)",
+ "(22L,1L,7L,4L,19L,18L,7L,8L,0L)",
+ "(23L,24L,14L,11L,20L,21L,8L,7L,9L)",
+ "(24L,16L,25L,20L,21L,20L,8L,3L,5L)",
+ "(25L,25L,27L,23L,22L,22L,9L,1L,9L)",
+ "(26L,21L,8L,7L,23L,25L,9L,8L,8L)",
+ "(27L,17L,4L,2L,24L,26L,9L,9L,6L)",
+ "(28L,10L,8L,6L,25L,25L,9L,8L,4L)",
+ "(29L,11L,15L,9L,25L,24L,9L,7L,4L)",
+ "(30L,12L,23L,17L,25L,23L,9L,4L,4L)"
+ });
+ Util.checkQueryOutputsAfterSort(data.get("result"), expected);
}
// See PIG-3726
@@ -157,9 +151,8 @@ public class TestRank3 {
+ "store A into 'empty_result' using mock.Storage();";
Util.registerMultiLineQuery(pigServer, query);
-
- Set<Tuple> expected = ImmutableSet.of();
- verifyExpected(data.get("empty_result"), expected);
+ List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new
String[]{});
+ Util.checkQueryOutputsAfterSort(data.get("empty_result"), expected);
}
@Test
@@ -195,10 +188,4 @@ public class TestRank3 {
Util.checkQueryOutputsAfterSort(data.get("R4"), expectedResults);
}
- public void verifyExpected(List<Tuple> out, Set<Tuple> expected) {
- for (Tuple tup : out) {
- assertTrue(expected + " contains " + tup, expected.contains(tup));
- }
- }
-
}