[jira] [Commented] (PIG-3404) Improve Pig to ignore bad files or inaccessible files or folders

2013-09-24 Thread Jerry Chen (JIRA)

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

Jerry Chen commented on PIG-3404:
-

Hi Park, sorry for the late response and I am glad that we can discuss this 
topic further in this JIRA. 

Just as mentioned in the JIRA description, we are taking the approach of 
“Ignore bad files” flag for each storage. Different storages can be controlled 
separately instead of a global flag.  On the other hand, in our use cases, we 
also want to take care that the current user may not have permission to access 
any subdirectories of the input directory, which can be looked as “bad 
directory” in concept.

Another thing is the ignore ratio. We currently take an even simpler approach 
of “ignore all” or “ignore nothing” using a flag. Just as you mentioned, 
PIG-3059 uses a threshold to control how many bad input splits can be ignored. 
This is a good thing. While the question is “How many cases in reality that we 
need a ratio is not 0 and 1?” 

I went through the patch in PIG-3059. I was trying to understand how the ratio 
is controlled globally in a distributed MapReduce task environment. It seems 
that in InputErrorTracker.java, you use a local variable (numErrors) for error 
tracking. I may miss something there but it would be very helpful if you can 
help explain.

Thank you too for providing the helpful information and let’s continue the 
discussion. 


> Improve Pig to ignore bad files or inaccessible files or folders
> 
>
> Key: PIG-3404
> URL: https://issues.apache.org/jira/browse/PIG-3404
> Project: Pig
>  Issue Type: New Feature
>  Components: data
>Affects Versions: 0.11.2
>Reporter: Jerry Chen
>  Labels: Rhino
> Attachments: PIG-3404.patch
>
>
> There are use cases in Pig:
> * A directory is used as the input of a load operation. It is possible that 
> one or more files in that directory are bad files (for example, corrupted or 
> bad data caused by compression).
> * A directory is used as the input of a load operation. The current user may 
> not have permission to access any subdirectories or files of that directory.
> The current Pig implementation will abort the whole Pig job for such cases. 
> It would be useful to have option to allow the job to continue and ignore the 
> bad files or inaccessible files/folders without abort the job, ideally, log 
> or print a warning for such error or violations. This requirement is not 
> trivial because for big data set for large analytics applications, this is 
> not always possible to sort out the  good data for processing; Ignore a few 
> of bad files may be a better choice for such situations.
> We propose to use “Ignore bad files” flag to address this problem. 
> AvroStorage and related file format in Pig already has this flag but it is 
> not complete to cover all the cases mentioned above. We would improve the 
> PigStorage and related text format to support this new flag as well as 
> improve AvroStorage and related facilities to completely support the concept.
> The flag is “Storage” (For example, PigStorage or AvroStorage) based and can 
> be set for each load operation respectively. The value of this flag will be 
> false if it is not explicitly set. Ideally, we can provide a global pig 
> parameter which forces the default value to true for all load functions even 
> if it is not explicitly set in the LOAD statement.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (PIG-3404) Improve Pig to ignore bad files or inaccessible files or folders

2013-09-24 Thread Jerry Chen (JIRA)

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

Jerry Chen updated PIG-3404:


Attachment: PIG-3404.patch

Patch for reference

> Improve Pig to ignore bad files or inaccessible files or folders
> 
>
> Key: PIG-3404
> URL: https://issues.apache.org/jira/browse/PIG-3404
> Project: Pig
>  Issue Type: New Feature
>  Components: data
>Affects Versions: 0.11.2
>Reporter: Jerry Chen
>  Labels: Rhino
> Attachments: PIG-3404.patch
>
>
> There are use cases in Pig:
> * A directory is used as the input of a load operation. It is possible that 
> one or more files in that directory are bad files (for example, corrupted or 
> bad data caused by compression).
> * A directory is used as the input of a load operation. The current user may 
> not have permission to access any subdirectories or files of that directory.
> The current Pig implementation will abort the whole Pig job for such cases. 
> It would be useful to have option to allow the job to continue and ignore the 
> bad files or inaccessible files/folders without abort the job, ideally, log 
> or print a warning for such error or violations. This requirement is not 
> trivial because for big data set for large analytics applications, this is 
> not always possible to sort out the  good data for processing; Ignore a few 
> of bad files may be a better choice for such situations.
> We propose to use “Ignore bad files” flag to address this problem. 
> AvroStorage and related file format in Pig already has this flag but it is 
> not complete to cover all the cases mentioned above. We would improve the 
> PigStorage and related text format to support this new flag as well as 
> improve AvroStorage and related facilities to completely support the concept.
> The flag is “Storage” (For example, PigStorage or AvroStorage) based and can 
> be set for each load operation respectively. The value of this flag will be 
> false if it is not explicitly set. Ideally, we can provide a global pig 
> parameter which forces the default value to true for all load functions even 
> if it is not explicitly set in the LOAD statement.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3477) *Nix commands do not execute sequentially

2013-09-24 Thread Daniel Dai (JIRA)

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

Daniel Dai commented on PIG-3477:
-

This is expected. Load is a statement which will run until Pig sees a store. 
rmf is a command which runs immediately. Maybe we shall improve the document to 
make it clear.

> *Nix commands do not execute sequentially
> -
>
> Key: PIG-3477
> URL: https://issues.apache.org/jira/browse/PIG-3477
> Project: Pig
>  Issue Type: Bug
>Reporter: Keith Dsouza
>
> Pig does not evaluate statements in sequence and evaluates the *nix statement 
> (RMF, MV) encountered first and then processes the rest of the PIG statements 
> which makes it impossible to sequentially use both PIG statements as well as 
> Unix statements in a job.
> For example:
> I want to read data from A and A_new_data and store it in A, I use this
> LOAD A
> -- Read and merge A and A.new and create A_new_data
> RMF A
> STORE A_new_data into A 
> However, RMF A runs first making the rest of my job useless. This is 
> unexpected behavior

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


Re: Review Request 14274: PIG-2672 Optimize the use of DistributedCache

2013-09-24 Thread Daniel Dai

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



trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java


Why use a 3-level directory here? Seems a little hard to manually locate 
the jar on hdfs if needed.


- Daniel Dai


On Sept. 21, 2013, 1:21 a.m., Aniket Mokashi wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/14274/
> ---
> 
> (Updated Sept. 21, 2013, 1:21 a.m.)
> 
> 
> Review request for pig, Cheolsoo Park, DanielWX DanielWX, Dmitriy Ryaboy, 
> Julien Le Dem, and Rohini Palaniswamy.
> 
> 
> Bugs: PIG-2672
> https://issues.apache.org/jira/browse/PIG-2672
> 
> 
> Repository: pig
> 
> 
> Description
> ---
> 
> added jar.cache.location option
> 
> 
> Diffs
> -
> 
>   trunk/src/org/apache/pig/PigConstants.java 1525188 
>   
> trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
>  1525188 
>   trunk/src/org/apache/pig/impl/PigContext.java 1525188 
>   trunk/src/org/apache/pig/impl/io/FileLocalizer.java 1525188 
>   trunk/test/org/apache/pig/test/TestJobControlCompiler.java 1525188 
> 
> Diff: https://reviews.apache.org/r/14274/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Aniket Mokashi
> 
>



[jira] [Commented] (PIG-3453) Implement a Storm backend to Pig

2013-09-24 Thread Brian ONeill (JIRA)

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

Brian ONeill commented on PIG-3453:
---

I've been looking at SummingBird as well, and have had success connecting Druid 
w/ Storm.  In our specific case, we have data scientists running Pig scripts 
off-line/in batch that I would love to straight-up deploy into our real-time 
data ingestion system (on top of Storm).

I'm going to continue w/ the PoC to see if I can convert one of those Pig 
scripts into a Storm topology.  I have the Pig script running on a yarn 
cluster.  I'm going to try out storm-yarn, then deploy the corresponding 
topology and do an unofficial "race" between hdfs+pig vs. kafka+storm.  I'll 
let you know how I fare.

> Implement a Storm backend to Pig
> 
>
> Key: PIG-3453
> URL: https://issues.apache.org/jira/browse/PIG-3453
> Project: Pig
>  Issue Type: New Feature
>Reporter: Pradeep Gollakota
>  Labels: storm
>
> There is a lot of interest around implementing a Storm backend to Pig for 
> streaming processing. The proposal and initial discussions can be found at 
> https://cwiki.apache.org/confluence/display/PIG/Pig+on+Storm+Proposal

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-2672) Optimize the use of DistributedCache

2013-09-24 Thread Koji Noguchi (JIRA)

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

Koji Noguchi commented on PIG-2672:
---

bq. In fact, any misconfiguration is still protected by SHA (hard to collide).

