Re: Review Request 40743: PIG-4709 Improve performance of GROUPBY operator on Spark

2015-12-17 Thread Pallavi Rao


> On Dec. 16, 2015, 6:57 a.m., kelly zhang wrote:
> > src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java, 
> > line 229
> > 
> >
> > Change 
> >  boolean noCombiner = 
> > conf.getBoolean(PigConfiguration.PIG_EXEC_NO_COMBINER, false);
> > 
> > to 
> > 
> >  boolean noCombiner = 
> > Boolean.parseBoolean(pc.getProperties().getProperty(
> > PigConfiguration.PIG_EXEC_NO_COMBINER, "false"));
> > 
> > we get the configuration from the PigContext not the 
> > configuration.   we can do the same modification to noSecondaryKey and 
> > isMultiQuery.

The conf is created locally from PigContext itself:
Configuration conf = ConfigurationUtil.toConfiguration(pc.getProperties());

Code is more readable with my change without having to do pc.getProperties() 
every time. Functionality is not affected.


> On Dec. 16, 2015, 6:57 a.m., kelly zhang wrote:
> > src/org/apache/pig/backend/hadoop/executionengine/spark/converter/LocalRearrangeConverter.java,
> >  line 64
> > 
> >
> > please don't remove this kind of code
> > if (LOG.isDebugEnabled()) {
> > LOG.debug("LocalRearrangeFunction in " + t);
> > }
> > 
> > developer uses it to print message to log file. users will not see it.

I have added the debug statement. Not removed it. It wasn't there before.


> On Dec. 16, 2015, 6:57 a.m., kelly zhang wrote:
> > src/org/apache/pig/backend/hadoop/executionengine/spark/converter/PigSecondaryKeyComparatorSpark.java,
> >  line 29
> > 
> >
> > remove the comment.

Oops. My bad. Will remove.


> On Dec. 16, 2015, 6:57 a.m., kelly zhang wrote:
> > src/org/apache/pig/backend/hadoop/executionengine/spark/converter/PreCombinerLocalRearrangeConverter.java,
> >  line 35
> > 
> >
> > PreCombinerLocalRearrangeConverter 's code is almost same as 
> > LocalRearrangeConverter's code. You can reuse LocalRearrangeConverter.
> > 
> > If i miss something, please tell me.

Yes. The code very similar, only the operator is different. I can change 
LocalRearrangeConverter to take in a PhysicalOperator, rather than 
POLocalRearrange. That way, the same class be re-used. Fine?


> On Dec. 16, 2015, 6:57 a.m., kelly zhang wrote:
> > src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/SparkCombinerOptimizer.java,
> >  line 63
> > 
> >
> > rename SparkCombinerOptimizer to CombinerOptimizer. This class is in 
> > spark package. No need to enphasize SparkCombinerCombinerOptimizer.

Ok. Will do.


> On Dec. 16, 2015, 6:57 a.m., kelly zhang wrote:
> > test/org/apache/pig/test/TestCombiner.java, line 263
> > 
> >
> > It is nice to add new unit tests but can we ensure that it passes in 
> > tez and mr mode?

It is a straight-forward PIG script, should pass on other exec modes too. Will 
double check.


> On Dec. 16, 2015, 6:57 a.m., kelly zhang wrote:
> > src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/SparkCombinerOptimizer.java,
> >  line 87
> > 
> >
> > After reading your code? I found that following should be:
> > // Output:
> > // foreach (using algebraicOp.Final)
> > //   -> reduceBy (uses algebraicOp.Intermediate)
> > //  -> localRearrange
> > // -> foreach (using algebraicOp.Initial)

Yep. Will change. It should read:
// Output:
// foreach (using algebraicOp.Final)
//   -> reduceBy (uses algebraicOp.Intermediate)
//  -> localRearrange
// -> foreach (using algebraicOp.Initial)
// -> CombinerRearrange


- Pallavi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/40743/#review109469
---


On Dec. 9, 2015, 5:49 a.m., Pallavi Rao wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/40743/
> ---
> 
> (Updated Dec. 9, 2015, 5:49 a.m.)
> 
> 
> Review request for pig, Mohit Sabharwal and Xuefu Zhang.
> 
> 
> Bugs: PIG-4709
> https://issues.apache.org/jira/browse/PIG-4709
> 
> 
> Repository: pig-git
> 
> 
> Description
> ---
> 
> Currently, the GROUPBY operator of PIG is mapped by Spark's 

Re: Review Request 40743: PIG-4709 Improve performance of GROUPBY operator on Spark

2015-12-17 Thread Pallavi Rao

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/40743/
---

(Updated Dec. 18, 2015, 6:47 a.m.)


Review request for pig, Mohit Sabharwal and Xuefu Zhang.


Changes
---

Addressed Kelly's comments


Bugs: PIG-4709
https://issues.apache.org/jira/browse/PIG-4709


Repository: pig-git


Description
---

Currently, the GROUPBY operator of PIG is mapped by Spark's CoGroup. When the 
grouped data is consumed by subsequent operations to perform algebraic 
operations, this is sub-optimal as there is lot of shuffle traffic.
The Spark Plan must be optimized to use reduceBy, where possible, so that a 
combiner is used.

Introduced a combiner optimizer that does the following:
// Checks for algebraic operations and if they exist.
// Replaces global rearrange (cogroup) with reduceBy as follows:
// Input:
// foreach (using algebraicOp)
//   -> packager
//  -> globalRearrange
//  -> localRearrange
// Output:
// foreach (using algebraicOp.Final)
//   -> reduceBy (uses algebraicOp.Intermediate)
//  -> foreach (using algebraicOp.Initial)
//  -> localRearrange


