Pig Contributor meeting notes
Twitter hosted this month's Pig contributor meeting. Developers from Yahoo, Twitter, LinkedIn, RichRelevance, and Cloudera were present. 1. Howl First, Alan Gates demoed Howl, a project whose goal is to provide table management service for all of hadoop. The vision is that ultimately you will be able to read/write data using regular MR, or Pig, or Hive, and read it using any of those three, with full support of a partition-aware metadata store that will tell you what data is available, what its schema is, etc, reusing a single table abstraction. Currently, tables are created using (a restricted subset of) Hive ddl statements; a howl cli for this will be created, which will enforce the restricted subset. Writing to the table using Pig or MapReduce is supported. Reading can already be done using all three. At the moment, a single Pig store statement can only store into a single partition; adding ability to "spray" across partitions is on the roadmap. This, and a good api for interacting with the metastore, are the two areas that were identified as good opportunities for the wider developer community to get involved with the project. The source code is on GitHub, and is at the moment synchronized with the development trunk manually; Yahoo folks will look into changing this. Security is a concern, and Yahoo will be working on it. Making it possible for Hive to write to the tables is at the moment not as high a priority as the others listed, it would basically involve just writing a Hive SerDe (an equivalent of Pig's StoreFunc). 2. Azkaban presentation Russel Jurney and Richard Park from LinkedIn presented the workflow management tool open-sourced by LinkedIn, called Azkaban. It allows you to declare job dependencies, has a web interface for launching and monitoring jobs, etc. It has a special exec mode for Pig that lets you set some Pig-specific options on a per-job basis. It does not currently have triggering or job-instance parameter substitution (it does have job-level parameter substitution). When asked what would Pig could do to make life easier for Azkaban, the two things Richard identified were registering jars through the grunt command line and a way to monitor the running job -- both of these are already in trunk, so we're in pretty good shaped for 0.8 3. Piggybank discussion Kevin Weil led a discussion of the piggybank. There are a few problems with it -- it's released on the Pig schedule, and has quite a few barriers to submission that are, anecdotally at least, preventing people from contributing. Several options were discussed, with the group finally settling on starting a community-curated GitHub project for piggybank. It will have a number of committers from different companies, and will aim to make it easy for folks to contribute (all contribs will still have to have tests, and be Apache 2.0-licensed). More details will be forthcoming as we figure them out. Initially this project will be seeded with the current Piggybank functions some time after 0.8 is branched. The initial list of committers Kevin Weil (Twitter), Dmitriy Ryaboy (Twitter), Carl Steinbach (Cloudera), and Russel Jurney (LinkedIn). Yahoo will also nominate someone. Please send us any thoughts you might have on this subject. It was suggested that a lot of common code might be shared with Hive UDFs, which have the same problems as Piggybank does, and that perhaps the project can be another collaboration point between the projects. Not clear how that would work, Carl will talk to other Hive people. Pig 0.9 So far the items on the list for 0.9 are: better type propagation / resolution story and documentation, perhaps different parser (ANTLR?), some performance tweaks, and map types with fixed-type values. Much still to be decided. The next contributor meeting will be hosted by LinkedIn in October. -Dmitriy
[jira] Updated: (PIG-1568) Optimization rule FilterAboveForeach is too restrictive and doesn't handle project * correctly
[ https://issues.apache.org/jira/browse/PIG-1568?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Xuefu Zhang updated PIG-1568: - Attachment: jira-1568-1.patch > Optimization rule FilterAboveForeach is too restrictive and doesn't handle > project * correctly > -- > > Key: PIG-1568 > URL: https://issues.apache.org/jira/browse/PIG-1568 > Project: Pig > Issue Type: Bug >Reporter: Xuefu Zhang >Assignee: Xuefu Zhang > Fix For: 0.8.0 > > Attachments: jira-1568-1.patch > > > FilterAboveForeach rule is to optimize the plan by pushing up filter above > previous foreach operator. However, during code review, two major problems > were found: > 1. Current implementation assumes that if no projection is found in the > filter condition then all columns from foreach are projected. This issue > prevents the following optimization: > A = LOAD 'file.txt' AS (a(u,v), b, c); > B = FOREACH A GENERATE $0, b; > C = FILTER B BY 8 > 5; > STORE C INTO 'empty'; > 2. Current implementation doesn't handle * probjection, which means project > all columns. As a result, it wasn't able to optimize the following: > A = LOAD 'file.txt' AS (a(u,v), b, c); > B = FOREACH A GENERATE $0, b; > C = FILTER B BY Identity.class.getName(*) > 5; > STORE C INTO 'empty'; -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1568) Optimization rule FilterAboveForeach is too restrictive and doesn't handle project * correctly
[ https://issues.apache.org/jira/browse/PIG-1568?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Xuefu Zhang updated PIG-1568: - Status: Patch Available (was: Open) > Optimization rule FilterAboveForeach is too restrictive and doesn't handle > project * correctly > -- > > Key: PIG-1568 > URL: https://issues.apache.org/jira/browse/PIG-1568 > Project: Pig > Issue Type: Bug >Reporter: Xuefu Zhang >Assignee: Xuefu Zhang > Fix For: 0.8.0 > > Attachments: jira-1568-1.patch > > > FilterAboveForeach rule is to optimize the plan by pushing up filter above > previous foreach operator. However, during code review, two major problems > were found: > 1. Current implementation assumes that if no projection is found in the > filter condition then all columns from foreach are projected. This issue > prevents the following optimization: > A = LOAD 'file.txt' AS (a(u,v), b, c); > B = FOREACH A GENERATE $0, b; > C = FILTER B BY 8 > 5; > STORE C INTO 'empty'; > 2. Current implementation doesn't handle * probjection, which means project > all columns. As a result, it wasn't able to optimize the following: > A = LOAD 'file.txt' AS (a(u,v), b, c); > B = FOREACH A GENERATE $0, b; > C = FILTER B BY Identity.class.getName(*) > 5; > STORE C INTO 'empty'; -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1518) multi file input format for loaders
[ https://issues.apache.org/jira/browse/PIG-1518?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yan Zhou updated PIG-1518: -- Status: Open (was: Patch Available) > multi file input format for loaders > --- > > Key: PIG-1518 > URL: https://issues.apache.org/jira/browse/PIG-1518 > Project: Pig > Issue Type: Improvement >Reporter: Olga Natkovich >Assignee: Yan Zhou > Fix For: 0.8.0 > > Attachments: PIG-1518.patch, PIG-1518.patch, PIG-1518.patch, > PIG-1518.patch, PIG-1518.patch, PIG-1518.patch > > > We frequently run in the situation where Pig needs to deal with small files > in the input. In this case a separate map is created for each file which > could be very inefficient. > It would be greate to have an umbrella input format that can take multiple > files and use them in a single split. We would like to see this working with > different data formats if possible. > There are already a couple of input formats doing similar thing: > MultifileInputFormat as well as CombinedInputFormat; howevere, neither works > with ne Hadoop 20 API. > We at least want to do a feasibility study for Pig 0.8.0. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1518) multi file input format for loaders
[ https://issues.apache.org/jira/browse/PIG-1518?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yan Zhou updated PIG-1518: -- Attachment: PIG-1518.patch Improvement on logging info. > multi file input format for loaders > --- > > Key: PIG-1518 > URL: https://issues.apache.org/jira/browse/PIG-1518 > Project: Pig > Issue Type: Improvement >Reporter: Olga Natkovich >Assignee: Yan Zhou > Fix For: 0.8.0 > > Attachments: PIG-1518.patch, PIG-1518.patch, PIG-1518.patch, > PIG-1518.patch, PIG-1518.patch, PIG-1518.patch > > > We frequently run in the situation where Pig needs to deal with small files > in the input. In this case a separate map is created for each file which > could be very inefficient. > It would be greate to have an umbrella input format that can take multiple > files and use them in a single split. We would like to see this working with > different data formats if possible. > There are already a couple of input formats doing similar thing: > MultifileInputFormat as well as CombinedInputFormat; howevere, neither works > with ne Hadoop 20 API. > We at least want to do a feasibility study for Pig 0.8.0. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1564) add support for multiple filesystems
[ https://issues.apache.org/jira/browse/PIG-1564?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12902720#action_12902720 ] Andrew Hitchcock commented on PIG-1564: --- Hi Jeff, Before I add a unit test I'd like confirmation that I'm going about this the right way. In my previous patch (PIG-1505) it was mentioned that HDataStorage was deprecated and this patch has some changes to HDataStorage and related classes. When is the planned deprecation for HDataStorage and is there anything that needs to be modified in addition to HDataStorage for this to work? I'd also note that this is a trunk rebase of a patch that we currently have in production with Pig 0.3 and Pig 0.6. Thanks, Andrew > add support for multiple filesystems > > > Key: PIG-1564 > URL: https://issues.apache.org/jira/browse/PIG-1564 > Project: Pig > Issue Type: Improvement >Reporter: Andrew Hitchcock > Attachments: PIG-1564-1.patch > > > Currently you can't run Pig scripts that read data from one file system and > write it to another. Also, Grunt doesn't support CDing from one directory to > another on different file systems. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1399) Logical Optimizer: Expression optimizor rule
[ https://issues.apache.org/jira/browse/PIG-1399?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yan Zhou updated PIG-1399: -- Attachment: PIG-1399.patch rebasing on the latest trunk > Logical Optimizer: Expression optimizor rule > > > Key: PIG-1399 > URL: https://issues.apache.org/jira/browse/PIG-1399 > Project: Pig > Issue Type: Sub-task > Components: impl >Affects Versions: 0.7.0 >Reporter: Daniel Dai >Assignee: Yan Zhou > Fix For: 0.8.0 > > Attachments: PIG-1399.patch, PIG-1399.patch, PIG-1399.patch, > PIG-1399.patch > > > We can optimize expression in several ways: > 1. Constant pre-calculation > Example: > B = filter A by a0 > 5+7; > => B = filter A by a0 > 12; > 2. Boolean expression optimization > Example: > B = filter A by not (not(a0>5) or a>10); > => B = filter A by a0>5 and a<=10; -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Assigned: (PIG-1568) Optimization rule FilterAboveForeach is too restrictive and doesn't handle project * correctly
[ https://issues.apache.org/jira/browse/PIG-1568?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Xuefu Zhang reassigned PIG-1568: Assignee: Xuefu Zhang > Optimization rule FilterAboveForeach is too restrictive and doesn't handle > project * correctly > -- > > Key: PIG-1568 > URL: https://issues.apache.org/jira/browse/PIG-1568 > Project: Pig > Issue Type: Bug >Reporter: Xuefu Zhang >Assignee: Xuefu Zhang > Fix For: 0.8.0 > > > FilterAboveForeach rule is to optimize the plan by pushing up filter above > previous foreach operator. However, during code review, two major problems > were found: > 1. Current implementation assumes that if no projection is found in the > filter condition then all columns from foreach are projected. This issue > prevents the following optimization: > A = LOAD 'file.txt' AS (a(u,v), b, c); > B = FOREACH A GENERATE $0, b; > C = FILTER B BY 8 > 5; > STORE C INTO 'empty'; > 2. Current implementation doesn't handle * probjection, which means project > all columns. As a result, it wasn't able to optimize the following: > A = LOAD 'file.txt' AS (a(u,v), b, c); > B = FOREACH A GENERATE $0, b; > C = FILTER B BY Identity.class.getName(*) > 5; > STORE C INTO 'empty'; -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Created: (PIG-1568) Optimization rule FilterAboveForeach is too restrictive and doesn't handle project * correctly
Optimization rule FilterAboveForeach is too restrictive and doesn't handle project * correctly -- Key: PIG-1568 URL: https://issues.apache.org/jira/browse/PIG-1568 Project: Pig Issue Type: Bug Reporter: Xuefu Zhang Fix For: 0.8.0 FilterAboveForeach rule is to optimize the plan by pushing up filter above previous foreach operator. However, during code review, two major problems were found: 1. Current implementation assumes that if no projection is found in the filter condition then all columns from foreach are projected. This issue prevents the following optimization: A = LOAD 'file.txt' AS (a(u,v), b, c); B = FOREACH A GENERATE $0, b; C = FILTER B BY 8 > 5; STORE C INTO 'empty'; 2. Current implementation doesn't handle * probjection, which means project all columns. As a result, it wasn't able to optimize the following: A = LOAD 'file.txt' AS (a(u,v), b, c); B = FOREACH A GENERATE $0, b; C = FILTER B BY Identity.class.getName(*) > 5; STORE C INTO 'empty'; -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Created: (PIG-1567) Optimization rule FilterAboveForeach is too restrictive and doesn't handle project * correctly
Optimization rule FilterAboveForeach is too restrictive and doesn't handle project * correctly -- Key: PIG-1567 URL: https://issues.apache.org/jira/browse/PIG-1567 Project: Pig Issue Type: Bug Reporter: Xuefu Zhang Fix For: 0.8.0 FilterAboveForeach rule is to optimize the plan by pushing up filter above previous foreach operator. However, during code review, two major problems were found: 1. Current implementation assumes that if no projection is found in the filter condition then all columns from foreach are projected. This issue prevents the following optimization: A = LOAD 'file.txt' AS (a(u,v), b, c); B = FOREACH A GENERATE $0, b; C = FILTER B BY 8 > 5; STORE C INTO 'empty'; 2. Current implementation doesn't handle * probjection, which means project all columns. As a result, it wasn't able to optimize the following: A = LOAD 'file.txt' AS (a(u,v), b, c); B = FOREACH A GENERATE $0, b; C = FILTER B BY Identity.class.getName(*) > 5; STORE C INTO 'empty'; -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1531) Pig gobbles up error messages
[ https://issues.apache.org/jira/browse/PIG-1531?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12902576#action_12902576 ] Ashutosh Chauhan commented on PIG-1531: --- * In addition to error Msg, you also need to set error code on the exception you are throwing. * Since you are catching exceptions thrown by user code (StoreFunc Interface) it is not safe to assume that e.getMessage() will be non-null or non-empty string. This will result in NPE. You need to check for it and provide a generic error Msg in those cases. * Generic error msg should also contain output location String. Since if user didnt provide it, that wont get printed. So, you can reword the message as "Output location validation failed for: . More Information to follow:" * Since, PigException extends from IOException. The IOException you are catching can also be a PigException, you need to test if it is and then set the message and error code. * In case of non-existent input location I am still seeing the generic message "ERROR 2997: Unable to recreate exception from backend error: org.apache.pig.backend.executionengine.ExecException: ERROR 2118: Unable to create input splits for: file:///Users/chauhana/workspace/pig-1531/a" Though the full stack trace is printed at the end which contains the underlying error String. Its more confusing because now there are three different error messages amid a java stack trace. * This warrants a testcase for regression purposes. (Infact error reporting behavior already changed since the time I opened this bug.) > Pig gobbles up error messages > - > > Key: PIG-1531 > URL: https://issues.apache.org/jira/browse/PIG-1531 > Project: Pig > Issue Type: Bug >Affects Versions: 0.7.0 >Reporter: Ashutosh Chauhan >Assignee: niraj rai > Fix For: 0.8.0 > > Attachments: PIG_1531.patch > > > Consider the following. I have my own Storer implementing StoreFunc and I am > throwing FrontEndException (and other Exceptions derived from PigException) > in its various methods. I expect those error messages to be shown in error > scenarios. Instead Pig gobbles up my error messages and shows its own generic > error message like: > {code} > 010-07-31 14:14:25,414 [main] ERROR org.apache.pig.tools.grunt.Grunt - ERROR > 2116: Unexpected error. Could not validate the output specification for: > default.partitoned > Details at logfile: /Users/ashutosh/workspace/pig/pig_1280610650690.log > {code} > Instead I expect it to display my error messages which it stores away in that > log file. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1497) Mandatory rule PartitionFilterOptimizer
[ https://issues.apache.org/jira/browse/PIG-1497?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Daniel Dai updated PIG-1497: Status: Resolved (was: Patch Available) Hadoop Flags: [Reviewed] Resolution: Fixed Did a combined test-patch with PIG-1514: [exec] -1 overall. [exec] [exec] +1 @author. The patch does not contain any @author tags. [exec] [exec] +1 tests included. The patch appears to include 80 new or modified tests. [exec] [exec] +1 javadoc. The javadoc tool did not generate any warning messages. [exec] [exec] +1 javac. The applied patch does not increase the total number of javac compiler warnings. [exec] [exec] +1 findbugs. The patch does not introduce any new Findbugs warnings. [exec] [exec] -1 release audit. The applied patch generated 443 release audit warnings (more than the trunk's current 433 warnings). All new source code have the license header. Patch committed. Thanks Xuefu! > Mandatory rule PartitionFilterOptimizer > --- > > Key: PIG-1497 > URL: https://issues.apache.org/jira/browse/PIG-1497 > Project: Pig > Issue Type: Sub-task > Components: impl >Affects Versions: 0.8.0 >Reporter: Daniel Dai >Assignee: Xuefu Zhang > Fix For: 0.8.0 > > Attachments: jira-1497-0.patch > > > Need to migrate PartitionFilterOptimizer to new logical optimizer. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Created: (PIG-1566) Support globbing for registering jars in pig script.
Support globbing for registering jars in pig script. Key: PIG-1566 URL: https://issues.apache.org/jira/browse/PIG-1566 Project: Pig Issue Type: Improvement Affects Versions: 0.7.0 Reporter: Ravi Phulari Fix For: 0.8.0 Currently user can not register pig jars with globing. For example following register script will fail. {quote} register /etc/jars/*.jar {quote} It will be great if we can support such globing for registering jars. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1514) Migrate logical optimization rule: OpLimitOptimizer
[ https://issues.apache.org/jira/browse/PIG-1514?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Daniel Dai updated PIG-1514: Status: Resolved (was: Patch Available) Hadoop Flags: [Reviewed] Resolution: Fixed Did a combined test-patch with PIG-1497: [exec] -1 overall. [exec] [exec] +1 @author. The patch does not contain any @author tags. [exec] [exec] +1 tests included. The patch appears to include 80 new or modified tests. [exec] [exec] +1 javadoc. The javadoc tool did not generate any warning messages. [exec] [exec] +1 javac. The applied patch does not increase the total number of javac compiler warnings. [exec] [exec] +1 findbugs. The patch does not introduce any new Findbugs warnings. [exec] [exec] -1 release audit. The applied patch generated 443 release audit warnings (more than the trunk's current 433 warnings). All new source code has license header except for test benchmarks (new-optlimitplan*.dot) Patch committed. Thanks Xuefu! > Migrate logical optimization rule: OpLimitOptimizer > --- > > Key: PIG-1514 > URL: https://issues.apache.org/jira/browse/PIG-1514 > Project: Pig > Issue Type: Sub-task > Components: impl >Affects Versions: 0.7.0 >Reporter: Daniel Dai >Assignee: Xuefu Zhang > Fix For: 0.8.0 > > Attachments: jira-1514-0.patch, jira-1514-1.patch > > -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1321) Logical Optimizer: Merge cascading foreach
[ https://issues.apache.org/jira/browse/PIG-1321?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12902499#action_12902499 ] Xuefu Zhang commented on PIG-1321: -- Merge is prevented if the first foreach contains flatten. UDF is allowed in any of the foreachs for this optimization. > Logical Optimizer: Merge cascading foreach > -- > > Key: PIG-1321 > URL: https://issues.apache.org/jira/browse/PIG-1321 > Project: Pig > Issue Type: Sub-task > Components: impl >Affects Versions: 0.7.0 >Reporter: Daniel Dai >Assignee: Xuefu Zhang > Fix For: 0.8.0 > > Attachments: jira-1321-2.patch, pig-1321.patch > > > We can merge consecutive foreach statement. > Eg: > b = foreach a generate a0#'key1' as b0, a0#'key2' as b1, a1; > c = foreach b generate b0#'kk1', b0#'kk2', b1, a1; > => c = foreach a generate a0#'key1'#'kk1', a0#'key1'#'kk2', a0#'key2', a1; -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
RE: [jira] Commented: (PIG-1501) need to investigate the impact of compression on pig performance
Thank for quick turnaround Tejas. Yan -Original Message- From: Thejas M Nair (JIRA) [mailto:j...@apache.org] Sent: Wednesday, August 25, 2010 8:54 AM To: pig-dev@hadoop.apache.org Subject: [jira] Commented: (PIG-1501) need to investigate the impact of compression on pig performance [ https://issues.apache.org/jira/browse/PIG-1501?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12902484#action_12902484 ] Thejas M Nair commented on PIG-1501: +1 > need to investigate the impact of compression on pig performance > > > Key: PIG-1501 > URL: https://issues.apache.org/jira/browse/PIG-1501 > Project: Pig > Issue Type: Test >Reporter: Olga Natkovich >Assignee: Yan Zhou > Fix For: 0.8.0 > > Attachments: compress_perf_data.txt, compress_perf_data_2.txt, > PIG-1501.patch, PIG-1501.patch, PIG-1501.patch > > > We would like to understand how compressing map results as well as well as > reducer output in a chain of MR jobs impacts performance. We can use PigMix > queries for this investigation. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1501) need to investigate the impact of compression on pig performance
[ https://issues.apache.org/jira/browse/PIG-1501?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12902484#action_12902484 ] Thejas M Nair commented on PIG-1501: +1 > need to investigate the impact of compression on pig performance > > > Key: PIG-1501 > URL: https://issues.apache.org/jira/browse/PIG-1501 > Project: Pig > Issue Type: Test >Reporter: Olga Natkovich >Assignee: Yan Zhou > Fix For: 0.8.0 > > Attachments: compress_perf_data.txt, compress_perf_data_2.txt, > PIG-1501.patch, PIG-1501.patch, PIG-1501.patch > > > We would like to understand how compressing map results as well as well as > reducer output in a chain of MR jobs impacts performance. We can use PigMix > queries for this investigation. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1501) need to investigate the impact of compression on pig performance
[ https://issues.apache.org/jira/browse/PIG-1501?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yan Zhou updated PIG-1501: -- Attachment: PIG-1501.patch Address the review comments, code rebasing on the latest trunk. > need to investigate the impact of compression on pig performance > > > Key: PIG-1501 > URL: https://issues.apache.org/jira/browse/PIG-1501 > Project: Pig > Issue Type: Test >Reporter: Olga Natkovich >Assignee: Yan Zhou > Fix For: 0.8.0 > > Attachments: compress_perf_data.txt, compress_perf_data_2.txt, > PIG-1501.patch, PIG-1501.patch, PIG-1501.patch > > > We would like to understand how compressing map results as well as well as > reducer output in a chain of MR jobs impacts performance. We can use PigMix > queries for this investigation. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1321) Logical Optimizer: Merge cascading foreach
[ https://issues.apache.org/jira/browse/PIG-1321?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12902392#action_12902392 ] Mridul Muralidharan commented on PIG-1321: -- Is the merge prevented only if flatten is present, or for any udf ? > Logical Optimizer: Merge cascading foreach > -- > > Key: PIG-1321 > URL: https://issues.apache.org/jira/browse/PIG-1321 > Project: Pig > Issue Type: Sub-task > Components: impl >Affects Versions: 0.7.0 >Reporter: Daniel Dai >Assignee: Xuefu Zhang > Fix For: 0.8.0 > > Attachments: jira-1321-2.patch, pig-1321.patch > > > We can merge consecutive foreach statement. > Eg: > b = foreach a generate a0#'key1' as b0, a0#'key2' as b1, a1; > c = foreach b generate b0#'kk1', b0#'kk2', b1, a1; > => c = foreach a generate a0#'key1'#'kk1', a0#'key1'#'kk2', a0#'key2', a1; -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Commented: (PIG-1518) multi file input format for loaders
[ https://issues.apache.org/jira/browse/PIG-1518?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12902350#action_12902350 ] Mridul Muralidharan commented on PIG-1518: -- Might be a good idea to contact aruniyer who maintains the FISH implementation. It is essentially built upon pig split and custom loader. > multi file input format for loaders > --- > > Key: PIG-1518 > URL: https://issues.apache.org/jira/browse/PIG-1518 > Project: Pig > Issue Type: Improvement >Reporter: Olga Natkovich >Assignee: Yan Zhou > Fix For: 0.8.0 > > Attachments: PIG-1518.patch, PIG-1518.patch, PIG-1518.patch, > PIG-1518.patch, PIG-1518.patch > > > We frequently run in the situation where Pig needs to deal with small files > in the input. In this case a separate map is created for each file which > could be very inefficient. > It would be greate to have an umbrella input format that can take multiple > files and use them in a single split. We would like to see this working with > different data formats if possible. > There are already a couple of input formats doing similar thing: > MultifileInputFormat as well as CombinedInputFormat; howevere, neither works > with ne Hadoop 20 API. > We at least want to do a feasibility study for Pig 0.8.0. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1563) SUBSTRING function is broken
[ https://issues.apache.org/jira/browse/PIG-1563?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dmitriy V. Ryaboy updated PIG-1563: --- Attachment: PIG_1563.patch Updated the functions with the appropriate argToFunc mappings, added tests. Also tried on current build of trunk to make sure all the up- and down- casting magic works, and it does. Note the new methods in test.Utils and pig.impl.util.Utils -- pretty handy. > SUBSTRING function is broken > > > Key: PIG-1563 > URL: https://issues.apache.org/jira/browse/PIG-1563 > Project: Pig > Issue Type: Bug >Affects Versions: 0.8.0 >Reporter: Olga Natkovich >Assignee: Dmitriy V. Ryaboy > Fix For: 0.8.0 > > Attachments: PIG_1563.patch > > > Script: > A = load 'studenttab10k' as (name, age, gpa); > C = foreach A generate SUBSTRING(name, 0,5); > E = limit C 10; > dump E; > Output is always empty: > () > () > () > () > () > () > () > () > () > () -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Updated: (PIG-1563) SUBSTRING function is broken
[ https://issues.apache.org/jira/browse/PIG-1563?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dmitriy V. Ryaboy updated PIG-1563: --- Status: Patch Available (was: Open) Affects Version/s: 0.8.0 > SUBSTRING function is broken > > > Key: PIG-1563 > URL: https://issues.apache.org/jira/browse/PIG-1563 > Project: Pig > Issue Type: Bug >Affects Versions: 0.8.0 >Reporter: Olga Natkovich >Assignee: Dmitriy V. Ryaboy > Fix For: 0.8.0 > > Attachments: PIG_1563.patch > > > Script: > A = load 'studenttab10k' as (name, age, gpa); > C = foreach A generate SUBSTRING(name, 0,5); > E = limit C 10; > dump E; > Output is always empty: > () > () > () > () > () > () > () > () > () > () -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.
[jira] Assigned: (PIG-1563) SUBSTRING function is broken
[ https://issues.apache.org/jira/browse/PIG-1563?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dmitriy V. Ryaboy reassigned PIG-1563: -- Assignee: Dmitriy V. Ryaboy (was: Yan Zhou) > SUBSTRING function is broken > > > Key: PIG-1563 > URL: https://issues.apache.org/jira/browse/PIG-1563 > Project: Pig > Issue Type: Bug >Reporter: Olga Natkovich >Assignee: Dmitriy V. Ryaboy > Fix For: 0.8.0 > > > Script: > A = load 'studenttab10k' as (name, age, gpa); > C = foreach A generate SUBSTRING(name, 0,5); > E = limit C 10; > dump E; > Output is always empty: > () > () > () > () > () > () > () > () > () > () -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.