SHA is meaningless here unless verified by the trusted entity. (NodeManager or 
TaskTracker in HADOOP-9639).
Say abc.jar was installed locally. UserEvil can figure out what the shared hdfs 
path is since he has access to the local file.
Then UserEvil can upload any kind of jar with that filename as long as he is 
the first user to upload.

Now, any users trying to use this local abc.jar would be unknowingly executing 
the random jar uploaded by this UserEvil.


> Optimize the use of DistributedCache
> 
>
> Key: PIG-2672
> URL: https://issues.apache.org/jira/browse/PIG-2672
> Project: Pig
>  Issue Type: Improvement
>Reporter: Rohini Palaniswamy
>Assignee: Aniket Mokashi
> Fix For: 0.12.0
>
> Attachments: PIG-2672.patch
>
>
> Pig currently copies jar files to a temporary location in hdfs and then adds 
> them to DistributedCache for each job launched. This is inefficient in terms 
> of 
>* Space - The jars are distributed to task trackers for every job taking 
> up lot of local temporary space in tasktrackers.
>* Performance - The jar distribution impacts the job launch time.  

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] Subscription: PIG patch available

2013-09-24 Thread jira
Issue Subscription
Filter: PIG patch available (14 issues)

Subscriber: pigdaily

Key Summary
PIG-3470Print configuration variables in grunt
https://issues.apache.org/jira/browse/PIG-3470
PIG-3458ScalarExpression lost with multiquery optimization
https://issues.apache.org/jira/browse/PIG-3458
PIG-3451EvalFunc ctor reflection to determine value of type param T is 
brittle
https://issues.apache.org/jira/browse/PIG-3451
PIG-3449Move JobCreationException to 
org.apache.pig.backend.hadoop.executionengine
https://issues.apache.org/jira/browse/PIG-3449
PIG-3441Allow Pig to use default resources from Configuration objects
https://issues.apache.org/jira/browse/PIG-3441
PIG-3434Null subexpression in bincond nullifies outer tuple (or bag)
https://issues.apache.org/jira/browse/PIG-3434
PIG-3388No support for Regex for row filter in 
org.apache.pig.backend.hadoop.hbase.HBaseStorage
https://issues.apache.org/jira/browse/PIG-3388
PIG-3325Adding a tuple to a bag is slow
https://issues.apache.org/jira/browse/PIG-3325
PIG-3292Logical plan invalid state: duplicate uid in schema during 
self-join to get cross product
https://issues.apache.org/jira/browse/PIG-3292
PIG-3257Add unique identifier UDF
https://issues.apache.org/jira/browse/PIG-3257
PIG-3117A debug mode in which pig does not delete temporary files
https://issues.apache.org/jira/browse/PIG-3117
PIG-3088Add a builtin udf which removes prefixes
https://issues.apache.org/jira/browse/PIG-3088
PIG-3021Split results missing records when there is null values in the 
column comparison
https://issues.apache.org/jira/browse/PIG-3021
PIG-2672Optimize the use of DistributedCache
https://issues.apache.org/jira/browse/PIG-2672

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


[jira] [Commented] (PIG-3295) Casting from bytearray failing after Union (even when each field is from a single Loader)

2013-09-24 Thread Daniel Dai (JIRA)

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

Daniel Dai commented on PIG-3295:
-

+1.

> Casting from bytearray failing after Union (even when each field is from a 
> single Loader)
> -
>
> Key: PIG-3295
> URL: https://issues.apache.org/jira/browse/PIG-3295
> Project: Pig
>  Issue Type: Bug
>  Components: parser
>Reporter: Koji Noguchi
>Assignee: Koji Noguchi
>Priority: Minor
> Attachments: pig-3295-v01.patch, pig-3295-v02.patch, 
> pig-3295-v03.patch, pig-3295-v04.patch, pig-3295-v05.patch, pig-3295-v06.patch
>
>
> One example
> {noformat}
> A = load 'data1.txt' as line:bytearray;
> B = load 'c1.txt' using TextLoader() as cookie1;
> C = load 'c2.txt' using TextLoader() as cookie2;
> B2 = join A by line, B by cookie1;
> C2 = join A by line, C by cookie2;
> D = union onschema B2,C2; -- D: {A::line: bytearray,B::cookie1: 
> bytearray,C::cookie2: bytearray}
> E = foreach D generate (chararray) line, (chararray) cookie1, (chararray) 
> cookie2;
> dump E;
> {noformat}
> This script fails at runtime with 
> "Caused by: org.apache.pig.backend.executionengine.ExecException: ERROR 1075: 
> Received a bytearray from the UDF. Cannot determine how to convert the 
> bytearray to string."
> This is different from PIG-3293 such that each field in 'D' belongs to a 
> single loader whereas on PIG-3293, it came from multiple loader.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-2672) Optimize the use of DistributedCache

2013-09-24 Thread Rohini Palaniswamy (JIRA)

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

Rohini Palaniswamy commented on PIG-2672:
-

I guess you don't have to check for permissions as you are anyways returning 
null on a IOException. 

> Optimize the use of DistributedCache
> 
>
> Key: PIG-2672
> URL: https://issues.apache.org/jira/browse/PIG-2672
> Project: Pig
>  Issue Type: Improvement
>Reporter: Rohini Palaniswamy
>Assignee: Aniket Mokashi
> Fix For: 0.12.0
>
> Attachments: PIG-2672.patch
>
>
> Pig currently copies jar files to a temporary location in hdfs and then adds 
> them to DistributedCache for each job launched. This is inefficient in terms 
> of 
>* Space - The jars are distributed to task trackers for every job taking 
> up lot of local temporary space in tasktrackers.
>* Performance - The jar distribution impacts the job launch time.  

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


Re: Review Request 14274: PIG-2672 Optimize the use of DistributedCache

2013-09-24 Thread Rohini Palaniswamy

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



trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java


If the cacheDir is not absolute, can you do a new 
Path(fs.getHomeDirectory(), cacheDir); Else need to support something like 
${user.name} in the property.


- Rohini Palaniswamy


On Sept. 21, 2013, 1:21 a.m., Aniket Mokashi wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/14274/
> ---
> 
> (Updated Sept. 21, 2013, 1:21 a.m.)
> 
> 
> Review request for pig, Cheolsoo Park, DanielWX DanielWX, Dmitriy Ryaboy, 
> Julien Le Dem, and Rohini Palaniswamy.
> 
> 
> Bugs: PIG-2672
> https://issues.apache.org/jira/browse/PIG-2672
> 
> 
> Repository: pig
> 
> 
> Description
> ---
> 
> added jar.cache.location option
> 
> 
> Diffs
> -
> 
>   trunk/src/org/apache/pig/PigConstants.java 1525188 
>   
> trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
>  1525188 
>   trunk/src/org/apache/pig/impl/PigContext.java 1525188 
>   trunk/src/org/apache/pig/impl/io/FileLocalizer.java 1525188 
>   trunk/test/org/apache/pig/test/TestJobControlCompiler.java 1525188 
> 
> Diff: https://reviews.apache.org/r/14274/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Aniket Mokashi
> 
>



Re: Review Request 14274: PIG-2672 Optimize the use of DistributedCache

2013-09-24 Thread Rohini Palaniswamy


> On Sept. 25, 2013, 12:13 a.m., Rohini Palaniswamy wrote:
> > trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java,
> >  line 1495
> > 
> >
> > Since the name of the file on hdfs is different from that of the actual 
> > file, create a symlink with the actual filename. Some users might depend on 
> > the actual file name.

One case I see is python scripts(jython UDFs) which do imports based on the 
file name. Would be the same for other scripting languages that we support. It 
would be good to run the full unit and e2e test with your patch before going 
for a commit


- Rohini


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


On Sept. 21, 2013, 1:21 a.m., Aniket Mokashi wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/14274/
> ---
> 
> (Updated Sept. 21, 2013, 1:21 a.m.)
> 
> 
> Review request for pig, Cheolsoo Park, DanielWX DanielWX, Dmitriy Ryaboy, 
> Julien Le Dem, and Rohini Palaniswamy.
> 
> 
> Bugs: PIG-2672
> https://issues.apache.org/jira/browse/PIG-2672
> 
> 
> Repository: pig
> 
> 
> Description
> ---
> 
> added jar.cache.location option
> 
> 
> Diffs
> -
> 
>   trunk/src/org/apache/pig/PigConstants.java 1525188 
>   
> trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
>  1525188 
>   trunk/src/org/apache/pig/impl/PigContext.java 1525188 
>   trunk/src/org/apache/pig/impl/io/FileLocalizer.java 1525188 
>   trunk/test/org/apache/pig/test/TestJobControlCompiler.java 1525188 
> 
> Diff: https://reviews.apache.org/r/14274/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Aniket Mokashi
> 
>