Diffs (updated)
-

  
src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java
 f8c1658 
  
src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/PORollupHIIForEach.java
 aca347d 
  src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java 
a4dbadd 
  
src/org/apache/pig/backend/hadoop/executionengine/spark/converter/GlobalRearrangeConverter.java
 5f74992 
  
src/org/apache/pig/backend/hadoop/executionengine/spark/converter/LocalRearrangeConverter.java
 9ce0492 
  
src/org/apache/pig/backend/hadoop/executionengine/spark/converter/PigSecondaryKeyComparatorSpark.java
 PRE-CREATION 
  
src/org/apache/pig/backend/hadoop/executionengine/spark/converter/ReduceByConverter.java
 PRE-CREATION 
  
src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POReduceBySpark.java
 PRE-CREATION 
  
src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/CombinerOptimizer.java
 PRE-CREATION 
  
src/org/apache/pig/backend/hadoop/executionengine/util/CombinerOptimizerUtil.java
 6b66ca1 
  
src/org/apache/pig/backend/hadoop/executionengine/util/SecondaryKeyOptimizerUtil.java
 546d91e 
  test/org/apache/pig/test/TestCombiner.java df44293 

Diff: https://reviews.apache.org/r/40743/diff/


Testing
---

The patch unblocked one UT in TestCombiner. Added another UT in the same class. 
Also did some manual testing.


Thanks,

Pallavi Rao



[jira] [Resolved] (PIG-4293) Enable unit test "TestNativeMapReduce" for spark

2015-12-17 Thread Xuefu Zhang (JIRA)

 [ 
https://issues.apache.org/jira/browse/PIG-4293?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xuefu Zhang resolved PIG-4293.
--
Resolution: Fixed

Committed to Spark branch. Thanks, Liyun!

> Enable unit test "TestNativeMapReduce" for spark
> 
>
> Key: PIG-4293
> URL: https://issues.apache.org/jira/browse/PIG-4293
> Project: Pig
>  Issue Type: Sub-task
>  Components: spark
>Reporter: liyunzhang_intel
>Assignee: liyunzhang_intel
> Fix For: spark-branch
>
> Attachments: PIG-4293.patch, PIG-4293_1.patch, 
> TEST-org.apache.pig.test.TestNativeMapReduce.txt
>
>
> error log is attached



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (PIG-4754) Fix UT failures in TestScriptLanguage

2015-12-17 Thread Xuefu Zhang (JIRA)

 [ 
https://issues.apache.org/jira/browse/PIG-4754?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xuefu Zhang resolved PIG-4754.
--
Resolution: Fixed

Committed to Spark branch. Thanks, Xianda.

> Fix UT failures in TestScriptLanguage
> -
>
> Key: PIG-4754
> URL: https://issues.apache.org/jira/browse/PIG-4754
> Project: Pig
>  Issue Type: Sub-task
>  Components: spark
>Reporter: Xianda Ke
>Assignee: Xianda Ke
> Fix For: spark-branch
>
> Attachments: PIG-4754.patch, PIG-4754_1.patch
>
>
> org.apache.pig.test.TestScriptLanguage.runParallelTest2
> Error Message
> job should succeed
> Stacktrace
> junit.framework.AssertionFailedError: job should succeed
>   at 
> org.apache.pig.test.TestScriptLanguage.runPigRunner(TestScriptLanguage.java:96)
>   at 
> org.apache.pig.test.TestScriptLanguage.runPigRunner(TestScriptLanguage.java:105)
>   at 
> org.apache.pig.test.TestScriptLanguage.runParallelTest2(TestScriptLanguage.java:311)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: Review Request 40743: PIG-4709 Improve performance of GROUPBY operator on Spark

2015-12-17 Thread Pallavi Rao


> On Dec. 16, 2015, 6:57 a.m., kelly zhang wrote:
> > src/org/apache/pig/backend/hadoop/executionengine/spark/converter/ReduceByConverter.java,
> >  line 195
> > 
> >
> > why not adding following code:
> > // Ensure output is consistent with the output of KeyValueFunction
> > result.append(t.get(0));
> > Tuple valueTuple = tf.newTuple();
> > for (Object o : ((Tuple) r.result).getAll()) {
> > if (!o.equals(key)) {
> > valueTuple.append(o);
> > }
> > }
> > result.append(valueTuple);
> >   
> >   I think returning r is ok.

If we return r, the result will be this:
(ABC,(2),(3)) - A tuple with key followed by values.

if we return result, it will look like this:
(ABC,((2),(3))) - A tuple with key and a value tuple (containing values).

The latter is what we want and hence the jugglery there. I can add a comment in 
the code to make this clear.


- Pallavi


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/40743/#review109469
---


On Dec. 9, 2015, 5:49 a.m., Pallavi Rao wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/40743/
> ---
> 
> (Updated Dec. 9, 2015, 5:49 a.m.)
> 
> 
> Review request for pig, Mohit Sabharwal and Xuefu Zhang.
> 
> 
> Bugs: PIG-4709
> https://issues.apache.org/jira/browse/PIG-4709
> 
> 
> Repository: pig-git
> 
> 
> Description
> ---
> 
> Currently, the GROUPBY operator of PIG is mapped by Spark's CoGroup. When the 
> grouped data is consumed by subsequent operations to perform algebraic 
> operations, this is sub-optimal as there is lot of shuffle traffic.
> The Spark Plan must be optimized to use reduceBy, where possible, so that a 
> combiner is used.
> 
> Introduced a combiner optimizer that does the following:
> // Checks for algebraic operations and if they exist.
> // Replaces global rearrange (cogroup) with reduceBy as follows:
> // Input:
> // foreach (using algebraicOp)
> //   -> packager
> //  -> globalRearrange
> //  -> localRearrange
> // Output:
> // foreach (using algebraicOp.Final)
> //   -> reduceBy (uses algebraicOp.Intermediate)
> //  -> foreach (using algebraicOp.Initial)
> //  -> localRearrange
> 
> 
> Diffs
> -
> 
>   
> src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java
>  f8c1658 
>   
> src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/PORollupHIIForEach.java
>  aca347d 
>   src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java 
> a4dbadd 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/GlobalRearrangeConverter.java
>  5f74992 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/LocalRearrangeConverter.java
>  9ce0492 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/PigSecondaryKeyComparatorSpark.java
>  PRE-CREATION 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/PreCombinerLocalRearrangeConverter.java
>  PRE-CREATION 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/converter/ReduceByConverter.java
>  PRE-CREATION 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POReduceBySpark.java
>  PRE-CREATION 
>   
> src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/SparkCombinerOptimizer.java
>  PRE-CREATION 
>   
> src/org/apache/pig/backend/hadoop/executionengine/util/CombinerOptimizerUtil.java
>  6b66ca1 
>   
> src/org/apache/pig/backend/hadoop/executionengine/util/SecondaryKeyOptimizerUtil.java
>  546d91e 
>   test/org/apache/pig/test/TestCombiner.java df44293 
> 
> Diff: https://reviews.apache.org/r/40743/diff/
> 
> 
> Testing
> ---
> 
> The patch unblocked one UT in TestCombiner. Added another UT in the same 
> class. Also did some manual testing.
> 
> 
> Thanks,
> 
> Pallavi Rao
> 
>



[jira] [Updated] (PIG-4709) Improve performance of GROUPBY operator on Spark

2015-12-17 Thread Pallavi Rao (JIRA)

 [ 
https://issues.apache.org/jira/browse/PIG-4709?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Pallavi Rao updated PIG-4709:
-
Attachment: PIG-4709-v2.patch

Addressed [~kellyzly]'s comments

> Improve performance of GROUPBY operator on Spark
> 
>
> Key: PIG-4709
> URL: https://issues.apache.org/jira/browse/PIG-4709
> Project: Pig
>  Issue Type: Sub-task
>  Components: spark
>Reporter: Pallavi Rao
>Assignee: Pallavi Rao
>  Labels: spork
> Fix For: spark-branch
>
> Attachments: PIG-4709-v1.patch, PIG-4709-v2.patch, PIG-4709.patch, 
> TEST-org.apache.pig.test.TestCombiner.xml
>
>
> Currently, the GROUPBY operator of PIG is mapped by Spark's CoGroup. When the 
> grouped data is consumed by subsequent operations to perform algebraic 
> operations, this is sub-optimal as there is lot of shuffle traffic. 
> The Spark Plan must be optimized to use reduceBy, where possible, so that a 
> combiner is used.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (PIG-4675) Operators with multiple predecessors fail under multiquery optimization

2015-12-17 Thread Mohit Sabharwal (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-4675?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15063531#comment-15063531
 ] 

Mohit Sabharwal commented on PIG-4675:
--

+1(non-binding)

> Operators with multiple predecessors fail under multiquery optimization
> ---
>
> Key: PIG-4675
> URL: https://issues.apache.org/jira/browse/PIG-4675
> Project: Pig
>  Issue Type: Sub-task
>  Components: spark
>Reporter: Peter Lin
>Assignee: liyunzhang_intel
> Fix For: spark-branch
>
> Attachments: PIG-4675_1.patch, PIG-4675_2.patch, PIG-4675_3.patch, 
> name.txt, ssn.txt, test.pig
>
>
> We are testing the spark branch pig recently with mapr3 and spark 1.5. It 
> turns out if we use more than 1 store command in the pig script will have 
> exception from the second store command. 
>  SSN = load '/test/ssn.txt' using PigStorage() as (ssn:long);
>  SSN_NAME = load '/test/name.txt' using PigStorage() as (ssn:long, 
> name:chararray);
>  X = JOIN SSN by ssn LEFT OUTER, SSN_NAME by ssn USING 'replicated';
>  R1 = limit SSN_NAME 10;
>  store R1 into '/tmp/test1_r1'; 
>  store X into '/tmp/test1_x';
> Exception Details:
> 15/09/11 13:37:00 INFO storage.MemoryStore: ensureFreeSpace(114448) called 
> with curMem=359237, maxMem=503379394
> 15/09/11 13:37:00 INFO storage.MemoryStore: Block broadcast_2 stored as 
> values in memory (estimated size 111.8 KB, free 479.6 MB)
> 15/09/11 13:37:00 INFO storage.MemoryStore: ensureFreeSpace(32569) called 
> with curMem=473685, maxMem=503379394
> 15/09/11 13:37:00 INFO storage.MemoryStore: Block broadcast_2_piece0 stored 
> as bytes in memory (estimated size 31.8 KB, free 479.6 MB)
> 15/09/11 13:37:00 INFO storage.BlockManagerInfo: Added broadcast_2_piece0 in 
> memory on 10.51.2.82:55960 (size: 31.8 KB, free: 479.9 MB)
> 15/09/11 13:37:00 INFO spark.SparkContext: Created broadcast 2 from 
> newAPIHadoopRDD at LoadConverter.java:88
> 15/09/11 13:37:00 WARN util.ClosureCleaner: Expected a closure; got 
> org.apache.pig.backend.hadoop.executionengine.spark.converter.LoadConverter$ToTupleFunction
> 15/09/11 13:37:00 INFO spark.SparkLauncher: Converting operator POForEach 
> (Name: SSN: New For Each(false)[bag] - scope-17 Operator Key: scope-17)
> 15/09/11 13:37:00 INFO spark.SparkLauncher: Converting operator POFRJoin 
> (Name: X: FRJoin[tuple] - scope-22 Operator Key: scope-22)
> 15/09/11 13:37:00 ERROR spark.SparkLauncher: throw exception in 
> sparkOperToRDD:
> java.lang.RuntimeException: Should have greater than1 predecessors for class 
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFRJoin.
>  Got : 1
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil.assertPredecessorSizeGreaterThan(SparkUtil.java:93)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.converter.FRJoinConverter.convert(FRJoinConverter.java:55)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.converter.FRJoinConverter.convert(FRJoinConverter.java:46)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:633)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:600)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:621)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.sparkOperToRDD(SparkLauncher.java:552)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.sparkPlanToRDD(SparkLauncher.java:501)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.launchPig(SparkLauncher.java:204)
> at 
> org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.launchPig(HExecutionEngine.java:301)
> at org.apache.pig.PigServer.launchPlan(PigServer.java:1390)
> at 
> org.apache.pig.PigServer.executeCompiledLogicalPlan(PigServer.java:1375)
> at org.apache.pig.PigServer.execute(PigServer.java:1364)
> at org.apache.pig.PigServer.executeBatch(PigServer.java:415)
> at org.apache.pig.PigServer.executeBatch(PigServer.java:398)
> at 
> org.apache.pig.tools.grunt.GruntParser.executeBatch(GruntParser.java:171)
> at 
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:234)
> at 
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:205)
> at org.apache.pig.tools.grunt.Grunt.exec(Grunt.java:81)
> at org.apache.pig.Main.run(Main.java:624)
> at org.apache.pig.Main.main(Main.java:170)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (PIG-4675) Operators with multiple predecessors fail under multiquery optimization