[jira] [Commented] (PIG-2672) Optimize the use of DistributedCache

2013-09-24 Thread Rohini Palaniswamy (JIRA)

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

Rohini Palaniswamy commented on PIG-2672:
-

[~aniket486],

bq. Currently jars get copied to /tmp/temp-/ which can be written by 
all users
   No. They do not. They go into /user//.staging which is readable 
and writable only by that user. Even if it were to go to /tmp/temp- (where the 
intermediate files now go), we have the dfs.umaskmode set to 077 so only the 
user has rwx and no one else has.

  It is good to have shared cluster location, but if someone accidentally 
deletes that directory then all user jobs already launched will fail. It would 
be good if you can add the check to see if the cachedir is writable before 
trying to create it there. People with a multi-tenant environment like us can 
then chose to place frequently used jars in the shared cluster location but 
protect it with 755 so that others don't write into it.

> Optimize the use of DistributedCache
> 
>
> Key: PIG-2672
> URL: https://issues.apache.org/jira/browse/PIG-2672
> Project: Pig
>  Issue Type: Improvement
>Reporter: Rohini Palaniswamy
>Assignee: Aniket Mokashi
> Fix For: 0.12.0
>
> Attachments: PIG-2672.patch
>
>
> Pig currently copies jar files to a temporary location in hdfs and then adds 
> them to DistributedCache for each job launched. This is inefficient in terms 
> of 
>* Space - The jars are distributed to task trackers for every job taking 
> up lot of local temporary space in tasktrackers.
>* Performance - The jar distribution impacts the job launch time.  

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


Re: Review Request 14274: PIG-2672 Optimize the use of DistributedCache

2013-09-24 Thread Rohini Palaniswamy

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


There are some white spaces in the patch and code is not formatted. Noticed 
that many places have no space before and after operators like + (concat), !=, 
etc.


trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java


If hdfs path use as is and do not ship to jar cache. It will also save time 
and hash checks.



trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java


Since the name of the file on hdfs is different from that of the actual 
file, create a symlink with the actual filename. Some users might depend on the 
actual file name.



trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java


First do a file size comparison before calculating checksum for better 
efficiency



trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java


Can write a PathFilter that matches the filesize and does a endsWith "-" + 
suffix.

On the filtered list, then can apply the checksum name check.


- Rohini Palaniswamy


On Sept. 21, 2013, 1:21 a.m., Aniket Mokashi wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/14274/
> ---
> 
> (Updated Sept. 21, 2013, 1:21 a.m.)
> 
> 
> Review request for pig, Cheolsoo Park, DanielWX DanielWX, Dmitriy Ryaboy, 
> Julien Le Dem, and Rohini Palaniswamy.
> 
> 
> Bugs: PIG-2672
> https://issues.apache.org/jira/browse/PIG-2672
> 
> 
> Repository: pig
> 
> 
> Description
> ---
> 
> added jar.cache.location option
> 
> 
> Diffs
> -
> 
>   trunk/src/org/apache/pig/PigConstants.java 1525188 
>   
> trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
>  1525188 
>   trunk/src/org/apache/pig/impl/PigContext.java 1525188 
>   trunk/src/org/apache/pig/impl/io/FileLocalizer.java 1525188 
>   trunk/test/org/apache/pig/test/TestJobControlCompiler.java 1525188 
> 
> Diff: https://reviews.apache.org/r/14274/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Aniket Mokashi
> 
>



[jira] [Resolved] (PIG-3360) Some intermittent negative e2e tests fail on hadoop 2

2013-09-24 Thread Daniel Dai (JIRA)

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

Daniel Dai resolved PIG-3360.
-

  Resolution: Fixed
Hadoop Flags: Reviewed

Patch committed to both 0.12 branch and trunk.

> Some intermittent negative e2e tests fail on hadoop 2
> -
>
> Key: PIG-3360
> URL: https://issues.apache.org/jira/browse/PIG-3360
> Project: Pig
>  Issue Type: Bug
>  Components: impl
>Reporter: Daniel Dai
>Assignee: Daniel Dai
> Fix For: 0.12.0
>
> Attachments: PIG-3360-1.patch, PIG-3360-2.patch
>
>
> One example is StreamingErrors_2. Here is the stack we get:
> Backend error message
> -
> Error: org.apache.pig.backend.executionengine.ExecException: ERROR 2055: 
> Received Error while processing the map plan: 'perl PigStreamingBad.pl middle 
> ' failed with exit status: 2
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.runPipeline(PigGenericMapBase.java:311)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:278)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:64)
>   at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:145)
>   at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:763)
>   at org.apache.hadoop.mapred.MapTask.run(MapTask.java:339)
>   at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:162)
>   at java.security.AccessController.doPrivileged(Native Method)
>   at javax.security.auth.Subject.doAs(Subject.java:396)
>   at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1477)
>   at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:157)
> Pig Stack Trace
> ---
> ERROR 2244: Job failed, hadoop does not return any error message
> org.apache.pig.backend.executionengine.ExecException: ERROR 2244: Job failed, 
> hadoop does not return any error message
>   at 
> org.apache.pig.tools.grunt.GruntParser.executeBatch(GruntParser.java:145)
>   at 
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:198)
>   at 
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:170)
>   at org.apache.pig.tools.grunt.Grunt.exec(Grunt.java:84)
>   at org.apache.pig.Main.run(Main.java:604)
>   at org.apache.pig.Main.main(Main.java:157)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>   at java.lang.reflect.Method.invoke(Method.java:597)
>   at org.apache.hadoop.util.RunJar.main(RunJar.java:212)
> 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Resolved] (PIG-3468) PIG-3123 breaks e2e test Jython_Diagnostics_2

2013-09-24 Thread Daniel Dai (JIRA)

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

Daniel Dai resolved PIG-3468.
-

  Resolution: Fixed
Hadoop Flags: Reviewed

Patch committed to both 0.12 branch and trunk.

> PIG-3123 breaks e2e test Jython_Diagnostics_2
> -
>
> Key: PIG-3468
> URL: https://issues.apache.org/jira/browse/PIG-3468
> Project: Pig
>  Issue Type: Bug
>  Components: impl
>Reporter: Daniel Dai
>Assignee: Daniel Dai
> Fix For: 0.12.0
>
> Attachments: PIG-3468-1.patch
>
>
> PIG-3123 optimized TypeCastInserter by adding a castInserted flag for LOLoad 
> which do not need a LOForEach just to do the pruning. However, this flag is 
> also used in illustrate to visualize the output from the loader 
> (DisplayExamples:110). That's why Jython_Diagnostics_2 is broken.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3360) Some intermittent negative e2e tests fail on hadoop 2

2013-09-24 Thread Thejas M Nair (JIRA)

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

Thejas M Nair commented on PIG-3360:


Looks good. +1


> Some intermittent negative e2e tests fail on hadoop 2
> -
>
> Key: PIG-3360
> URL: https://issues.apache.org/jira/browse/PIG-3360
> Project: Pig
>  Issue Type: Bug
>  Components: impl
>Reporter: Daniel Dai
>Assignee: Daniel Dai
> Fix For: 0.12.0
>
> Attachments: PIG-3360-1.patch, PIG-3360-2.patch
>
>
> One example is StreamingErrors_2. Here is the stack we get:
> Backend error message
> -
> Error: org.apache.pig.backend.executionengine.ExecException: ERROR 2055: 
> Received Error while processing the map plan: 'perl PigStreamingBad.pl middle 
> ' failed with exit status: 2
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.runPipeline(PigGenericMapBase.java:311)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:278)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:64)
>   at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:145)
>   at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:763)
>   at org.apache.hadoop.mapred.MapTask.run(MapTask.java:339)
>   at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:162)
>   at java.security.AccessController.doPrivileged(Native Method)
>   at javax.security.auth.Subject.doAs(Subject.java:396)
>   at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1477)
>   at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:157)
> Pig Stack Trace
> ---
> ERROR 2244: Job failed, hadoop does not return any error message
> org.apache.pig.backend.executionengine.ExecException: ERROR 2244: Job failed, 
> hadoop does not return any error message
>   at 
> org.apache.pig.tools.grunt.GruntParser.executeBatch(GruntParser.java:145)
>   at 
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:198)
>   at 
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:170)
>   at org.apache.pig.tools.grunt.Grunt.exec(Grunt.java:84)
>   at org.apache.pig.Main.run(Main.java:604)
>   at org.apache.pig.Main.main(Main.java:157)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>   at java.lang.reflect.Method.invoke(Method.java:597)
>   at org.apache.hadoop.util.RunJar.main(RunJar.java:212)
> 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-2417) Streaming UDFs - allow users to easily write UDFs in scripting languages with no JVM implementation.