2015-12-17 Thread liyunzhang_intel (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-4675?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15063532#comment-15063532
 ] 

liyunzhang_intel commented on PIG-4675:
---

[~xuefuz]: [~mohitsabharwal] has +1. please help commit this patch.

> Operators with multiple predecessors fail under multiquery optimization
> ---
>
> Key: PIG-4675
> URL: https://issues.apache.org/jira/browse/PIG-4675
> Project: Pig
>  Issue Type: Sub-task
>  Components: spark
>Reporter: Peter Lin
>Assignee: liyunzhang_intel
> Fix For: spark-branch
>
> Attachments: PIG-4675_1.patch, PIG-4675_2.patch, PIG-4675_3.patch, 
> name.txt, ssn.txt, test.pig
>
>
> We are testing the spark branch pig recently with mapr3 and spark 1.5. It 
> turns out if we use more than 1 store command in the pig script will have 
> exception from the second store command. 
>  SSN = load '/test/ssn.txt' using PigStorage() as (ssn:long);
>  SSN_NAME = load '/test/name.txt' using PigStorage() as (ssn:long, 
> name:chararray);
>  X = JOIN SSN by ssn LEFT OUTER, SSN_NAME by ssn USING 'replicated';
>  R1 = limit SSN_NAME 10;
>  store R1 into '/tmp/test1_r1'; 
>  store X into '/tmp/test1_x';
> Exception Details:
> 15/09/11 13:37:00 INFO storage.MemoryStore: ensureFreeSpace(114448) called 
> with curMem=359237, maxMem=503379394
> 15/09/11 13:37:00 INFO storage.MemoryStore: Block broadcast_2 stored as 
> values in memory (estimated size 111.8 KB, free 479.6 MB)
> 15/09/11 13:37:00 INFO storage.MemoryStore: ensureFreeSpace(32569) called 
> with curMem=473685, maxMem=503379394
> 15/09/11 13:37:00 INFO storage.MemoryStore: Block broadcast_2_piece0 stored 
> as bytes in memory (estimated size 31.8 KB, free 479.6 MB)
> 15/09/11 13:37:00 INFO storage.BlockManagerInfo: Added broadcast_2_piece0 in 
> memory on 10.51.2.82:55960 (size: 31.8 KB, free: 479.9 MB)
> 15/09/11 13:37:00 INFO spark.SparkContext: Created broadcast 2 from 
> newAPIHadoopRDD at LoadConverter.java:88
> 15/09/11 13:37:00 WARN util.ClosureCleaner: Expected a closure; got 
> org.apache.pig.backend.hadoop.executionengine.spark.converter.LoadConverter$ToTupleFunction
> 15/09/11 13:37:00 INFO spark.SparkLauncher: Converting operator POForEach 
> (Name: SSN: New For Each(false)[bag] - scope-17 Operator Key: scope-17)
> 15/09/11 13:37:00 INFO spark.SparkLauncher: Converting operator POFRJoin 
> (Name: X: FRJoin[tuple] - scope-22 Operator Key: scope-22)
> 15/09/11 13:37:00 ERROR spark.SparkLauncher: throw exception in 
> sparkOperToRDD:
> java.lang.RuntimeException: Should have greater than1 predecessors for class 
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFRJoin.
>  Got : 1
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil.assertPredecessorSizeGreaterThan(SparkUtil.java:93)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.converter.FRJoinConverter.convert(FRJoinConverter.java:55)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.converter.FRJoinConverter.convert(FRJoinConverter.java:46)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:633)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:600)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:621)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.sparkOperToRDD(SparkLauncher.java:552)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.sparkPlanToRDD(SparkLauncher.java:501)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.launchPig(SparkLauncher.java:204)
> at 
> org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.launchPig(HExecutionEngine.java:301)
> at org.apache.pig.PigServer.launchPlan(PigServer.java:1390)
> at 
> org.apache.pig.PigServer.executeCompiledLogicalPlan(PigServer.java:1375)
> at org.apache.pig.PigServer.execute(PigServer.java:1364)
> at org.apache.pig.PigServer.executeBatch(PigServer.java:415)
> at org.apache.pig.PigServer.executeBatch(PigServer.java:398)
> at 
> org.apache.pig.tools.grunt.GruntParser.executeBatch(GruntParser.java:171)
> at 
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:234)
> at 
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:205)
> at org.apache.pig.tools.grunt.Grunt.exec(Grunt.java:81)
> at org.apache.pig.Main.run(Main.java:624)
> at org.apache.pig.Main.main(Main.java:170)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] Subscription: PIG patch available

2015-12-17 Thread jira
Issue Subscription
Filter: PIG patch available (32 issues)

Subscriber: pigdaily

Key Summary
PIG-4763Insufficient check for the number of arguments in runpigmix.pl
https://issues.apache.org/jira/browse/PIG-4763
PIG-4760TezDAGStats.convertToHadoopCounters is not used, but impose MR 
counter limit
https://issues.apache.org/jira/browse/PIG-4760
PIG-4745DataBag should protect content of passed list of tuples
https://issues.apache.org/jira/browse/PIG-4745
PIG-4734TOMAP schema inferring breaks some scripts in type checking for 
bincond
https://issues.apache.org/jira/browse/PIG-4734
PIG-4709Improve performance of GROUPBY operator on Spark
https://issues.apache.org/jira/browse/PIG-4709
PIG-4684Exception should be changed to warning when job diagnostics cannot 
be fetched
https://issues.apache.org/jira/browse/PIG-4684
PIG-4677Display failure information on stop on failure
https://issues.apache.org/jira/browse/PIG-4677
PIG-4675Operators with multiple predecessors fail under multiquery 
optimization
https://issues.apache.org/jira/browse/PIG-4675
PIG-4656Improve String serialization and comparator performance in 
BinInterSedes
https://issues.apache.org/jira/browse/PIG-4656
PIG-4641Print the instance of Object without using toString()
https://issues.apache.org/jira/browse/PIG-4641
PIG-4598Allow user defined plan optimizer rules
https://issues.apache.org/jira/browse/PIG-4598
PIG-4581thread safe issue in NodeIdGenerator
https://issues.apache.org/jira/browse/PIG-4581
PIG-4551Partition filter is not pushed down in case of SPLIT
https://issues.apache.org/jira/browse/PIG-4551
PIG-4539New PigUnit
https://issues.apache.org/jira/browse/PIG-4539
PIG-4515org.apache.pig.builtin.Distinct throws ClassCastException
https://issues.apache.org/jira/browse/PIG-4515
PIG-4455Should use DependencyOrderWalker instead of DepthFirstWalker in 
MRPrinter
https://issues.apache.org/jira/browse/PIG-4455
PIG-4373Implement PIG-3861 in Tez
https://issues.apache.org/jira/browse/PIG-4373
PIG-4341Add CMX support to pig.tmpfilecompression.codec
https://issues.apache.org/jira/browse/PIG-4341
PIG-4323PackageConverter hanging in Spark
https://issues.apache.org/jira/browse/PIG-4323
PIG-4313StackOverflowError in LIMIT operation on Spark
https://issues.apache.org/jira/browse/PIG-4313
PIG-4251Pig on Storm
https://issues.apache.org/jira/browse/PIG-4251
PIG-4111Make Pig compiles with avro-1.7.7
https://issues.apache.org/jira/browse/PIG-4111
PIG-4002Disable combiner when map-side aggregation is used
https://issues.apache.org/jira/browse/PIG-4002
PIG-3952PigStorage accepts '-tagSplit' to return full split information
https://issues.apache.org/jira/browse/PIG-3952
PIG-3911Define unique fields with @OutputSchema
https://issues.apache.org/jira/browse/PIG-3911
PIG-3877Getting Geo Latitude/Longitude from Address Lines
https://issues.apache.org/jira/browse/PIG-3877
PIG-3873Geo distance calculation using Haversine
https://issues.apache.org/jira/browse/PIG-3873
PIG-3866Create ThreadLocal classloader per PigContext
https://issues.apache.org/jira/browse/PIG-3866
PIG-3864ToDate(userstring, format, timezone) computes DateTime with strange 
handling of Daylight Saving Time with location based timezones
https://issues.apache.org/jira/browse/PIG-3864
PIG-3851Upgrade jline to 2.11
https://issues.apache.org/jira/browse/PIG-3851
PIG-3668COR built-in function when atleast one of the coefficient values is 
NaN
https://issues.apache.org/jira/browse/PIG-3668
PIG-3587add functionality for rolling over dates
https://issues.apache.org/jira/browse/PIG-3587

You may edit this subscription at:
https://issues.apache.org/jira/secure/FilterSubscription!default.jspa?subId=16328=12322384


[jira] [Commented] (PIG-4293) Enable unit test "TestNativeMapReduce" for spark

2015-12-17 Thread Mohit Sabharwal (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-4293?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15063344#comment-15063344
 ] 

Mohit Sabharwal commented on PIG-4293:
--

+1 (non-binding)

> Enable unit test "TestNativeMapReduce" for spark
> 
>
> Key: PIG-4293
> URL: https://issues.apache.org/jira/browse/PIG-4293
> Project: Pig
>  Issue Type: Sub-task
>  Components: spark
>Reporter: liyunzhang_intel
>Assignee: liyunzhang_intel
> Fix For: spark-branch
>
> Attachments: PIG-4293.patch, PIG-4293_1.patch, 
> TEST-org.apache.pig.test.TestNativeMapReduce.txt
>
>
> error log is attached



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (PIG-4675) Operators with multiple predecessors fail under multiquery optimization