2013-09-24 Thread Daniel Dai (JIRA)

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

Daniel Dai commented on PIG-2417:
-

I compiled on both my RHEL6 and Windows, seems fine for me. We can change the 
javadoc anyway to fix the issue.

> Streaming UDFs -  allow users to easily write UDFs in scripting languages 
> with no JVM implementation.
> -
>
> Key: PIG-2417
> URL: https://issues.apache.org/jira/browse/PIG-2417
> Project: Pig
>  Issue Type: Improvement
>Affects Versions: 0.12.0
>Reporter: Jeremy Karn
>Assignee: Jeremy Karn
> Fix For: 0.12.0
>
> Attachments: PIG-2417-4.patch, PIG-2417-5.patch, PIG-2417-6.patch, 
> PIG-2417-7.patch, PIG-2417-8.patch, PIG-2417-9-1.patch, PIG-2417-9-2.patch, 
> PIG-2417-9.patch, PIG-2417-e2e.patch, streaming2.patch, streaming3.patch, 
> streaming.patch
>
>
> The goal of Streaming UDFs is to allow users to easily write UDFs in 
> scripting languages with no JVM implementation or a limited JVM 
> implementation.  The initial proposal is outlined here: 
> https://cwiki.apache.org/confluence/display/PIG/StreamingUDFs.
> In order to implement this we need new syntax to distinguish a streaming UDF 
> from an embedded JVM UDF.  I'd propose something like the following (although 
> I'm not sure 'language' is the best term to be using):
> {code}define my_streaming_udfs language('python') 
> ship('my_streaming_udfs.py'){code}
> We'll also need a language-specific controller script that gets shipped to 
> the cluster which is responsible for reading the input stream, deserializing 
> the input data, passing it to the user written script, serializing that 
> script output, and writing that to the output stream.
> Finally, we'll need to add a StreamingUDF class that extends evalFunc.  This 
> class will likely share some of the existing code in POStream and 
> ExecutableManager (where it make sense to pull out shared code) to stream 
> data to/from the controller script.
> One alternative approach to creating the StreamingUDF EvalFunc is to use the 
> POStream operator directly.  This would involve inserting the POStream 
> operator instead of the POUserFunc operator whenever we encountered a 
> streaming UDF while building the physical plan.  This approach seemed 
> problematic because there would need to be a lot of changes in order to 
> support POStream in all of the places we want to be able use UDFs (For 
> example - to operate on a single field inside of a for each statement).

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3468) PIG-3123 breaks e2e test Jython_Diagnostics_2

2013-09-24 Thread Alan Gates (JIRA)

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

Alan Gates commented on PIG-3468:
-

+1

> PIG-3123 breaks e2e test Jython_Diagnostics_2
> -
>
> Key: PIG-3468
> URL: https://issues.apache.org/jira/browse/PIG-3468
> Project: Pig
>  Issue Type: Bug
>  Components: impl
>Reporter: Daniel Dai
>Assignee: Daniel Dai
> Fix For: 0.12.0
>
> Attachments: PIG-3468-1.patch
>
>
> PIG-3123 optimized TypeCastInserter by adding a castInserted flag for LOLoad 
> which do not need a LOForEach just to do the pruning. However, this flag is 
> also used in illustrate to visualize the output from the loader 
> (DisplayExamples:110). That's why Jython_Diagnostics_2 is broken.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-2672) Optimize the use of DistributedCache

2013-09-24 Thread Aniket Mokashi (JIRA)

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

Aniket Mokashi commented on PIG-2672:
-

[~cheolsoo], thanks for your comments. I will work on the patch to make it more 
production ready. I have tried it on a simple job, but not in production yet.

[~knoguchi], I do not understand your concern here. Currently jars get copied 
to /tmp/temp-/ which can be written by all users. I do not see how jar 
cache is less secure than the current approach. In fact, any misconfiguration 
is still protected by SHA (hard to collide).

I do not see any benefit of restricting to use /user//.pig as its not 
mandatory to have that directory secure for users (Am I right?). If you look 
closely, cluster cache and user cache have exactly similar behavior. The only 
reason we have two is for easy configuration and better dedup of jars across 
the cluster.


> Optimize the use of DistributedCache
> 
>
> Key: PIG-2672
> URL: https://issues.apache.org/jira/browse/PIG-2672
> Project: Pig
>  Issue Type: Improvement
>Reporter: Rohini Palaniswamy
>Assignee: Aniket Mokashi
> Fix For: 0.12.0
>
> Attachments: PIG-2672.patch
>
>
> Pig currently copies jar files to a temporary location in hdfs and then adds 
> them to DistributedCache for each job launched. This is inefficient in terms 
> of 
>* Space - The jars are distributed to task trackers for every job taking 
> up lot of local temporary space in tasktrackers.
>* Performance - The jar distribution impacts the job launch time.  

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-2417) Streaming UDFs - allow users to easily write UDFs in scripting languages with no JVM implementation.

2013-09-24 Thread Rohini Palaniswamy (JIRA)

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

Rohini Palaniswamy commented on PIG-2417:
-

I see compilation fails with 4 similar errors in TestStreamingUDF.java when 
running it on Linux. Works fine on Mac. 

 test/org/apache/pig/impl/builtin/TestStreamingUDF.java:287: error: unmappable 
character for encoding UTF8

>From javac documentation:
-encoding encoding
Set the source file encoding name, such as EUC-JP and UTF-8. If -encoding is 
not specified, the platform default converter is used.

Not sure what the platform defaults exactly are in MAC, as inside a java 
program file.encoding and Charset.defaultCharset() are UTF8. Either we should 
specify -encoding in the ant javac invocation or fix the test to use \u. 

[~daijy],
   Does it compile fine on Windows?

> Streaming UDFs -  allow users to easily write UDFs in scripting languages 
> with no JVM implementation.
> -
>
> Key: PIG-2417
> URL: https://issues.apache.org/jira/browse/PIG-2417
> Project: Pig
>  Issue Type: Improvement
>Affects Versions: 0.12.0
>Reporter: Jeremy Karn
>Assignee: Jeremy Karn
> Fix For: 0.12.0
>
> Attachments: PIG-2417-4.patch, PIG-2417-5.patch, PIG-2417-6.patch, 
> PIG-2417-7.patch, PIG-2417-8.patch, PIG-2417-9-1.patch, PIG-2417-9-2.patch, 
> PIG-2417-9.patch, PIG-2417-e2e.patch, streaming2.patch, streaming3.patch, 
> streaming.patch
>
>
> The goal of Streaming UDFs is to allow users to easily write UDFs in 
> scripting languages with no JVM implementation or a limited JVM 
> implementation.  The initial proposal is outlined here: 
> https://cwiki.apache.org/confluence/display/PIG/StreamingUDFs.
> In order to implement this we need new syntax to distinguish a streaming UDF 
> from an embedded JVM UDF.  I'd propose something like the following (although 
> I'm not sure 'language' is the best term to be using):
> {code}define my_streaming_udfs language('python') 
> ship('my_streaming_udfs.py'){code}
> We'll also need a language-specific controller script that gets shipped to 
> the cluster which is responsible for reading the input stream, deserializing 
> the input data, passing it to the user written script, serializing that 
> script output, and writing that to the output stream.
> Finally, we'll need to add a StreamingUDF class that extends evalFunc.  This 
> class will likely share some of the existing code in POStream and 
> ExecutableManager (where it make sense to pull out shared code) to stream 
> data to/from the controller script.
> One alternative approach to creating the StreamingUDF EvalFunc is to use the 
> POStream operator directly.  This would involve inserting the POStream 
> operator instead of the POUserFunc operator whenever we encountered a 
> streaming UDF while building the physical plan.  This approach seemed 
> problematic because there would need to be a lot of changes in order to 
> support POStream in all of the places we want to be able use UDFs (For 
> example - to operate on a single field inside of a for each statement).

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3445) Make Parquet format available out of the box in Pig

2013-09-24 Thread Dmitriy V. Ryaboy (JIRA)

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

Dmitriy V. Ryaboy commented on PIG-3445:


Other loaders like csv, avro, json, xml, etc (even RC, though it's in piggybank 
due to heavy dependencies and lack of support) are all in already so I don't 
see this as unfair, but as consistent.
Not packaging the pq jars into pig monojar and instead adding them, the way we 
add guava et al for hbase, sounds like a good idea.
[~julienledem] should we do that by providing a simple wrapper in pig builtins, 
or by messing with the job conf in parquet's own loader/storer?