2015-12-17 Thread Mohit Sabharwal (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-4675?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15063353#comment-15063353
 ] 

Mohit Sabharwal commented on PIG-4675:
--

Thanks, [~kellyzly]. I had one minor comment. Otherwise LGTM.

> Operators with multiple predecessors fail under multiquery optimization
> ---
>
> Key: PIG-4675
> URL: https://issues.apache.org/jira/browse/PIG-4675
> Project: Pig
>  Issue Type: Sub-task
>  Components: spark
>Reporter: Peter Lin
>Assignee: liyunzhang_intel
> Fix For: spark-branch
>
> Attachments: PIG-4675_1.patch, PIG-4675_2.patch, name.txt, ssn.txt, 
> test.pig
>
>
> We are testing the spark branch pig recently with mapr3 and spark 1.5. It 
> turns out if we use more than 1 store command in the pig script will have 
> exception from the second store command. 
>  SSN = load '/test/ssn.txt' using PigStorage() as (ssn:long);
>  SSN_NAME = load '/test/name.txt' using PigStorage() as (ssn:long, 
> name:chararray);
>  X = JOIN SSN by ssn LEFT OUTER, SSN_NAME by ssn USING 'replicated';
>  R1 = limit SSN_NAME 10;
>  store R1 into '/tmp/test1_r1'; 
>  store X into '/tmp/test1_x';
> Exception Details:
> 15/09/11 13:37:00 INFO storage.MemoryStore: ensureFreeSpace(114448) called 
> with curMem=359237, maxMem=503379394
> 15/09/11 13:37:00 INFO storage.MemoryStore: Block broadcast_2 stored as 
> values in memory (estimated size 111.8 KB, free 479.6 MB)
> 15/09/11 13:37:00 INFO storage.MemoryStore: ensureFreeSpace(32569) called 
> with curMem=473685, maxMem=503379394
> 15/09/11 13:37:00 INFO storage.MemoryStore: Block broadcast_2_piece0 stored 
> as bytes in memory (estimated size 31.8 KB, free 479.6 MB)
> 15/09/11 13:37:00 INFO storage.BlockManagerInfo: Added broadcast_2_piece0 in 
> memory on 10.51.2.82:55960 (size: 31.8 KB, free: 479.9 MB)
> 15/09/11 13:37:00 INFO spark.SparkContext: Created broadcast 2 from 
> newAPIHadoopRDD at LoadConverter.java:88
> 15/09/11 13:37:00 WARN util.ClosureCleaner: Expected a closure; got 
> org.apache.pig.backend.hadoop.executionengine.spark.converter.LoadConverter$ToTupleFunction
> 15/09/11 13:37:00 INFO spark.SparkLauncher: Converting operator POForEach 
> (Name: SSN: New For Each(false)[bag] - scope-17 Operator Key: scope-17)
> 15/09/11 13:37:00 INFO spark.SparkLauncher: Converting operator POFRJoin 
> (Name: X: FRJoin[tuple] - scope-22 Operator Key: scope-22)
> 15/09/11 13:37:00 ERROR spark.SparkLauncher: throw exception in 
> sparkOperToRDD:
> java.lang.RuntimeException: Should have greater than1 predecessors for class 
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFRJoin.
>  Got : 1
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil.assertPredecessorSizeGreaterThan(SparkUtil.java:93)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.converter.FRJoinConverter.convert(FRJoinConverter.java:55)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.converter.FRJoinConverter.convert(FRJoinConverter.java:46)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:633)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:600)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:621)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.sparkOperToRDD(SparkLauncher.java:552)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.sparkPlanToRDD(SparkLauncher.java:501)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.launchPig(SparkLauncher.java:204)
> at 
> org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.launchPig(HExecutionEngine.java:301)
> at org.apache.pig.PigServer.launchPlan(PigServer.java:1390)
> at 
> org.apache.pig.PigServer.executeCompiledLogicalPlan(PigServer.java:1375)
> at org.apache.pig.PigServer.execute(PigServer.java:1364)
> at org.apache.pig.PigServer.executeBatch(PigServer.java:415)
> at org.apache.pig.PigServer.executeBatch(PigServer.java:398)
> at 
> org.apache.pig.tools.grunt.GruntParser.executeBatch(GruntParser.java:171)
> at 
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:234)
> at 
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:205)
> at org.apache.pig.tools.grunt.Grunt.exec(Grunt.java:81)
> at org.apache.pig.Main.run(Main.java:624)
> at org.apache.pig.Main.main(Main.java:170)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (PIG-4754) Fix UT failures in TestScriptLanguage

2015-12-17 Thread Mohit Sabharwal (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-4754?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15063427#comment-15063427
 ] 

Mohit Sabharwal commented on PIG-4754:
--

+1(non-binding). LGTM.

could you please add a comment explaining why that block is protected & update 
the patch?

> Fix UT failures in TestScriptLanguage
> -
>
> Key: PIG-4754
> URL: https://issues.apache.org/jira/browse/PIG-4754
> Project: Pig
>  Issue Type: Sub-task
>  Components: spark
>Reporter: Xianda Ke
>Assignee: Xianda Ke
> Fix For: spark-branch
>
> Attachments: PIG-4754.patch
>
>
> org.apache.pig.test.TestScriptLanguage.runParallelTest2
> Error Message
> job should succeed
> Stacktrace
> junit.framework.AssertionFailedError: job should succeed
>   at 
> org.apache.pig.test.TestScriptLanguage.runPigRunner(TestScriptLanguage.java:96)
>   at 
> org.apache.pig.test.TestScriptLanguage.runPigRunner(TestScriptLanguage.java:105)
>   at 
> org.apache.pig.test.TestScriptLanguage.runParallelTest2(TestScriptLanguage.java:311)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (PIG-4763) Insufficient check for the number of arguments in runpigmix.pl

2015-12-17 Thread Kengo Seki (JIRA)
Kengo Seki created PIG-4763:
---

 Summary: Insufficient check for the number of arguments in 
runpigmix.pl
 Key: PIG-4763
 URL: https://issues.apache.org/jira/browse/PIG-4763
 Project: Pig
  Issue Type: Bug
Reporter: Kengo Seki
Priority: Trivial


runpigmix.pl first checks the number of arguments as follows:

{code}
  3 if(scalar(@ARGV) < 6 )
  4 {
  5 print STDERR "Usage: $0 
[parallel] 
[numruns] [runmapreduce] [cleanup_after_test]\n";
  6 exit(-1);
  7 }
{code}

but the number of the required parameters is 8, so this check seems 
insufficient.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (PIG-4763) Insufficient check for the number of arguments in runpigmix.pl

2015-12-17 Thread Kengo Seki (JIRA)

 [ 
https://issues.apache.org/jira/browse/PIG-4763?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Kengo Seki updated PIG-4763:

Attachment: PIG-4763.1.patch

> Insufficient check for the number of arguments in runpigmix.pl
> --
>
> Key: PIG-4763
> URL: https://issues.apache.org/jira/browse/PIG-4763
> Project: Pig
>  Issue Type: Bug
>Reporter: Kengo Seki
>Priority: Trivial
> Attachments: PIG-4763.1.patch
>
>
> runpigmix.pl first checks the number of arguments as follows:
> {code}
>   3 if(scalar(@ARGV) < 6 )
>   4 {
>   5 print STDERR "Usage: $0
>  
> [parallel] [numruns] [runmapreduce] [cleanup_after_test]\n";
>   6 exit(-1);
>   7 }
> {code}
> but the number of the required parameters is 8, so this check seems 
> insufficient.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (PIG-4763) Insufficient check for the number of arguments in runpigmix.pl

2015-12-17 Thread Kengo Seki (JIRA)

 [ 
https://issues.apache.org/jira/browse/PIG-4763?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Kengo Seki updated PIG-4763:

Assignee: Kengo Seki
  Status: Patch Available  (was: Open)

> Insufficient check for the number of arguments in runpigmix.pl
> --
>
> Key: PIG-4763
> URL: https://issues.apache.org/jira/browse/PIG-4763
> Project: Pig
>  Issue Type: Bug
>Reporter: Kengo Seki
>Assignee: Kengo Seki
>Priority: Trivial
> Attachments: PIG-4763.1.patch
>
>
> runpigmix.pl first checks the number of arguments as follows:
> {code}
>   3 if(scalar(@ARGV) < 6 )
>   4 {
>   5 print STDERR "Usage: $0
>  
> [parallel] [numruns] [runmapreduce] [cleanup_after_test]\n";
>   6 exit(-1);
>   7 }
> {code}
> but the number of the required parameters is 8, so this check seems 
> insufficient.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (PIG-4293) Enable unit test "TestNativeMapReduce" for spark

2015-12-17 Thread liyunzhang_intel (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-4293?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15063478#comment-15063478
 ] 

liyunzhang_intel commented on PIG-4293:
---

[~xuefuz]: Mohit has +1, please merge PIG-4293_1.patch to branch.

> Enable unit test "TestNativeMapReduce" for spark
> 
>
> Key: PIG-4293
> URL: https://issues.apache.org/jira/browse/PIG-4293
> Project: Pig
>  Issue Type: Sub-task
>  Components: spark
>Reporter: liyunzhang_intel
>Assignee: liyunzhang_intel
> Fix For: spark-branch
>
> Attachments: PIG-4293.patch, PIG-4293_1.patch, 
> TEST-org.apache.pig.test.TestNativeMapReduce.txt
>
>
> error log is attached



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (PIG-4675) Operators with multiple predecessors fail under multiquery optimization

2015-12-17 Thread liyunzhang_intel (JIRA)

 [ 
https://issues.apache.org/jira/browse/PIG-4675?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

liyunzhang_intel updated PIG-4675:
--
Attachment: PIG-4675_3.patch

[~mohitsabharwal]:
 in PIG-4675_3.patch, adding change according  to your comment in the review 
board.


> Operators with multiple predecessors fail under multiquery optimization
> ---
>
> Key: PIG-4675
> URL: https://issues.apache.org/jira/browse/PIG-4675
> Project: Pig
>  Issue Type: Sub-task
>  Components: spark
>Reporter: Peter Lin
>Assignee: liyunzhang_intel
> Fix For: spark-branch
>
> Attachments: PIG-4675_1.patch, PIG-4675_2.patch, PIG-4675_3.patch, 
> name.txt, ssn.txt, test.pig
>
>
> We are testing the spark branch pig recently with mapr3 and spark 1.5. It 
> turns out if we use more than 1 store command in the pig script will have 
> exception from the second store command. 
>  SSN = load '/test/ssn.txt' using PigStorage() as (ssn:long);
>  SSN_NAME = load '/test/name.txt' using PigStorage() as (ssn:long, 
> name:chararray);
>  X = JOIN SSN by ssn LEFT OUTER, SSN_NAME by ssn USING 'replicated';
>  R1 = limit SSN_NAME 10;
>  store R1 into '/tmp/test1_r1'; 
>  store X into '/tmp/test1_x';
> Exception Details:
> 15/09/11 13:37:00 INFO storage.MemoryStore: ensureFreeSpace(114448) called 
> with curMem=359237, maxMem=503379394
> 15/09/11 13:37:00 INFO storage.MemoryStore: Block broadcast_2 stored as 
> values in memory (estimated size 111.8 KB, free 479.6 MB)
> 15/09/11 13:37:00 INFO storage.MemoryStore: ensureFreeSpace(32569) called 
> with curMem=473685, maxMem=503379394
> 15/09/11 13:37:00 INFO storage.MemoryStore: Block broadcast_2_piece0 stored 
> as bytes in memory (estimated size 31.8 KB, free 479.6 MB)
> 15/09/11 13:37:00 INFO storage.BlockManagerInfo: Added broadcast_2_piece0 in 
> memory on 10.51.2.82:55960 (size: 31.8 KB, free: 479.9 MB)
> 15/09/11 13:37:00 INFO spark.SparkContext: Created broadcast 2 from 
> newAPIHadoopRDD at LoadConverter.java:88
> 15/09/11 13:37:00 WARN util.ClosureCleaner: Expected a closure; got 
> org.apache.pig.backend.hadoop.executionengine.spark.converter.LoadConverter$ToTupleFunction
> 15/09/11 13:37:00 INFO spark.SparkLauncher: Converting operator POForEach 
> (Name: SSN: New For Each(false)[bag] - scope-17 Operator Key: scope-17)
> 15/09/11 13:37:00 INFO spark.SparkLauncher: Converting operator POFRJoin 
> (Name: X: FRJoin[tuple] - scope-22 Operator Key: scope-22)
> 15/09/11 13:37:00 ERROR spark.SparkLauncher: throw exception in 
> sparkOperToRDD:
> java.lang.RuntimeException: Should have greater than1 predecessors for class 
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFRJoin.
>  Got : 1
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil.assertPredecessorSizeGreaterThan(SparkUtil.java:93)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.converter.FRJoinConverter.convert(FRJoinConverter.java:55)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.converter.FRJoinConverter.convert(FRJoinConverter.java:46)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:633)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:600)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:621)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.sparkOperToRDD(SparkLauncher.java:552)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.sparkPlanToRDD(SparkLauncher.java:501)
> at 
> org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.launchPig(SparkLauncher.java:204)
> at 
> org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.launchPig(HExecutionEngine.java:301)
> at org.apache.pig.PigServer.launchPlan(PigServer.java:1390)
> at 
> org.apache.pig.PigServer.executeCompiledLogicalPlan(PigServer.java:1375)
> at org.apache.pig.PigServer.execute(PigServer.java:1364)
> at org.apache.pig.PigServer.executeBatch(PigServer.java:415)
> at org.apache.pig.PigServer.executeBatch(PigServer.java:398)
> at 
> org.apache.pig.tools.grunt.GruntParser.executeBatch(GruntParser.java:171)
> at 
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:234)
> at 
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:205)
> at org.apache.pig.tools.grunt.Grunt.exec(Grunt.java:81)
> at org.apache.pig.Main.run(Main.java:624)
> at org.apache.pig.Main.main(Main.java:170)