> Make Parquet format available out of the box in Pig
> ---
>
> Key: PIG-3445
> URL: https://issues.apache.org/jira/browse/PIG-3445
> Project: Pig
>  Issue Type: Improvement
>Reporter: Julien Le Dem
> Fix For: 0.12.0
>
> Attachments: PIG-3445.patch
>
>
> We would add the Parquet jar in the Pig packages to make it available out of 
> the box to pig users.
> On top of that we could add the parquet.pig package to the list of packages 
> to search for UDFs. (alternatively, the parquet jar could contain classes 
> name or.apache.pig.builtin.ParquetLoader and ParquetStorer)
> This way users can use Parquet simply by typing:
> A = LOAD 'foo' USING ParquetLoader();
> STORE A INTO 'bar' USING ParquetStorer();

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (PIG-3479) Fix BigInt, BigDec, Date serialization. Improve perf of PigNullableWritable deserilization

2013-09-24 Thread Dmitriy V. Ryaboy (JIRA)

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

Dmitriy V. Ryaboy updated PIG-3479:
---

  Resolution: Fixed
Release Note: Skewed join internals improved to get 10% or better 
improvement on reducers by eliminating unnecessary reflection.
  Status: Resolved  (was: Patch Available)

Committed to trunk and 0.12

> Fix BigInt, BigDec, Date serialization. Improve perf of PigNullableWritable 
> deserilization
> --
>
> Key: PIG-3479
> URL: https://issues.apache.org/jira/browse/PIG-3479
> Project: Pig
>  Issue Type: Bug
>Reporter: Dmitriy V. Ryaboy
>Assignee: Dmitriy V. Ryaboy
> Fix For: 0.12.0
>
> Attachments: PIG-3479.patch, PIG-3479.whitespace.patch
>
>
> While working on something unrelated I discovered some serialization errors 
> with recently added data types, and a heavy use of reflection slowing down 
> PigNullableWritable deserialization.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (PIG-3479) Fix BigInt, BigDec, Date serialization. Improve perf of PigNullableWritable deserilization

2013-09-24 Thread Dmitriy V. Ryaboy (JIRA)

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

Dmitriy V. Ryaboy updated PIG-3479:
---

Attachment: PIG-3479.whitespace.patch

Same patch, but with whitespace changes. Committing this.

> Fix BigInt, BigDec, Date serialization. Improve perf of PigNullableWritable 
> deserilization
> --
>
> Key: PIG-3479
> URL: https://issues.apache.org/jira/browse/PIG-3479
> Project: Pig
>  Issue Type: Bug
>Reporter: Dmitriy V. Ryaboy
>Assignee: Dmitriy V. Ryaboy
> Fix For: 0.12.0
>
> Attachments: PIG-3479.patch, PIG-3479.whitespace.patch
>
>
> While working on something unrelated I discovered some serialization errors 
> with recently added data types, and a heavy use of reflection slowing down 
> PigNullableWritable deserialization.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3458) ScalarExpression lost with multiquery optimization

2013-09-24 Thread Daniel Dai (JIRA)

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

Daniel Dai commented on PIG-3458:
-

+1. Please commit to trunk and 0.12 branch.

> ScalarExpression lost with multiquery optimization
> --
>
> Key: PIG-3458
> URL: https://issues.apache.org/jira/browse/PIG-3458
> Project: Pig
>  Issue Type: Bug
>Reporter: Koji Noguchi
>Assignee: Koji Noguchi
> Attachments: pig-3458-v01.patch, pig-3458-v02.patch
>
>
> Our user reported an issue where their scalar results goes missing when 
> having two store statements.
> {noformat}
> A = load 'test1.txt' using PigStorage('\t') as (a:chararray, count:long);
> B = group A all;
> C = foreach B generate SUM(A.count) as total ;
> store C into 'deleteme6_C' using PigStorage(',');
> Z = load 'test2.txt' using PigStorage('\t') as (a:chararray, id:chararray );
> Y = group Z by id;
> X = foreach Y generate group, C.total;
> store X into 'deleteme6_X' using PigStorage(',');
> Inputs
>  pig> cat test1.txt
> a   1
> b   2
> c   8
> d   9
>  pig> cat test2.txt
> a   z
> b   y
> c   x
>  pig>
> {noformat}
> Result X should contain the total count of '20' but instead it's empty.
> {noformat}
>  pig> cat deleteme6_C/part-r-0
> 20
>  pig> cat deleteme6_X/part-r-0
> x,
> y,
> z,
>  pig>
> {noformat}
> This works if we take out first "store C" statement.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (PIG-3458) ScalarExpression lost with multiquery optimization

2013-09-24 Thread Daniel Dai (JIRA)

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

Daniel Dai updated PIG-3458:


Fix Version/s: 0.12

> ScalarExpression lost with multiquery optimization
> --
>
> Key: PIG-3458
> URL: https://issues.apache.org/jira/browse/PIG-3458
> Project: Pig
>  Issue Type: Bug
>Reporter: Koji Noguchi
>Assignee: Koji Noguchi
> Fix For: 0.12
>
> Attachments: pig-3458-v01.patch, pig-3458-v02.patch
>
>
> Our user reported an issue where their scalar results goes missing when 
> having two store statements.
> {noformat}
> A = load 'test1.txt' using PigStorage('\t') as (a:chararray, count:long);
> B = group A all;
> C = foreach B generate SUM(A.count) as total ;
> store C into 'deleteme6_C' using PigStorage(',');
> Z = load 'test2.txt' using PigStorage('\t') as (a:chararray, id:chararray );
> Y = group Z by id;
> X = foreach Y generate group, C.total;
> store X into 'deleteme6_X' using PigStorage(',');
> Inputs
>  pig> cat test1.txt
> a   1
> b   2
> c   8
> d   9
>  pig> cat test2.txt
> a   z
> b   y
> c   x
>  pig>
> {noformat}
> Result X should contain the total count of '20' but instead it's empty.
> {noformat}
>  pig> cat deleteme6_C/part-r-0
> 20
>  pig> cat deleteme6_X/part-r-0
> x,
> y,
> z,
>  pig>
> {noformat}
> This works if we take out first "store C" statement.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3480) TFile-based tmpfile compression crashes in some cases

2013-09-24 Thread Rohini Palaniswamy (JIRA)

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

Rohini Palaniswamy commented on PIG-3480:
-

We do have complex types like bag of maps and bag of bags and one or two levels 
of nesting. But I assume you have way more nesting than we do. Does that matter 
though as what is written to TFile is just bytes for both key and value?

We use lzo. It would be good to try gz and see if the problem is with lzo for 
you. 

2013-09-24 21:10:21,289 INFO [main] 
com.hadoop.compression.lzo.GPLNativeCodeLoader: Loaded native gpl library
2013-09-24 21:10:21,291 INFO [main] com.hadoop.compression.lzo.LzoCodec: 
Successfully loaded & initialized native-lzo library
2013-09-24 21:10:21,293 INFO [main] org.apache.hadoop.io.compress.CodecPool: 
Got brand-new compressor [.lzo_deflate]

I don't think hadoop version should matter as we had hadoop 1.x till mid 2012. 

> TFile-based tmpfile compression crashes in some cases
> -
>
> Key: PIG-3480
> URL: https://issues.apache.org/jira/browse/PIG-3480
> Project: Pig
>  Issue Type: Bug
>Reporter: Dmitriy V. Ryaboy
> Fix For: 0.12
>
> Attachments: PIG-3480.patch
>
>
> When pig tmpfile compression is on, some jobs fail inside core hadoop 
> internals.
> Suspect TFile is the problem, because an experiment in replacing TFile with 
> SequenceFile succeeded.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3445) Make Parquet format available out of the box in Pig

2013-09-24 Thread Daniel Dai (JIRA)

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

Daniel Dai commented on PIG-3445:
-

Size maybe one thing, but still, doing a favor for Parquet sounds unfair to 
other loaders. Is it possible to push the jar dependency logic into LoadFunc, 
only shipping jar to backend when use the LoadFunc.

> Make Parquet format available out of the box in Pig
> ---
>
> Key: PIG-3445
> URL: https://issues.apache.org/jira/browse/PIG-3445
> Project: Pig
>  Issue Type: Improvement
>Reporter: Julien Le Dem
> Fix For: 0.12
>
> Attachments: PIG-3445.patch
>
>
> We would add the Parquet jar in the Pig packages to make it available out of 
> the box to pig users.
> On top of that we could add the parquet.pig package to the list of packages 
> to search for UDFs. (alternatively, the parquet jar could contain classes 
> name or.apache.pig.builtin.ParquetLoader and ParquetStorer)
> This way users can use Parquet simply by typing:
> A = LOAD 'foo' USING ParquetLoader();
> STORE A INTO 'bar' USING ParquetStorer();

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3479) Fix BigInt, BigDec, Date serialization. Improve perf of PigNullableWritable deserilization

2013-09-24 Thread Jonathan Coveney (JIRA)

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

Jonathan Coveney commented on PIG-3479:
---

+1