--
This message was sent by Atlassian 

[jira] [Updated] (PIG-4754) Fix UT failures in TestScriptLanguage

2015-12-17 Thread Xianda Ke (JIRA)

 [ 
https://issues.apache.org/jira/browse/PIG-4754?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xianda Ke updated PIG-4754:
---
Attachment: PIG-4754_1.patch

> Fix UT failures in TestScriptLanguage
> -
>
> Key: PIG-4754
> URL: https://issues.apache.org/jira/browse/PIG-4754
> Project: Pig
>  Issue Type: Sub-task
>  Components: spark
>Reporter: Xianda Ke
>Assignee: Xianda Ke
> Fix For: spark-branch
>
> Attachments: PIG-4754.patch, PIG-4754_1.patch
>
>
> org.apache.pig.test.TestScriptLanguage.runParallelTest2
> Error Message
> job should succeed
> Stacktrace
> junit.framework.AssertionFailedError: job should succeed
>   at 
> org.apache.pig.test.TestScriptLanguage.runPigRunner(TestScriptLanguage.java:96)
>   at 
> org.apache.pig.test.TestScriptLanguage.runPigRunner(TestScriptLanguage.java:105)
>   at 
> org.apache.pig.test.TestScriptLanguage.runParallelTest2(TestScriptLanguage.java:311)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (PIG-4754) Fix UT failures in TestScriptLanguage

2015-12-17 Thread Xianda Ke (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-4754?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15063491#comment-15063491
 ] 

Xianda Ke commented on PIG-4754:


Hi [~mohitsabharwal], Thanks for your review.  
Some comments were added. PIG-4754_1.patch is attached.  [~xuefuz], please help 
commit this patch.

> Fix UT failures in TestScriptLanguage
> -
>
> Key: PIG-4754
> URL: https://issues.apache.org/jira/browse/PIG-4754
> Project: Pig
>  Issue Type: Sub-task
>  Components: spark
>Reporter: Xianda Ke
>Assignee: Xianda Ke
> Fix For: spark-branch
>
> Attachments: PIG-4754.patch, PIG-4754_1.patch
>
>
> org.apache.pig.test.TestScriptLanguage.runParallelTest2
> Error Message
> job should succeed
> Stacktrace
> junit.framework.AssertionFailedError: job should succeed
>   at 
> org.apache.pig.test.TestScriptLanguage.runPigRunner(TestScriptLanguage.java:96)
>   at 
> org.apache.pig.test.TestScriptLanguage.runPigRunner(TestScriptLanguage.java:105)
>   at 
> org.apache.pig.test.TestScriptLanguage.runParallelTest2(TestScriptLanguage.java:311)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)