> Fix BigInt, BigDec, Date serialization. Improve perf of PigNullableWritable 
> deserilization
> --
>
> Key: PIG-3479
> URL: https://issues.apache.org/jira/browse/PIG-3479
> Project: Pig
>  Issue Type: Bug
>Reporter: Dmitriy V. Ryaboy
>Assignee: Dmitriy V. Ryaboy
> Fix For: 0.12
>
> Attachments: PIG-3479.patch
>
>
> While working on something unrelated I discovered some serialization errors 
> with recently added data types, and a heavy use of reflection slowing down 
> PigNullableWritable deserialization.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3480) TFile-based tmpfile compression crashes in some cases

2013-09-24 Thread Olga Natkovich (JIRA)

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

Olga Natkovich commented on PIG-3480:
-

Could this be related to Hadoop version? 

> TFile-based tmpfile compression crashes in some cases
> -
>
> Key: PIG-3480
> URL: https://issues.apache.org/jira/browse/PIG-3480
> Project: Pig
>  Issue Type: Bug
>Reporter: Dmitriy V. Ryaboy
> Fix For: 0.12
>
> Attachments: PIG-3480.patch
>
>
> When pig tmpfile compression is on, some jobs fail inside core hadoop 
> internals.
> Suspect TFile is the problem, because an experiment in replacing TFile with 
> SequenceFile succeeded.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3480) TFile-based tmpfile compression crashes in some cases

2013-09-24 Thread Dmitriy V. Ryaboy (JIRA)

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

Dmitriy V. Ryaboy commented on PIG-3480:


Rohini I suspect this might be something about complex data types, which afaik 
are pretty rare at Y! and extremely common at Twitter.

> TFile-based tmpfile compression crashes in some cases
> -
>
> Key: PIG-3480
> URL: https://issues.apache.org/jira/browse/PIG-3480
> Project: Pig
>  Issue Type: Bug
>Reporter: Dmitriy V. Ryaboy
> Fix For: 0.12
>
> Attachments: PIG-3480.patch
>
>
> When pig tmpfile compression is on, some jobs fail inside core hadoop 
> internals.
> Suspect TFile is the problem, because an experiment in replacing TFile with 
> SequenceFile succeeded.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3480) TFile-based tmpfile compression crashes in some cases

2013-09-24 Thread Dmitriy V. Ryaboy (JIRA)

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

Dmitriy V. Ryaboy commented on PIG-3480:


Rohini, do you guys use lzo or gz compression? Maybe it's just lzo that's 
breaking. I can test gz. That never actually occurred to me, I just assumed 
this is completely busted because I could never get it to work (since 2010..)

> TFile-based tmpfile compression crashes in some cases
> -
>
> Key: PIG-3480
> URL: https://issues.apache.org/jira/browse/PIG-3480
> Project: Pig
>  Issue Type: Bug
>Reporter: Dmitriy V. Ryaboy
> Fix For: 0.12
>
> Attachments: PIG-3480.patch
>
>
> When pig tmpfile compression is on, some jobs fail inside core hadoop 
> internals.
> Suspect TFile is the problem, because an experiment in replacing TFile with 
> SequenceFile succeeded.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (PIG-3360) Some intermittent negative e2e tests fail on hadoop 2

2013-09-24 Thread Daniel Dai (JIRA)

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

Daniel Dai updated PIG-3360:


Attachment: PIG-3360-2.patch

The new patch put the logic in shims layer. The API TaskReport.getCurrentStatus 
is introduced in 0.21. However, we only see issues with the getProgress trick 
in Hadoop 2. So putting in the shims should be fine and avoid reflection which 
makes code ugly.

> Some intermittent negative e2e tests fail on hadoop 2
> -
>
> Key: PIG-3360
> URL: https://issues.apache.org/jira/browse/PIG-3360
> Project: Pig
>  Issue Type: Bug
>  Components: impl
>Reporter: Daniel Dai
>Assignee: Daniel Dai
> Fix For: 0.12
>
> Attachments: PIG-3360-1.patch, PIG-3360-2.patch
>
>
> One example is StreamingErrors_2. Here is the stack we get:
> Backend error message
> -
> Error: org.apache.pig.backend.executionengine.ExecException: ERROR 2055: 
> Received Error while processing the map plan: 'perl PigStreamingBad.pl middle 
> ' failed with exit status: 2
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.runPipeline(PigGenericMapBase.java:311)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:278)
>   at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:64)
>   at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:145)
>   at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:763)
>   at org.apache.hadoop.mapred.MapTask.run(MapTask.java:339)
>   at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:162)
>   at java.security.AccessController.doPrivileged(Native Method)
>   at javax.security.auth.Subject.doAs(Subject.java:396)
>   at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1477)
>   at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:157)
> Pig Stack Trace
> ---
> ERROR 2244: Job failed, hadoop does not return any error message
> org.apache.pig.backend.executionengine.ExecException: ERROR 2244: Job failed, 
> hadoop does not return any error message
>   at 
> org.apache.pig.tools.grunt.GruntParser.executeBatch(GruntParser.java:145)
>   at 
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:198)
>   at 
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:170)
>   at org.apache.pig.tools.grunt.Grunt.exec(Grunt.java:84)
>   at org.apache.pig.Main.run(Main.java:604)
>   at org.apache.pig.Main.main(Main.java:157)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>   at java.lang.reflect.Method.invoke(Method.java:597)
>   at org.apache.hadoop.util.RunJar.main(RunJar.java:212)
> 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3480) TFile-based tmpfile compression crashes in some cases

2013-09-24 Thread Rohini Palaniswamy (JIRA)

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

Rohini Palaniswamy commented on PIG-3480:
-

[~dvryaboy],
   We have been running it with pig.tmpfilecompression=true and 
pig.tmpfilecompression.codec=lzo as defaults from 2010 and have not encountered 
any issues so far. The problem might be elsewhere and TFile might not be the 
issue.

> TFile-based tmpfile compression crashes in some cases
> -
>
> Key: PIG-3480
> URL: https://issues.apache.org/jira/browse/PIG-3480
> Project: Pig
>  Issue Type: Bug
>Reporter: Dmitriy V. Ryaboy
> Fix For: 0.12
>
> Attachments: PIG-3480.patch
>
>
> When pig tmpfile compression is on, some jobs fail inside core hadoop 
> internals.
> Suspect TFile is the problem, because an experiment in replacing TFile with 
> SequenceFile succeeded.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3480) TFile-based tmpfile compression crashes in some cases

2013-09-24 Thread Dmitriy V. Ryaboy (JIRA)

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

Dmitriy V. Ryaboy commented on PIG-3480:


[~knoguchi] yeah, I'm not sure the stack trace is relevant -- it's the only 
part that's not consistent about this.

The problem goes away when I set pig.tmpfilecompression to false, or when I 
replace TFile with SequenceFile.
I've also seen stack traces that were inside TFile, and had to do with some LZO 
decoding issues.. the actual error is really hard to capture, other than the 
fact that mappers fail consistently.

> TFile-based tmpfile compression crashes in some cases
> -
>
> Key: PIG-3480
> URL: https://issues.apache.org/jira/browse/PIG-3480
> Project: Pig
>  Issue Type: Bug
>Reporter: Dmitriy V. Ryaboy
> Fix For: 0.12
>
> Attachments: PIG-3480.patch
>
>
> When pig tmpfile compression is on, some jobs fail inside core hadoop 
> internals.
> Suspect TFile is the problem, because an experiment in replacing TFile with 
> SequenceFile succeeded.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3480) TFile-based tmpfile compression crashes in some cases

2013-09-24 Thread Koji Noguchi (JIRA)

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

Koji Noguchi commented on PIG-3480:
---

Dmitriy, isn't your stacktrace failing at mapred.IFile and not TFile? 

> This problem does happen consistently – 100% of the time on my script that 
> shows this problem. 
>
And this problem goes away once tmpcompression is turned off? 
(pig.tmpfilecompression=false)

> TFile-based tmpfile compression crashes in some cases
> -
>
> Key: PIG-3480
> URL: https://issues.apache.org/jira/browse/PIG-3480
> Project: Pig
>  Issue Type: Bug
>Reporter: Dmitriy V. Ryaboy
> Fix For: 0.12
>
> Attachments: PIG-3480.patch
>
>
> When pig tmpfile compression is on, some jobs fail inside core hadoop 
> internals.
> Suspect TFile is the problem, because an experiment in replacing TFile with 
> SequenceFile succeeded.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3461) Rewrite PartitionFilterOptimizer to make it work for all the cases

2013-09-24 Thread Aniket Mokashi (JIRA)

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

Aniket Mokashi commented on PIG-3461:
-

Committed to trunk and 0.12 branch. Thanks again [~cheolsoo] for reviewing!

> Rewrite PartitionFilterOptimizer to make it work for all the cases
> --
>
> Key: PIG-3461
> URL: https://issues.apache.org/jira/browse/PIG-3461
> Project: Pig
>  Issue Type: Bug
>  Components: impl
>Affects Versions: 0.11.1
>Reporter: Aniket Mokashi
>Assignee: Aniket Mokashi
> Fix For: 0.12
>
> Attachments: PIG-3461-2.patch, PIG-3461-4.patch, PIG-3461-6.patch
>
>
> Current algorithm for Partition Filter pushdown identification fails in 
> several corner cases. We need to rewrite its logic so that it works in all 
> cases and does the maximum possible filter pushdown.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (PIG-3461) Rewrite PartitionFilterOptimizer to make it work for all the cases

2013-09-24 Thread Aniket Mokashi (JIRA)

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

Aniket Mokashi updated PIG-3461:


Resolution: Fixed
Status: Resolved  (was: Patch Available)

> Rewrite PartitionFilterOptimizer to make it work for all the cases
> --
>
> Key: PIG-3461
> URL: https://issues.apache.org/jira/browse/PIG-3461
> Project: Pig
>  Issue Type: Bug
>  Components: impl
>Affects Versions: 0.11.1
>Reporter: Aniket Mokashi
>Assignee: Aniket Mokashi
> Fix For: 0.12
>
> Attachments: PIG-3461-2.patch, PIG-3461-4.patch, PIG-3461-6.patch
>
>
> Current algorithm for Partition Filter pushdown identification fails in 
> several corner cases. We need to rewrite its logic so that it works in all 
> cases and does the maximum possible filter pushdown.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (PIG-3480) TFile-based tmpfile compression crashes in some cases

2013-09-24 Thread Dmitriy V. Ryaboy (JIRA)

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

Dmitriy V. Ryaboy updated PIG-3480:
---

Attachment: PIG-3480.patch

Attaching a rough patch which replaces use of TFile with SequenceFile.

Next steps:
- evaluate effect on size of compressed data for TFile vs SeqFile when TFile 
does work
- add tests, make TFile tests pass (in this file they fail, because of course 
TFile is not being used)
- make SeqFile the default method, since it doesn't break
- allow TFile use by a switch, since current users may want to keep it. I would 
prefer to not do that, but might if the first step shows significant 
differences.

Thoughts?
Especially from folks using TFile-based compression in production ([~rohini]?)

> TFile-based tmpfile compression crashes in some cases
> -
>
> Key: PIG-3480
> URL: https://issues.apache.org/jira/browse/PIG-3480
> Project: Pig
>  Issue Type: Bug
>Reporter: Dmitriy V. Ryaboy
> Fix For: 0.12
>
> Attachments: PIG-3480.patch
>
>
> When pig tmpfile compression is on, some jobs fail inside core hadoop 
> internals.
> Suspect TFile is the problem, because an experiment in replacing TFile with 
> SequenceFile succeeded.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Comment Edited] (PIG-3480) TFile-based tmpfile compression crashes in some cases

2013-09-24 Thread Dmitriy V. Ryaboy (JIRA)

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

Dmitriy V. Ryaboy edited comment on PIG-3480 at 9/24/13 6:36 PM:
-

For most of the tasks that fail, no stack trace is available on Hadoop 1 (they 
just die with "nonzero status 134").

I did catch one task with a stack trace:
{code}
java.io.IOException: Error while reading compressed data at
org.apache.hadoop.io.IOUtils.wrappedReadForCompressedData(IOUtils.java:205) at 
org.apache.hadoop.mapred.IFile$Reader.readData(IFile.java:342) at 
org.apache.hadoop.mapred.IFile$Reader.rejigData(IFile.java:373) at 
org.apache.hadoop.mapred.IFile$Reader.readNextBlock(IFile.java:357) at 
org.apache.hadoop.mapred.IFile$Reader.next(IFile.java:389) at 
org.apache.hadoop.mapred.Merger$Segment.next(Merger.java:220) at 
org.apache.hadoop.mapred.Merger$MergeQueue.merge(Merger.java:420) at 
org.apache.hadoop.mapred.Merger$MergeQueue.merge(Merger.java:381) at 
org.apache.hadoop.mapred.Merger.merge(Merger.java:77) at 
org.apache.hadoop.mapred.MapTask$MapOutputBuffer.mergeParts(MapTask.java:1548) 
at 
org.apache.hadoop.mapred.MapTask$MapOutputBuffer.flush(MapTask.java:1180) at 
org.apache.hadoop.mapred.MapTask$NewOutputCollector.close(MapTask.java:582) at 
org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:649) at 
org.apache.hadoop.mapred.MapTask.run(Map
{code}

No idea if this is relevant.

This problem does happen consistently -- 100% of the time on my script that 
shows this problem. Anecdotally, about 1/10 of our production scripts encounter 
this; I have not been able to establish a pattern yet.

  was (Author: dvryaboy):
For most of the tasks that fail, no stack trace is available on Hadoop 1 
(they just die with "nonzero status 134").

I did catch one task with a stack trace:
{code}
java.io.IOException: Error while reading compressed data at 
org.apache.hadoop.io.IOUtils.wrappedReadForCompressedData(IOUtils.java:205) at 
org.apache.hadoop.mapred.IFile$Reader.readData(IFile.java:342) at 
org.apache.hadoop.mapred.IFile$Reader.rejigData(IFile.java:373) at 
org.apache.hadoop.mapred.IFile$Reader.readNextBlock(IFile.java:357) at 
org.apache.hadoop.mapred.IFile$Reader.next(IFile.java:389) at 
org.apache.hadoop.mapred.Merger$Segment.next(Merger.java:220) at 
org.apache.hadoop.mapred.Merger$MergeQueue.merge(Merger.java:420) at 
org.apache.hadoop.mapred.Merger$MergeQueue.merge(Merger.java:381) at 
org.apache.hadoop.mapred.Merger.merge(Merger.java:77) at 
org.apache.hadoop.mapred.MapTask$MapOutputBuffer.mergeParts(MapTask.java:1548) 
at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.flush(MapTask.java:1180) at 
org.apache.hadoop.mapred.MapTask$NewOutputCollector.close(MapTask.java:582) at 
org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:649) at 
org.apache.hadoop.mapred.MapTask.run(Map
{code}

No idea if this is relevant.

This problem does happen consistently -- 100% of the time on my script that 
shows this problem. Anecdotally, about 1/10 of our production scripts encounter 
this; I have not been able to establish a pattern yet.
  
> TFile-based tmpfile compression crashes in some cases
> -
>
> Key: PIG-3480
> URL: https://issues.apache.org/jira/browse/PIG-3480
> Project: Pig
>  Issue Type: Bug
>Reporter: Dmitriy V. Ryaboy
> Fix For: 0.12
>
>
> When pig tmpfile compression is on, some jobs fail inside core hadoop 
> internals.
> Suspect TFile is the problem, because an experiment in replacing TFile with 
> SequenceFile succeeded.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3480) TFile-based tmpfile compression crashes in some cases

2013-09-24 Thread Dmitriy V. Ryaboy (JIRA)

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

Dmitriy V. Ryaboy commented on PIG-3480:


For most of the tasks that fail, no stack trace is available on Hadoop 1 (they 
just die with "nonzero status 134").

I did catch one task with a stack trace:
{code}
java.io.IOException: Error while reading compressed data at 
org.apache.hadoop.io.IOUtils.wrappedReadForCompressedData(IOUtils.java:205) at 
org.apache.hadoop.mapred.IFile$Reader.readData(IFile.java:342) at 
org.apache.hadoop.mapred.IFile$Reader.rejigData(IFile.java:373) at 
org.apache.hadoop.mapred.IFile$Reader.readNextBlock(IFile.java:357) at 
org.apache.hadoop.mapred.IFile$Reader.next(IFile.java:389) at 
org.apache.hadoop.mapred.Merger$Segment.next(Merger.java:220) at 
org.apache.hadoop.mapred.Merger$MergeQueue.merge(Merger.java:420) at 
org.apache.hadoop.mapred.Merger$MergeQueue.merge(Merger.java:381) at 
org.apache.hadoop.mapred.Merger.merge(Merger.java:77) at 
org.apache.hadoop.mapred.MapTask$MapOutputBuffer.mergeParts(MapTask.java:1548) 
at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.flush(MapTask.java:1180) at 
org.apache.hadoop.mapred.MapTask$NewOutputCollector.close(MapTask.java:582) at 
org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:649) at 
org.apache.hadoop.mapred.MapTask.run(Map
{code}

No idea if this is relevant.

This problem does happen consistently -- 100% of the time on my script that 
shows this problem. Anecdotally, about 1/10 of our production scripts encounter 
this; I have not been able to establish a pattern yet.

> TFile-based tmpfile compression crashes in some cases
> -
>
> Key: PIG-3480
> URL: https://issues.apache.org/jira/browse/PIG-3480
> Project: Pig
>  Issue Type: Bug
>Reporter: Dmitriy V. Ryaboy
> Fix For: 0.12
>
>
> When pig tmpfile compression is on, some jobs fail inside core hadoop 
> internals.
> Suspect TFile is the problem, because an experiment in replacing TFile with 
> SequenceFile succeeded.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Created] (PIG-3480) TFile-based tmpfile compression crashes in some cases

2013-09-24 Thread Dmitriy V. Ryaboy (JIRA)
Dmitriy V. Ryaboy created PIG-3480:
--

 Summary: TFile-based tmpfile compression crashes in some cases
 Key: PIG-3480
 URL: https://issues.apache.org/jira/browse/PIG-3480
 Project: Pig
  Issue Type: Bug
Reporter: Dmitriy V. Ryaboy
 Fix For: 0.12


When pig tmpfile compression is on, some jobs fail inside core hadoop internals.
Suspect TFile is the problem, because an experiment in replacing TFile with 
SequenceFile succeeded.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-2672) Optimize the use of DistributedCache

2013-09-24 Thread Koji Noguchi (JIRA)

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

Koji Noguchi commented on PIG-2672:
---

On the secure hadoop environment, this patch would basically create a hole and 
allow any users having write access to PIG_SHARED_CLUSTER_CACHE_LOCATION to  
become other users (who are sharing this cache location).

For now, can we instead limit the patch to /user//.pig or .staging 
add have extra check on permission 700 ?

I understand that you can make PIG_SHARED_CLUSTER_CACHE_LOCATION only writable 
by admin but I'm afraid this patch would make it too easy to misconfigure.

> Optimize the use of DistributedCache
> 
>
> Key: PIG-2672
> URL: https://issues.apache.org/jira/browse/PIG-2672
> Project: Pig
>  Issue Type: Improvement
>Reporter: Rohini Palaniswamy
>Assignee: Aniket Mokashi
> Fix For: 0.12
>
> Attachments: PIG-2672.patch
>
>
> Pig currently copies jar files to a temporary location in hdfs and then adds 
> them to DistributedCache for each job launched. This is inefficient in terms 
> of 
>* Space - The jars are distributed to task trackers for every job taking 
> up lot of local temporary space in tasktrackers.
>* Performance - The jar distribution impacts the job launch time.  

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-2672) Optimize the use of DistributedCache

2013-09-24 Thread Cheolsoo Park (JIRA)

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

Cheolsoo Park commented on PIG-2672:


Forgot to mention. [~aniket486], can you please document this? Perhaps in the 
[performance and efficiency page|http://pig.apache.org/docs/r0.11.0/perf.html]?

> Optimize the use of DistributedCache
> 
>
> Key: PIG-2672
> URL: https://issues.apache.org/jira/browse/PIG-2672
> Project: Pig
>  Issue Type: Improvement
>Reporter: Rohini Palaniswamy
>Assignee: Aniket Mokashi
> Fix For: 0.12
>
> Attachments: PIG-2672.patch
>
>
> Pig currently copies jar files to a temporary location in hdfs and then adds 
> them to DistributedCache for each job launched. This is inefficient in terms 
> of 
>* Space - The jars are distributed to task trackers for every job taking 
> up lot of local temporary space in tasktrackers.
>* Performance - The jar distribution impacts the job launch time.  

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (PIG-3445) Make Parquet format available out of the box in Pig

2013-09-24 Thread Dmitriy V. Ryaboy (JIRA)

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

Dmitriy V. Ryaboy updated PIG-3445:
---

Fix Version/s: 0.12

> Make Parquet format available out of the box in Pig
> ---
>
> Key: PIG-3445
> URL: https://issues.apache.org/jira/browse/PIG-3445
> Project: Pig
>  Issue Type: Improvement
>Reporter: Julien Le Dem
> Fix For: 0.12
>
> Attachments: PIG-3445.patch
>
>
> We would add the Parquet jar in the Pig packages to make it available out of 
> the box to pig users.
> On top of that we could add the parquet.pig package to the list of packages 
> to search for UDFs. (alternatively, the parquet jar could contain classes 
> name or.apache.pig.builtin.ParquetLoader and ParquetStorer)
> This way users can use Parquet simply by typing:
> A = LOAD 'foo' USING ParquetLoader();
> STORE A INTO 'bar' USING ParquetStorer();

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (PIG-3445) Make Parquet format available out of the box in Pig

2013-09-24 Thread Dmitriy V. Ryaboy (JIRA)

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

Dmitriy V. Ryaboy commented on PIG-3445:


The size of the dependency introduced by this is orders of magnitude smaller 
than the HBase (or Avro) one, since everything comes from a single project 
(unlike HBase's liberal use of guava, metric, ZK, and everything else under the 
sun). The total size is less than 1 meg.

Can we add parquet.pig to udf import list in the same patch?

> Make Parquet format available out of the box in Pig
> ---
>
> Key: PIG-3445
> URL: https://issues.apache.org/jira/browse/PIG-3445
> Project: Pig
>  Issue Type: Improvement
>Reporter: Julien Le Dem
> Attachments: PIG-3445.patch
>
>
> We would add the Parquet jar in the Pig packages to make it available out of 
> the box to pig users.
> On top of that we could add the parquet.pig package to the list of packages 
> to search for UDFs. (alternatively, the parquet jar could contain classes 
> name or.apache.pig.builtin.ParquetLoader and ParquetStorer)
> This way users can use Parquet simply by typing:
> A = LOAD 'foo' USING ParquetLoader();
> STORE A INTO 'bar' USING ParquetStorer();

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (PIG-3325) Adding a tuple to a bag is slow

2013-09-24 Thread Dmitriy V. Ryaboy (JIRA)

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

Dmitriy V. Ryaboy updated PIG-3325:
---

Fix Version/s: 0.12

> Adding a tuple to a bag is slow
> ---
>
> Key: PIG-3325
> URL: https://issues.apache.org/jira/browse/PIG-3325
> Project: Pig
>  Issue Type: Bug
>Affects Versions: 0.11, 0.12, 0.11.1, 0.11.2
>Reporter: Mark Wagner
>Assignee: Dmitriy V. Ryaboy
>Priority: Critical
> Fix For: 0.12
>
> Attachments: PIG-3325.2.patch, PIG-3325.3.patch, PIG-3325.demo.patch, 
> PIG-3325.optimize.1.patch
>
>
> The time it takes to add a tuple to a bag has increased significantly, 
> causing some jobs to take about 50x longer compared to 0.10.1. I've tracked 
> this down to PIG-2923, which has made adding a tuple heavier weight (it now 
> includes some memory estimation).

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (PIG-3479) Fix BigInt, BigDec, Date serialization. Improve perf of PigNullableWritable deserilization

2013-09-24 Thread Dmitriy V. Ryaboy (JIRA)

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

Dmitriy V. Ryaboy updated PIG-3479:
---

Fix Version/s: 0.12

> Fix BigInt, BigDec, Date serialization. Improve perf of PigNullableWritable 
> deserilization
> --
>
> Key: PIG-3479
> URL: https://issues.apache.org/jira/browse/PIG-3479
> Project: Pig
>  Issue Type: Bug
>Reporter: Dmitriy V. Ryaboy
>Assignee: Dmitriy V. Ryaboy
> Fix For: 0.12
>
> Attachments: PIG-3479.patch
>
>
> While working on something unrelated I discovered some serialization errors 
> with recently added data types, and a heavy use of reflection slowing down 
> PigNullableWritable deserialization.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (PIG-3325) Adding a tuple to a bag is slow

2013-09-24 Thread Dmitriy V. Ryaboy (JIRA)

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

Dmitriy V. Ryaboy updated PIG-3325:
---

Affects Version/s: 0.12

> Adding a tuple to a bag is slow
> ---
>
> Key: PIG-3325
> URL: https://issues.apache.org/jira/browse/PIG-3325
> Project: Pig
>  Issue Type: Bug
>Affects Versions: 0.11, 0.12, 0.11.1, 0.11.2
>Reporter: Mark Wagner
>Assignee: Dmitriy V. Ryaboy
>Priority: Critical
> Attachments: PIG-3325.2.patch, PIG-3325.3.patch, PIG-3325.demo.patch, 
> PIG-3325.optimize.1.patch
>
>
> The time it takes to add a tuple to a bag has increased significantly, 
> causing some jobs to take about 50x longer compared to 0.10.1. I've tracked 
> this down to PIG-2923, which has made adding a tuple heavier weight (it now 
> includes some memory estimation).

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira