Pig

Pig Setup


Requirements

Mandatory
Unix and Windows users need the following:

Optional

Download Pig

To get a Pig distribution, do the following:
  1. Download a recent stable release from one of the Apache Download Mirrors (see Pig Releases).
  2. Unpack the downloaded Pig distribution, and then note the following:
    • The Pig script file, pig, is located in the bin directory (/pig-n.n.n/bin/pig). The Pig environment variables are described in the Pig script file.
    • The Pig properties file, pig.properties, is located in the conf directory (/pig-n.n.n/conf/pig.properties). You can specify an alternate location using the PIG_CONF_DIR environment variable.
  3. Add /pig-n.n.n/bin to your path. Use export (bash,sh,ksh) or setenv (tcsh,csh). For example:
    $ export PATH=/<my-path-to-pig>/pig-n.n.n/bin:$PATH
  4. Test the Pig installation with this simple command: $ pig -help

Build Pig

To build pig, do the following:
  1. Check out the Pig code from SVN: svn co http://svn.apache.org/repos/asf/pig/trunk
  2. Build the code from the top directory: ant
    If the build is successful, you should see the pig.jar file created in that directory.
  3. Validate the pig.jar by running a unit test: ant test

Running Pig

You can run Pig (execute Pig Latin statements and Pig commands) using various modes.
Local Mode Tez Local Mode Spark Local Mode Mapreduce Mode Tez Mode Spark Mode
Interactive Mode yes experimental yes yes
Batch Mode yes experimental yes yes

Execution Modes

Pig has six execution modes or exectypes:
  • Local Mode - To run Pig in local mode, you need access to a single machine; all files are installed and run using your local host and file system. Specify local mode using the -x flag (pig -x local).
  • Tez Local Mode - To run Pig in tez local mode. It is similar to local mode, except internally Pig will invoke tez runtime engine. Specify Tez local mode using the -x flag (pig -x tez_local). Note: Tez local mode is experimental. There are some queries which just error out on bigger data in local mode.
  • Spark Local Mode - To run Pig in spark local mode. It is similar to local mode, except internally Pig will invoke spark runtime engine. Specify Spark local mode using the -x flag (pig -x spark_local). Note: Spark local mode is experimental. There are some queries which just error out on bigger data in local mode.
  • Mapreduce Mode - To run Pig in mapreduce mode, you need access to a Hadoop cluster and HDFS installation. Mapreduce mode is the default mode; you can, but don't need to, specify it using the -x flag (pig OR pig -x mapreduce).
  • Tez Mode - To run Pig in Tez mode, you need access to a Hadoop cluster and HDFS installation. Specify Tez mode using the -x flag (-x tez).
  • Spark Mode - To run Pig in Spark mode, you need access to a Spark, Yarn or Mesos cluster and HDFS installation. Specify Spark mode using the -x flag (-x spark). In Spark execution mode, it is necessary to set env::SPARK_MASTER to an appropriate value (local - local mode, yarn-client - yarn-client mode, mesos://host:port - spark on mesos or spark://host:port - spark cluster. For more information refer to spark documentation on Master URLs, yarn-cluster mode is currently not supported). Pig scripts run on Spark can take advantage of the dynamic allocation feature. The feature can be enabled by simply enabling spark.dynamicAllocation.enabled. Refer to spark configuration for additional configuration details. In general all properties in the pig script prefixed with spark. are copied to the Spark Application Configuration. Please note that Yarn auxillary service need to be enabled on Spark for this to work. See Spark documentation for additional details.

You can run Pig in either mode using the "pig" command (the bin/pig Perl script) or the "java" command (java -cp pig.jar ...).

Examples

This example shows how to run Pig in local and mapreduce mode using the pig command.
/* local mode */
$ pig -x local ...
 
/* Tez local mode */
$ pig -x tez_local ...
 
/* Spark local mode */
$ pig -x spark_local ...

/* mapreduce mode */
$ pig ...
or
$ pig -x mapreduce ...

/* Tez mode */
$ pig -x tez ...

/* Spark mode */
$ pig -x spark ...

Interactive Mode

You can run Pig in interactive mode using the Grunt shell. Invoke the Grunt shell using the "pig" command (as shown below) and then enter your Pig Latin statements and Pig commands interactively at the command line.

Example

These Pig Latin statements extract all user IDs from the /etc/passwd file. First, copy the /etc/passwd file to your local working directory. Next, invoke the Grunt shell by typing the "pig" command (in local or hadoop mode). Then, enter the Pig Latin statements interactively at the grunt prompt (be sure to include the semicolon after each statement). The DUMP operator will display the results to your terminal screen.
grunt> A = load 'passwd' using PigStorage(':'); 
grunt> B = foreach A generate $0 as id; 
grunt> dump B; 
Local Mode
$ pig -x local
... - Connecting to ...
grunt>
Tez Local Mode
$ pig -x tez_local
... - Connecting to ...
grunt> 
Spark Local Mode
$ pig -x spark_local
... - Connecting to ...
grunt> 
Mapreduce Mode
$ pig -x mapreduce
... - Connecting to ...
grunt> 

or

$ pig 
... - Connecting to ...
grunt> 
Tez Mode
$ pig -x tez
... - Connecting to ...
grunt> 
Spark Mode
$ pig -x spark
... - Connecting to ...
grunt>

Batch Mode

You can run Pig in batch mode using Pig scripts and the "pig" command (in local or hadoop mode).

Example

The Pig Latin statements in the Pig script (id.pig) extract all user IDs from the /etc/passwd file. First, copy the /etc/passwd file to your local working directory. Next, run the Pig script from the command line (using local or mapreduce mode). The STORE operator will write the results to a file (id.out).
/* id.pig */

A = load 'passwd' using PigStorage(':');  -- load the passwd file 
B = foreach A generate $0 as id;  -- extract the user IDs 
store B into 'id.out';  -- write the results to a file name id.out
Local Mode
$ pig -x local id.pig
Tez Local Mode
$ pig -x tez_local id.pig
Spark Local Mode
$ pig -x spark_local id.pig
Mapreduce Mode
$ pig id.pig
or
$ pig -x mapreduce id.pig
Tez Mode
$ pig -x tez id.pig
Spark Mode
$ pig -x spark id.pig

Pig Scripts

Use Pig scripts to place Pig Latin statements and Pig commands in a single file. While not required, it is good practice to identify the file using the *.pig extension.
You can run Pig scripts from the command line and from the Grunt shell (see the run and exec commands).
Pig scripts allow you to pass values to parameters using parameter substitution.

Comments in Scripts
You can include comments in Pig scripts:
  • For multi-line comments use /* …. */
  • For single-line comments use --
/* myscript.pig
My script is simple.
It includes three Pig Latin statements.
*/

A = LOAD 'student' USING PigStorage() AS (name:chararray, age:int, gpa:float); -- loading data
B = FOREACH A GENERATE name;  -- transforming data
DUMP B;  -- retrieving results

Scripts and Distributed File Systems
Pig supports running scripts (and Jar files) that are stored in HDFS, Amazon S3, and other distributed file systems. The script's full location URI is required (see REGISTER for information about Jar files). For example, to run a Pig script on HDFS, do the following:
$ pig hdfs://nn.mydomain.com:9020/myscripts/script.pig

Running jobs on a Kerberos secured cluster

Kerberos is a authentication system that uses tickets with a limited validity time.
As a consequence running a pig script on a kerberos secured hadoop cluster limits the running time to at most the remaining validity time of these kerberos tickets. When doing really complex analytics this may become a problem as the job may need to run for a longer time than these ticket times allow.

Short lived jobs

When running short jobs all you need to do is ensure that the user has been logged in into Kerberos via the normal kinit method.
The Hadoop job will automatically pickup these credentials and the job will run fine.

Long lived jobs

A kerberos keytab file is essentially a Kerberos specific form of the password of a user.
It is possible to enable a Hadoop job to request new tickets when they expire by creating a keytab file and make it part of the job that is running in the cluster. This will extend the maximum job duration beyond the maximum renew time of the kerberos tickets.
Usage:
  1. Create a keytab file for the required principal.
    Using the ktutil tool you can create a keytab using roughly these commands:
    addent -password -p niels@EXAMPLE.NL -k 1 -e rc4-hmac
    addent -password -p niels@EXAMPLE.NL -k 1 -e aes256-cts
    wkt niels.keytab
  2. Set the following properties (either via the .pigrc file or on the command line via -P file)
    • java.security.krb5.conf
      The path to the local krb5.conf file.
      Usually this is "/etc/krb5.conf"
    • hadoop.security.krb5.principal
      The pricipal you want to login with.
      Usually this would look like this "niels@EXAMPLE.NL"
    • hadoop.security.krb5.keytab
      The path to the local keytab file that must be used to authenticate with.
      Usually this would look like this "/home/niels/.krb/niels.keytab"
NOTE:All paths in these variables are local to the client system starting the actual pig script. This can be run without any special access to the cluster nodes.
Overall you would create a file that looks like this (assume we call it niels.kerberos.properties):
java.security.krb5.conf=/etc/krb5.conf
hadoop.security.krb5.principal=niels@EXAMPLE.NL
hadoop.security.krb5.keytab=/home/niels/.krb/niels.keytab
and start your script like this:
pig -P niels.kerberos.properties script.pig

Pig Latin Statements

Pig Latin statements are the basic constructs you use to process data using Pig. A Pig Latin statement is an operator that takes a relation as input and produces another relation as output. (This definition applies to all Pig Latin operators except LOAD and STORE which read data from and write data to the file system.) Pig Latin statements may include expressions and schemas. Pig Latin statements can span multiple lines and must end with a semi-colon ( ; ). By default, Pig Latin statements are processed using multi-query execution.
Pig Latin statements are generally organized as follows:
  • A LOAD statement to read data from the file system.
  • A series of "transformation" statements to process the data.
  • A DUMP statement to view results or a STORE statement to save the results.

Note that a DUMP or STORE statement is required to generate output.
  • In this example Pig will validate, but not execute, the LOAD and FOREACH statements.
    A = LOAD 'student' USING PigStorage() AS (name:chararray, age:int, gpa:float);
    B = FOREACH A GENERATE name;
    
  • In this example, Pig will validate and then execute the LOAD, FOREACH, and DUMP statements.
    A = LOAD 'student' USING PigStorage() AS (name:chararray, age:int, gpa:float);
    B = FOREACH A GENERATE name;
    DUMP B;
    (John)
    (Mary)
    (Bill)
    (Joe)
    

Loading Data

Use the LOAD operator and the load/store functions to read data into Pig (PigStorage is the default load function).

Working with Data

Pig allows you to transform data in many ways. As a starting point, become familiar with these operators:
  • Use the FILTER operator to work with tuples or rows of data. Use the FOREACH operator to work with columns of data.
  • Use the GROUP operator to group data in a single relation. Use the COGROUP, inner JOIN, and outer JOIN operators to group or join data in two or more relations.
  • Use the UNION operator to merge the contents of two or more relations. Use the SPLIT operator to partition the contents of a relation into multiple relations.

Storing Intermediate Results

Pig stores the intermediate data generated between MapReduce jobs in a temporary location on HDFS. This location must already exist on HDFS prior to use. This location can be configured using the pig.temp.dir property. The property's default value is "/tmp" which is the same as the hardcoded location in Pig 0.7.0 and earlier versions.

Storing Final Results

Use the STORE operator and the load/store functions to write results to the file system (PigStorage is the default store function).
Note: During the testing/debugging phase of your implementation, you can use DUMP to display results to your terminal screen. However, in a production environment you always want to use the STORE operator to save your results (see Store vs. Dump).

Debugging Pig Latin

Pig Latin provides operators that can help you debug your Pig Latin statements:
  • Use the DUMP operator to display results to your terminal screen.
  • Use the DESCRIBE operator to review the schema of a relation.
  • Use the EXPLAIN operator to view the logical, physical, or map reduce execution plans to compute a relation.
  • Use the ILLUSTRATE operator to view the step-by-step execution of a series of statements.
Shortcuts for Debugging Operators
Pig provides shortcuts for the frequently used debugging operators (DUMP, DESCRIBE, EXPLAIN, ILLUSTRATE). These shortcuts can be used in Grunt shell or within pig scripts. Following are the shortcuts supported by pig
  • \d alias - shourtcut for DUMP operator. If alias is ignored last defined alias will be used.
  • \de alias - shourtcut for DESCRIBE operator. If alias is ignored last defined alias will be used.
  • \e alias - shourtcut for EXPLAIN operator. If alias is ignored last defined alias will be used.
  • \i alias - shourtcut for ILLUSTRATE operator. If alias is ignored last defined alias will be used.
  • \q - To quit grunt shell

Pig Properties

Pig supports a number of Java properties that you can use to customize Pig behavior. You can retrieve a list of the properties using the help properties command. All of these properties are optional; none are required.


To specify Pig properties use one of these mechanisms:
  • The pig.properties file (add the directory that contains the pig.properties file to the classpath)
  • The -D and a Pig property in PIG_OPTS environment variable (export PIG_OPTS=-Dpig.tmpfilecompression=true)
  • The -P command line option and a properties file (pig -P mypig.properties)
  • The set command (set pig.exec.nocombiner true)
Note: The properties file uses standard Java property file format.
The following precedence order is supported: pig.properties < -D Pig property < -P properties file < set command. This means that if the same property is provided using the –D command line option as well as the –P command line option (properties file), the value of the property in the properties file will take precedence.

To specify Hadoop properties you can use the same mechanisms:
  • Hadoop configuration files (include pig-cluster-hadoop-site.xml)
  • The -D and a Hadoop property in PIG_OPTS environment variable (export PIG_OPTS=–Dmapreduce.task.profile=true)
  • The -P command line option and a property file (pig -P property_file)
  • The set command (set mapred.map.tasks.speculative.execution false)

The same precedence holds: Hadoop configuration files < -D Hadoop property < -P properties_file < set command.
Hadoop properties are not interpreted by Pig but are passed directly to Hadoop. Any Hadoop property can be passed this way.
All properties that Pig collects, including Hadoop properties, are available to any UDF via the UDFContext object. To get access to the properties, you can call the getJobConf method.

Pig Tutorial


The Pig tutorial shows you how to run Pig scripts using Pig's local mode, mapreduce mode, Tez mode and Spark mode (see Execution Modes).
To get started, do the following preliminary tasks:
  1. Make sure the JAVA_HOME environment variable is set the root of your Java installation.
  2. Make sure your PATH includes bin/pig (this enables you to run the tutorials using the "pig" command).
    $ export PATH=/<my-path-to-pig>/pig-0.17.0/bin:$PATH 
    
  3. Set the PIG_HOME environment variable:
    $ export PIG_HOME=/<my-path-to-pig>/pig-0.17.0 
    
  4. Create the pigtutorial.tar.gz file:
    • Move to the Pig tutorial directory (.../pig-0.17.0/tutorial).
    • Run the "ant" command from the tutorial directory. This will create the pigtutorial.tar.gz file.
  5. Copy the pigtutorial.tar.gz file from the Pig tutorial directory to your local directory.
  6. Unzip the pigtutorial.tar.gz file.
    $ tar -xzf pigtutorial.tar.gz
    
  7. A new directory named pigtmp is created. This directory contains the Pig Tutorial Files. These files work with Hadoop 0.20.2 and include everything you need to run Pig Script 1 and Pig Script 2.

Running the Pig Scripts in Local Mode

To run the Pig scripts in local mode, do the following:
  1. Move to the pigtmp directory.
  2. Execute the following command (using either script1-local.pig or script2-local.pig).
    $ pig -x local script1-local.pig
    
    Or if you are using Tez local mode:
    $ pig -x tez_local script1-local.pig
    
    Or if you are using Spark local mode:
    $ pig -x spark_local script1-local.pig
    
  3. Review the result files, located in the script1-local-results.txt directory. The output may contain a few Hadoop warnings which can be ignored:
    2010-04-08 12:55:33,642 [main] INFO  org.apache.hadoop.metrics.jvm.JvmMetrics 
    - Cannot initialize JVM Metrics with processName=JobTracker, sessionId= - already initialized
    

Running the Pig Scripts in Mapreduce Mode, Tez Mode or Spark Mode

To run the Pig scripts in mapreduce mode, do the following:
  1. Move to the pigtmp directory.
  2. Copy the excite.log.bz2 file from the pigtmp directory to the HDFS directory.
    $ hadoop fs –copyFromLocal excite.log.bz2 .
    
  3. Set the PIG_CLASSPATH environment variable to the location of the cluster configuration directory (the directory that contains the core-site.xml, hdfs-site.xml and mapred-site.xml files):
    export PIG_CLASSPATH=/mycluster/conf
    
    If you are using Tez, you will also need to put Tez configuration directory (the directory that contains the tez-site.xml):
    export PIG_CLASSPATH=/mycluster/conf:/tez/conf
    
    If you are using Spark, you will also need to specify SPARK_HOME and specify SPARK_JAR which is the hdfs location where you uploaded $SPARK_HOME/lib/spark-assembly*.jar:
    export SPARK_HOME=/mysparkhome/; export SPARK_JAR=hdfs://example.com:8020/spark-assembly*.jar
    Note: The PIG_CLASSPATH can also be used to add any other 3rd party dependencies or resource files a pig script may require. If there is also a need to make the added entries take the highest precedence in the Pig JVM's classpath order, one may also set the env-var PIG_USER_CLASSPATH_FIRST to any value, such as 'true' (and unset the env-var to disable).
  4. Set the HADOOP_CONF_DIR environment variable to the location of the cluster configuration directory:
    export HADOOP_CONF_DIR=/mycluster/conf
    
  5. Execute the following command (using either script1-hadoop.pig or script2-hadoop.pig):
    $ pig script1-hadoop.pig
    
    Or if you are using Tez:
    $ pig -x tez script1-hadoop.pig
    
    Or if you are using Spark:
    $ pig -x spark script1-hadoop.pig
    
  6. Review the result files, located in the script1-hadoop-results or script2-hadoop-results HDFS directory:
    $ hadoop fs -ls script1-hadoop-results
    $ hadoop fs -cat 'script1-hadoop-results/*' | less
    

Pig Tutorial Files

The contents of the Pig tutorial file (pigtutorial.tar.gz) are described here.
File Description
pig.jar Pig JAR file
tutorial.jar User defined functions (UDFs) and Java classes
script1-local.pig Pig Script 1, Query Phrase Popularity (local mode)
script1-hadoop.pig Pig Script 1, Query Phrase Popularity (mapreduce mode)
script2-local.pig Pig Script 2, Temporal Query Phrase Popularity (local mode)
script2-hadoop.pig Pig Script 2, Temporal Query Phrase Popularity (mapreduce mode)
excite-small.log Log file, Excite search engine (local mode)
excite.log.bz2 Log file, Excite search engine (mapreduce)
The user defined functions (UDFs) are described here.
UDF Description
ExtractHour Extracts the hour from the record.
NGramGenerator Composes n-grams from the set of words.
NonURLDetector Removes the record if the query field is empty or a URL.
ScoreGenerator Calculates a "popularity" score for the n-gram.
ToLower Changes the query field to lowercase.
TutorialUtil Divides the query string into a set of words.

Pig Script 1: Query Phrase Popularity

The Query Phrase Popularity script (script1-local.pig or script1-hadoop.pig) processes a search query log file from the Excite search engine and finds search phrases that occur with particular high frequency during certain times of the day.
The script is shown here:
  • Register the tutorial JAR file so that the included UDFs can be called in the script.
REGISTER ./tutorial.jar; 
  • Use the PigStorage function to load the excite log file (excite.log or excite-small.log) into the “raw” bag as an array of records with the fields user, time, and query.
raw = LOAD 'excite.log' USING PigStorage('\t') AS (user, time, query);
  • Call the NonURLDetector UDF to remove records if the query field is empty or a URL.
clean1 = FILTER raw BY org.apache.pig.tutorial.NonURLDetector(query);
  • Call the ToLower UDF to change the query field to lowercase.
clean2 = FOREACH clean1 GENERATE user, time, org.apache.pig.tutorial.ToLower(query) as query;
  • Because the log file only contains queries for a single day, we are only interested in the hour. The excite query log timestamp format is YYMMDDHHMMSS. Call the ExtractHour UDF to extract the hour (HH) from the time field.
houred = FOREACH clean2 GENERATE user, org.apache.pig.tutorial.ExtractHour(time) as hour, query;
  • Call the NGramGenerator UDF to compose the n-grams of the query.
ngramed1 = FOREACH houred GENERATE user, hour, flatten(org.apache.pig.tutorial.NGramGenerator(query)) as ngram;
  • Use the DISTINCT operator to get the unique n-grams for all records.
ngramed2 = DISTINCT ngramed1;
  • Use the GROUP operator to group records by n-gram and hour.
hour_frequency1 = GROUP ngramed2 BY (ngram, hour);
  • Use the COUNT function to get the count (occurrences) of each n-gram.
hour_frequency2 = FOREACH hour_frequency1 GENERATE flatten($0), COUNT($1) as count;
  • Use the GROUP operator to group records by n-gram only. Each group now corresponds to a distinct n-gram and has the count for each hour.
uniq_frequency1 = GROUP hour_frequency2 BY group::ngram;
  • For each group, identify the hour in which this n-gram is used with a particularly high frequency. Call the ScoreGenerator UDF to calculate a "popularity" score for the n-gram.
uniq_frequency2 = FOREACH uniq_frequency1 GENERATE flatten($0), flatten(org.apache.pig.tutorial.ScoreGenerator($1));
  • Use the FOREACH-GENERATE operator to assign names to the fields.
uniq_frequency3 = FOREACH uniq_frequency2 GENERATE $1 as hour, $0 as ngram, $2 as score, $3 as count, $4 as mean;
  • Use the FILTER operator to remove all records with a score less than or equal to 2.0.
filtered_uniq_frequency = FILTER uniq_frequency3 BY score > 2.0;
  • Use the ORDER operator to sort the remaining records by hour and score.
ordered_uniq_frequency = ORDER filtered_uniq_frequency BY hour, score;
  • Use the PigStorage function to store the results. The output file contains a list of n-grams with the following fields: hour, ngram, score, count, mean.
STORE ordered_uniq_frequency INTO '/tmp/tutorial-results' USING PigStorage(); 

Pig Script 2: Temporal Query Phrase Popularity

The Temporal Query Phrase Popularity script (script2-local.pig or script2-hadoop.pig) processes a search query log file from the Excite search engine and compares the occurrence of frequency of search phrases across two time periods separated by twelve hours.
The script is shown here:
  • Register the tutorial JAR file so that the user defined functions (UDFs) can be called in the script.
REGISTER ./tutorial.jar;
  • Use the PigStorage function to load the excite log file (excite.log or excite-small.log) into the “raw” bag as an array of records with the fields user, time, and query.
raw = LOAD 'excite.log' USING PigStorage('\t') AS (user, time, query);
  • Call the NonURLDetector UDF to remove records if the query field is empty or a URL.
clean1 = FILTER raw BY org.apache.pig.tutorial.NonURLDetector(query);
  • Call the ToLower UDF to change the query field to lowercase.
clean2 = FOREACH clean1 GENERATE user, time, org.apache.pig.tutorial.ToLower(query) as query;
  • Because the log file only contains queries for a single day, we are only interested in the hour. The excite query log timestamp format is YYMMDDHHMMSS. Call the ExtractHour UDF to extract the hour from the time field.
houred = FOREACH clean2 GENERATE user, org.apache.pig.tutorial.ExtractHour(time) as hour, query;
  • Call the NGramGenerator UDF to compose the n-grams of the query.
ngramed1 = FOREACH houred GENERATE user, hour, flatten(org.apache.pig.tutorial.NGramGenerator(query)) as ngram;
  • Use the DISTINCT operator to get the unique n-grams for all records.
ngramed2 = DISTINCT ngramed1;
  • Use the GROUP operator to group the records by n-gram and hour.
hour_frequency1 = GROUP ngramed2 BY (ngram, hour);
  • Use the COUNT function to get the count (occurrences) of each n-gram.
hour_frequency2 = FOREACH hour_frequency1 GENERATE flatten($0), COUNT($1) as count;
  • Use the FOREACH-GENERATE operator to assign names to the fields.
hour_frequency3 = FOREACH hour_frequency2 GENERATE $0 as ngram, $1 as hour, $2 as count;
  • Use the FILTERoperator to get the n-grams for hour ‘00’
hour00 = FILTER hour_frequency2 BY hour eq '00';
  • Uses the FILTER operators to get the n-grams for hour ‘12’
hour12 = FILTER hour_frequency3 BY hour eq '12';
  • Use the JOIN operator to get the n-grams that appear in both hours.
same = JOIN hour00 BY $0, hour12 BY $0;
  • Use the FOREACH-GENERATE operator to record their frequency.
same1 = FOREACH same GENERATE hour_frequency2::hour00::group::ngram as ngram, $2 as count00, $5 as count12;
  • Use the PigStorage function to store the results. The output file contains a list of n-grams with the following fields: ngram, count00, count12.
STORE same1 INTO '/tmp/tutorial-join-results' USING PigStorage();

Conventions

Conventions for the syntax and code examples in the Pig Latin Reference Manual are described here.
Convention Description Example
( ) Parentheses enclose one or more items.
Parentheses are also used to indicate the tuple data type.
Multiple items:
(1, abc, (2,4,6) )
[ ] Straight brackets enclose one or more optional items.
Straight brackets are also used to indicate the map data type. In this case <> is used to indicate optional items.
Optional items:
[INNER | OUTER]
{ } Curly brackets enclose two or more items, one of which is required.
Curly brackets also used to indicate the bag data type. In this case <> is used to indicate required items.
Two items, one required:
{ block | nested_block }
Horizontal ellipsis points indicate that you can repeat a portion of the code. Pig Latin syntax statement:
cat path [path …]
UPPERCASE
lowercase
In general, uppercase type indicates elements the system supplies.
In general, lowercase type indicates elements that you supply.
(These conventions are not strictly adherered to in all examples.)
See Case Sensitivity
Pig Latin statement:
a = LOAD 'data' AS (f1:int);
  • LOAD, AS - Pig keywords
  • a, f1 - aliases you supply
  • 'data' - data source you supply

Reserved Keywords

Pig reserved keywords are listed here.
-- A assert, and, any, all, arrange, as, asc, AVG
-- B bag, BinStorage, by, bytearray, BIGINTEGER, BIGDECIMAL
-- C cache, CASE, cat, cd, chararray, cogroup, CONCAT, copyFromLocal, copyToLocal, COUNT, cp, cross
-- D datetime, %declare, %default, define, dense, desc, describe, DIFF, distinct, double, du, dump
-- E e, E, eval, exec, explain
-- F f, F, filter, flatten, float, foreach, full
-- G generate, group
-- H help
-- I if, illustrate, import, inner, input, int, into, is
-- J join
-- K kill
-- L l, L, left, limit, load, long, ls
-- M map, matches, MAX, MIN, mkdir, mv
-- N not, null
-- O onschema, or, order, outer, output
-- P parallel, pig, PigDump, PigStorage, pwd
-- Q quit
-- R register, returns, right, rm, rmf, rollup, run
-- S sample, set, ship, SIZE, split, stderr, stdin, stdout, store, stream, SUM
-- T TextLoader, TOKENIZE, through, tuple
-- U union, using
-- V, W, X, Y, Z void

Case Sensitivity

The names (aliases) of relations and fields are case sensitive. The names of Pig Latin functions are case sensitive. The names of parameters (see Parameter Substitution) and all other Pig Latin keywords (see Reserved Keywords) are case insensitive.
In the example below, note the following:
  • The names (aliases) of relations A, B, and C are case sensitive.
  • The names (aliases) of fields f1, f2, and f3 are case sensitive.
  • Function names PigStorage and COUNT are case sensitive.
  • Keywords LOAD, USING, AS, GROUP, BY, FOREACH, GENERATE, and DUMP are case insensitive. They can also be written as load, using, as, group, by, etc.
  • In the FOREACH statement, the field in relation B is referred to by positional notation ($0).
grunt> A = LOAD 'data' USING PigStorage() AS (f1:int, f2:int, f3:int);
grunt> B = GROUP A BY f1;
grunt> C = FOREACH B GENERATE COUNT ($0);
grunt> DUMP C;

Data Types and More


Identifiers

Identifiers include the names of relations (aliases), fields, variables, and so on. In Pig, identifiers start with a letter and can be followed by any number of letters, digits, or underscores.
Valid identifiers:
A
A123
abc_123_BeX_
Invalid identifiers:
_A123
abc_$
A!B

Relations, Bags, Tuples, Fields

Pig Latin statements work with relations. A relation can be defined as follows:
  • A relation is a bag (more specifically, an outer bag).
  • A bag is a collection of tuples.
  • A tuple is an ordered set of fields.
  • A field is a piece of data.
A Pig relation is a bag of tuples. A Pig relation is similar to a table in a relational database, where the tuples in the bag correspond to the rows in a table. Unlike a relational table, however, Pig relations don't require that every tuple contain the same number of fields or that the fields in the same position (column) have the same type.
Also note that relations are unordered which means there is no guarantee that tuples are processed in any particular order. Furthermore, processing may be parallelized in which case tuples are not processed according to any total ordering.

Referencing Relations

Relations are referred to by name (or alias). Names are assigned by you as part of the Pig Latin statement. In this example the name (alias) of the relation is A.
A = LOAD 'student' USING PigStorage() AS (name:chararray, age:int, gpa:float);
DUMP A;
(John,18,4.0F)
(Mary,19,3.8F)
(Bill,20,3.9F)
(Joe,18,3.8F)
You an assign an alias to another alias. The new alias can be used in the place of the original alias to refer the original relation.
  A = LOAD 'student' USING PigStorage() AS (name:chararray, age:int, gpa:float);
  B = A;
  DUMP B;
  

Referencing Fields

Fields are referred to by positional notation or by name (alias).
  • Positional notation is generated by the system. Positional notation is indicated with the dollar sign ($) and begins with zero (0); for example, $0, $1, $2.
  • Names are assigned by you using schemas (or, in the case of the GROUP operator and some functions, by the system). You can use any name that is not a Pig keyword (see Identifiers for valid name examples).
Given relation A above, the three fields are separated out in this table.
First Field Second Field Third Field
Data type chararray int float
Positional notation (generated by system) $0 $1 $2
Possible name (assigned by you using a schema) name age gpa
Field value (for the first tuple) John 18 4.0
As shown in this example when you assign names to fields (using the AS schema clause) you can still refer to the fields using positional notation. However, for debugging purposes and ease of comprehension, it is better to use field names.
A = LOAD 'student' USING PigStorage() AS (name:chararray, age:int, gpa:float);
X = FOREACH A GENERATE name,$2;
DUMP X;
(John,4.0F)
(Mary,3.8F)
(Bill,3.9F)
(Joe,3.8F)
In this example an error is generated because the requested column ($3) is outside of the declared schema (positional notation begins with $0). Note that the error is caught before the statements are executed.
A = LOAD 'data' AS (f1:int,f2:int,f3:int);
B = FOREACH A GENERATE $3;
DUMP B;
2009-01-21 23:03:46,715 [main] ERROR org.apache.pig.tools.grunt.GruntParser - java.io.IOException: 
Out of bound access. Trying to access non-existent  : 3. Schema {f1: bytearray,f2: bytearray,f3: bytearray} has 3 column(s). 
etc ... 

Referencing Fields that are Complex Data Types

As noted, the fields in a tuple can be any data type, including the complex data types: bags, tuples, and maps.
  • Use the schemas for complex data types to name fields that are complex data types.
  • Use the dereference operators to reference and work with fields that are complex data types.
In this example the data file contains tuples. A schema for complex data types (in this case, tuples) is used to load the data. Then, dereference operators (the dot in t1.t1a and t2.$0) are used to access the fields in the tuples. Note that when you assign names to fields you can still refer to these fields using positional notation.
cat data;
(3,8,9) (4,5,6)
(1,4,7) (3,7,5)
(2,5,8) (9,5,8)

A = LOAD 'data' AS (t1:tuple(t1a:int, t1b:int,t1c:int),t2:tuple(t2a:int,t2b:int,t2c:int));

DUMP A;
((3,8,9),(4,5,6))
((1,4,7),(3,7,5))
((2,5,8),(9,5,8))

X = FOREACH A GENERATE t1.t1a,t2.$0;

DUMP X;
(3,4)
(1,3)
(2,9)

Data Types


Simple and Complex

Simple Types Description Example
int Signed 32-bit integer 10
long Signed 64-bit integer Data:     10L or 10l
Display: 10L
float 32-bit floating point Data:     10.5F or 10.5f or 10.5e2f or 10.5E2F
Display: 10.5F or 1050.0F
double 64-bit floating point Data:     10.5 or 10.5e2 or 10.5E2
Display: 10.5 or 1050.0
chararray Character array (string) in Unicode UTF-8 format hello world
bytearray Byte array (blob)
boolean boolean true/false (case insensitive)
datetime datetime 1970-01-01T00:00:00.000+00:00
biginteger Java BigInteger 200000000000
bigdecimal Java BigDecimal 33.456783321323441233442
Complex Types
tuple An ordered set of fields. (19,2)
bag An collection of tuples. {(19,2), (18,1)}
map A set of key value pairs. [open#apache]
Note the following general observations about data types:
  • Use schemas to assign types to fields.  If you don't assign types, fields default to type bytearray and implicit conversions are applied to the data depending on the context in which that data is used. For example, in relation B, f1 is converted to integer because 5 is integer. In relation C, f1 and f2 are converted to double because we don't know the type of either f1 or f2.
    A = LOAD 'data' AS (f1,f2,f3);
    B = FOREACH A GENERATE f1 + 5;
    C = FOREACH A generate f1 + f2;
    
  • If a schema is defined as part of a load statement, the load function will attempt to enforce the schema. If the data does not conform to the schema, the loader will generate a null value or an error.
    A = LOAD 'data' AS (name:chararray, age:int, gpa:float);
    
  • If an explicit cast is not supported, an error will occur. For example, you cannot cast a chararray to int.
    A = LOAD 'data' AS (name:chararray, age:int, gpa:float);
    B = FOREACH A GENERATE (int)name;
    
    This will cause an error …
  • If Pig cannot resolve incompatible types through implicit casts, an error will occur. For example, you cannot add chararray and float (see the Types Table for addition and subtraction).
    A = LOAD 'data' AS (name:chararray, age:int, gpa:float);
    B = FOREACH A GENERATE name + gpa;
    
    This will cause an error …
All data types have corresponding schemas.

Tuple

A tuple is an ordered set of fields.

Syntax
( field [, field …] )

Terms
(  ) A tuple is enclosed in parentheses ( ).
field A piece of data. A field can be any data type (including tuple and bag).

Usage
You can think of a tuple as a row with one or more fields, where each field can be any data type and any field may or may not have data. If a field has no data, then the following happens:
  • In a load statement, the loader will inject null into the tuple. The actual value that is substituted for null is loader specific; for example, PigStorage substitutes an empty field for null.
  • In a non-load statement, if a requested field is missing from a tuple, Pig will inject null.
Also see tuple schemas.

Example
In this example the tuple contains three fields.
(John,18,4.0F)

Bag

A bag is a collection of tuples.

Syntax: Inner bag
{ tuple [, tuple …] }

Terms
{  } An inner bag is enclosed in curly brackets { }.
tuple A tuple.

Usage
Note the following about bags:
  • A bag can have duplicate tuples.
  • A bag can have tuples with differing numbers of fields. However, if Pig tries to access a field that does not exist, a null value is substituted.
  • A bag can have tuples with fields that have different data types. However, for Pig to effectively process bags, the schemas of the tuples within those bags should be the same. For example, if half of the tuples include chararray fields and while the other half include float fields, only half of the tuples will participate in any kind of computation because the chararray fields will be converted to null. Bags have two forms: outer bag (or relation) and inner bag.
Also see bag schemas.

Example: Outer Bag
In this example A is a relation or bag of tuples. You can think of this bag as an outer bag.
A = LOAD 'data' as (f1:int, f2:int, f3:int);
DUMP A;
(1,2,3)
(4,2,1)
(8,3,4)
(4,3,3)

Example: Inner Bag
Now, suppose we group relation A by the first field to form relation X.
In this example X is a relation or bag of tuples. The tuples in relation X have two fields. The first field is type int. The second field is type bag; you can think of this bag as an inner bag.
X = GROUP A BY f1;
DUMP X;
(1,{(1,2,3)})
(4,{(4,2,1),(4,3,3)})
(8,{(8,3,4)})

Map

A map is a set of key/value pairs.

Syntax (<> denotes optional)
[ key#value <, key#value …> ]

Terms
[ ] Maps are enclosed in straight brackets [ ].
# Key value pairs are separated by the pound sign #.
key Must be chararray data type. Must be a unique value.
value Any data type (the defaults to bytearray).

Usage
Key values within a relation must be unique.
Also see map schemas.

Example
In this example the map includes two key value pairs.
[name#John,phone#5551212]

Nulls and Pig Latin

In Pig Latin, nulls are implemented using the SQL definition of null as unknown or non-existent. Nulls can occur naturally in data or can be the result of an operation.

Nulls, Operators, and Functions

Pig Latin operators and functions interact with nulls as shown in this table.
Operator Interaction
Comparison operators:
==, !=
>, <
>=, <=
If either subexpression is null, the result is null.
Comparison operator:
matches
If either the string being matched against or the string defining the match is null, the result is null.
Arithmetic operators:
 + , -, *, /
% modulo
? : bincond
CASE : case
If either subexpression is null, the resulting expression is null.
Null operator:
is null
If the tested value is null, returns true; otherwise, returns false (see Null Operators).
Null operator:
is not null
If the tested value is not null, returns true; otherwise, returns false (see Null Operators).
Dereference operators:
tuple (.) or map (#)
If the de-referenced tuple or map is null, returns null.
Operators:
COGROUP, GROUP, JOIN
These operators handle nulls differently (see examples below).
Function:
COUNT_STAR
This function counts all values, including nulls.
Cast operator Casting a null from one type to another type results in a null.
Functions:
AVG, MIN, MAX, SUM, COUNT
These functions ignore nulls.
Function:
CONCAT
If either subexpression is null, the resulting expression is null.
Function:
SIZE
If the tested object is null, returns null.
For Boolean subexpressions, note the results when nulls are used with these operators:
  • FILTER operator – If a filter expression results in null value, the filter does not pass them through (if X is null, !X is also null, and the filter will reject both).
  • Bincond operator – If a Boolean subexpression results in null value, the resulting expression is null (see the interactions above for Arithmetic operators)

Nulls and Constants

Nulls can be used as constant expressions in place of expressions of any type.
In this example a and null are projected.
A = LOAD 'data' AS (a, b, c).
B = FOREACH A GENERATE a, null;
In this example of an outer join, if the join key is missing from a table it is replaced by null.
A = LOAD 'student' AS (name: chararray, age: int, gpa: float);
B = LOAD 'votertab10k' AS (name: chararray, age: int, registration: chararray, donation: float);
C = COGROUP A BY name, B BY name;
D = FOREACH C GENERATE FLATTEN((IsEmpty(A) ? null : A)), FLATTEN((IsEmpty(B) ? null : B));
Like any other expression, null constants can be implicitly or explicitly cast.
In this example both a and null will be implicitly cast to double.
A = LOAD 'data' AS (a, b, c).
B = FOREACH A GENERATE a + null;
In this example  both a and null will be cast to int, a implicitly, and null explicitly.
A = LOAD 'data' AS (a, b, c).
B = FOREACH A GENERATE a + (int)null;

Operations That Produce Nulls

As noted, nulls can be the result of an operation. These operations can produce null values:
  • Division by zero
  • Returns from user defined functions (UDFs)
  • Dereferencing a field that does not exist.
  • Dereferencing a key that does not exist in a map. For example, given a map, info, containing [name#john, phone#5551212] if a user tries to use info#address a null is returned.
  • Accessing a field that does not exist in a tuple.

Example: Accessing a field that does not exist in a tuple
In this example nulls are injected if fields do not have data.
cat data;
    2   3
4   
7   8   9

A = LOAD 'data' AS (f1:int,f2:int,f3:int)

DUMP A;
(,2,3)
(4,,)
(7,8,9)

B = FOREACH A GENERATE f1,f2;

DUMP B;
(,2)
(4,)
(7,8)

Nulls and Load Functions

As noted, nulls can occur naturally in the data. If nulls are part of the data, it is the responsibility of the load function to handle them correctly. Keep in mind that what is considered a null value is loader-specific; however, the load function should always communicate null values to Pig by producing Java nulls.
The Pig Latin load functions (for example, PigStorage and TextLoader) produce null values wherever data is missing. For example, empty strings (chararrays) are not loaded; instead, they are replaced by nulls.
PigStorage is the default load function for the LOAD operator. In this example the is not null operator is used to filter names with null values.
A = LOAD 'student' AS (name, age, gpa); 
B = FILTER A BY name is not null;

Nulls and GROUP/COGROUP Operators

When using the GROUP operator with a single relation, records with a null group key are grouped together.
A = load 'student' as (name:chararray, age:int, gpa:float);
dump A;
(joe,18,2.5)
(sam,,3.0)
(bob,,3.5)

X = group A by age;
dump X;
(18,{(joe,18,2.5)})
(,{(sam,,3.0),(bob,,3.5)})
   
When using the GROUP (COGROUP) operator with multiple relations, records with a null group key from different relations are considered different and are grouped separately. In the example below note that there are two tuples in the output corresponding to the null group key: one that contains tuples from relation A (but not relation B) and one that contains tuples from relation B (but not relation A).
A = load 'student' as (name:chararray, age:int, gpa:float);
B = load 'student' as (name:chararray, age:int, gpa:float);
dump B;
(joe,18,2.5)
(sam,,3.0)
(bob,,3.5)

X = cogroup A by age, B by age;
dump X;
(18,{(joe,18,2.5)},{(joe,18,2.5)})
(,{(sam,,3.0),(bob,,3.5)},{})
(,{},{(sam,,3.0),(bob,,3.5)})

Nulls and JOIN Operator

The JOIN operator - when performing inner joins - adheres to the SQL standard and disregards (filters out) null values. (See also Drop Nulls Before a Join.)
A = load 'student' as (name:chararray, age:int, gpa:float);
B = load 'student' as (name:chararray, age:int, gpa:float);
dump B;
(joe,18,2.5)
(sam,,3.0)
(bob,,3.5)
  
X = join A by age, B by age;
dump X;
(joe,18,2.5,joe,18,2.5)

Constants

Pig provides constant representations for all data types except bytearrays.
Constant Example Notes
Simple Data Types
int 19
long 19L
float 19.2F or 1.92e2f
double 19.2 or 1.92e2
chararray 'hello world'
bytearray Not applicable.
boolean true/false Case insensitive.
biginteger 19211921192119211921BI
bigdecimal 192119211921.192119211921BD
Complex Data Types
tuple (19, 2, 1) A constant in this form creates a tuple.
bag { (19, 2), (1, 2) } A constant in this form creates a bag.
map [ 'name' # 'John', 'ext' # 5555 ] A constant in this form creates a map.
Please note the following:
  • On UTF-8 systems you can specify string constants consisting of printable ASCII characters such as 'abc'; you can specify control characters such as '\t'; and, you can specify a character in Unicode by starting it with '\u', for instance, '\u0001' represents Ctrl-A in hexadecimal (see Wikipedia ASCII, Unicode, and UTF-8). In theory, you should be able to specify non-UTF-8 constants on non-UTF-8 systems but as far as we know this has not been tested.
  • To specify a long constant, l or L must be appended to the number (for example, 12345678L). If the l or L is not specified, but the number is too large to fit into an int, the problem will be detected at parse time and the processing is terminated.
  • Any numeric constant with decimal point (for example, 1.5) and/or exponent (for example, 5e+1) is treated as double unless it ends with the following characters:
    • f or F in which case it is assigned type float (for example,  1.5f)
    • BD or bd in which case it is assigned type BigDecimal (for example,  12345678.12345678BD)
  • BigIntegers can be specified by supplying BI or bi at the end of the number (for example, 123456789123456BI)
  • There is no native constant type for datetime field. You can use a ToDate udf with chararray constant as argument to generate a datetime value.
The data type definitions for tuples, bags, and maps apply to constants:
  • A tuple can contain fields of any data type
  • A bag is a collection of tuples
  • A map key must be a chararray; a map value can be any data type
Complex constants (either with or without values) can be used in the same places scalar constants can be used; that is, in FILTER and GENERATE statements.
A = LOAD 'data' USING MyStorage() AS (T: tuple(name:chararray, age: int));
B = FILTER A BY T == ('john', 25);
D = FOREACH B GENERATE T.name, [25#5.6], {(1, 5, 18)};

Expressions

In Pig Latin, expressions are language constructs used with the FILTER, FOREACH, GROUP, and SPLIT operators as well as the eval functions.
Expressions are written in conventional mathematical infix notation and are adapted to the UTF-8 character set. Depending on the context, expressions can include:
  • Any Pig data type (simple data types, complex data types)
  • Any Pig operator (arithmetic, comparison, null, boolean, dereference, sign, and cast)
  • Any Pig built in function.
  • Any user defined function (UDF) written in Java.
In Pig Latin,
  • An arithmetic expression could look like this:
    X = GROUP A BY f2*f3;
    
  • A string expression could look like this, where a and b are both chararrays:
    X = FOREACH A GENERATE CONCAT(a,b);
    
  • A boolean expression could look like this:
    X = FILTER A BY (f1==8) OR (NOT (f2+f3 > f1));
    

Field Expressions

Field expressions represent a field or a dereference operator applied to a field.

Star Expressions

Star expressions ( * ) can be used to represent all the fields of a tuple. It is equivalent to writing out the fields explicitly. In the following example the definition of B and C are exactly the same, and MyUDF will be invoked with exactly the same arguments in both cases.
A = LOAD 'data' USING MyStorage() AS (name:chararray, age: int);
B = FOREACH A GENERATE *, MyUDF(name, age);
C = FOREACH A GENERATE name, age, MyUDF(*);
          
A common error when using the star expression is shown below. In this example, the programmer really wants to count the number of elements in the bag in the second field: COUNT($1).
G = GROUP A BY $0;
C = FOREACH G GENERATE COUNT(*)
          
There are some restrictions on use of the star expression when the input schema is unknown (null):
  • For GROUP/COGROUP, you can't include a star expression in a GROUP BY column.
  • For ORDER BY, if you have project-star as ORDER BY column, you can’t have any other ORDER BY column in that statement.

Project-Range Expressions

Project-range ( .. ) expressions can be used to project a range of columns from input. For example:
  • .. $x : projects columns $0 through $x, inclusive
  • $x .. : projects columns through end, inclusive
  • $x .. $y : projects columns through $y, inclusive
If the input relation has a schema, you can refer to columns by alias rather than by column position. You can also combine aliases and column positions in an expression; for example, "col1 .. $5" is valid.
Project-range can be used in all cases where the star expression ( * ) is allowed.
Project-range can be used in the following statements: FOREACH, JOIN, GROUP, COGROUP, and ORDER BY (also when ORDER BY is used within a nested FOREACH block).
A few examples are shown here:
..... 
grunt> F = foreach IN generate (int)col0, col1 .. col3; 
grunt> describe F; 
F: {col0: int,col1: bytearray,col2: bytearray,col3: bytearray} 
..... 
..... 
grunt> SORT = order IN by col2 .. col3, col0, col4 ..; 
..... 
..... 
J = join IN1 by $0 .. $3, IN2 by $0 .. $3; 
..... 
..... 
g = group l1 by b .. c; 
..... 
There are some restrictions on the use of project-to-end form of project-range (eg "x .. ") when the input schema is unknown (null):
  • For GROUP/COGROUP, the project-to-end form of project-range is not allowed.
  • For ORDER BY, the project-to-end form of project-range is supported only as the last sort column.
    ..... 
    grunt> describe IN; 
    Schema for IN unknown. 
    
    /* This statement is supported */
    SORT = order IN by $2 .. $3, $6 ..; 
    
    /* This statement is NOT supported */ 
    SORT = order IN by $2 .. $3, $6 ..; 
    ..... 
    

Boolean Expressions

Boolean expressions can be made up of UDFs that return a boolean value or boolean operators (see Boolean Operators).

Tuple Expressions

Tuple expressions form subexpressions into tuples. The tuple expression has the form (expression [, expression …]), where expression is a general expression. The simplest tuple expression is the star expression, which represents all fields.

General Expressions

General expressions can be made up of UDFs and almost any operator. Since Pig does not consider boolean a base type, the result of a general expression cannot be a boolean. Field expressions are the simpliest general expressions.

Schemas

Schemas enable you to assign names to fields and declare types for fields. Schemas are optional but we encourage you to use them whenever possible; type declarations result in better parse-time error checking and more efficient code execution.
Schemas for simple types and complex types can be used anywhere a schema definition is appropriate.
Schemas are defined with the LOAD, STREAM, and FOREACH operators using the AS clause. If you define a schema using the LOAD operator, then it is the load function that enforces the schema (see LOAD and User Defined Functions for more information).
Known Schema Handling
Note the following:
  • You can define a schema that includes both the field name and field type.
  • You can define a schema that includes the field name only; in this case, the field type defaults to bytearray.
  • You can choose not to define a schema; in this case, the field is un-named and the field type defaults to bytearray.
If you assign a name to a field, you can refer to that field using the name or by positional notation. If you don't assign a name to a field (the field is un-named) you can only refer to the field using positional notation.
If you assign a type to a field, you can subsequently change the type using the cast operators. If you don't assign a type to a field, the field defaults to bytearray; you can change the default type using the cast operators.

Unknown Schema Handling
Note the following:
  • When you JOIN/COGROUP/CROSS multiple relations, if any relation has an unknown schema (or no defined schema, also referred to as a null schema), the schema for the resulting relation is null.
  • If you FLATTEN a bag with empty inner schema, the schema for the resulting relation is null.
  • If you UNION two relations with incompatible schema, the schema for resulting relation is null.
  • If the schema is null, Pig treats all fields as bytearray (in the backend, Pig will determine the real type for the fields dynamically)
See the examples below. If a field's data type is not specified, Pig will use bytearray to denote an unknown type. If the number of fields is not known, Pig will derive an unknown schema.
/* The field data types are not specified ... */
a = load '1.txt' as (a0, b0);
a: {a0: bytearray,b0: bytearray}

/* The number of fields is not known ... */
a = load '1.txt';
a: Schema for a unknown
How Pig Handles Schema
As shown above, with a few exceptions Pig can infer the schema of a relationship up front. You can examine the schema of particular relation using DESCRIBE. Pig enforces this computed schema during the actual execution by casting the input data to the expected data type. If the process is successful the results are returned to the user; otherwise, a warning is generated for each record that failed to convert. Note that Pig does not know the actual types of the fields in the input data prior to the execution; rather, Pig determines the data types and performs the right conversions on the fly.
Having a deterministic schema is very powerful; however, sometimes it comes at the cost of performance. Consider the following example:
A = load 'input' as (x, y, z);
B = foreach A generate x+y;
If you do DESCRIBE on B, you will see a single column of type double. This is because Pig makes the safest choice and uses the largest numeric type when the schema is not know. In practice, the input data could contain integer values; however, Pig will cast the data to double and make sure that a double result is returned.
If the schema of a relation can’t be inferred, Pig will just use the runtime data as is and propagate it through the pipeline.

Schemas with LOAD and STREAM

With LOAD and STREAM operators, the schema following the AS keyword must be enclosed in parentheses.
In this example the LOAD statement includes a schema definition for simple data types.
A = LOAD 'data' AS (f1:int, f2:int);

Schemas with FOREACH

With FOREACH operators, the schema following the AS keyword must be enclosed in parentheses when the FLATTEN operator is used. Otherwise, the schema should not be enclosed in parentheses.
In this example the FOREACH statement includes FLATTEN and a schema for simple data types.
X = FOREACH C GENERATE FLATTEN(B) AS (f1:int, f2:int, f3:int), group;
In this example the FOREACH statement includes a schema for simple expression.
X = FOREACH A GENERATE f1+f2 AS x1:int;
In this example the FOREACH statement includes a schemas for multiple fields.
X = FOREACH A GENERATE f1 as user, f2 as age, f3 as gpa;

Schemas for Simple Data Types

Simple data types include int, long, float, double, chararray, bytearray, boolean, datetime, biginteger and bigdecimal.

Syntax
(alias[:type]) [, (alias[:type]) …] )

Terms
alias The name assigned to the field.
type (Optional) The simple data type assigned to the field.
The alias and type are separated by a colon ( : ).
If the type is omitted, the field defaults to type bytearray.
( , ) Multiple fields are enclosed in parentheses and separated by commas.

Examples
In this example the schema defines multiple types.
cat student;
John 18 4.0
Mary 19    3.8
Bill 20    3.9
Joe 18    3.8

A = LOAD 'student' AS (name:chararray, age:int, gpa:float);

DESCRIBE A;
A: {name: chararray,age: int,gpa: float}

DUMP A;
(John,18,4.0F)
(Mary,19,3.8F)
(Bill,20,3.9F)
(Joe,18,3.8F)
In this example field "gpa" will default to bytearray because no type is declared.
cat student;
John 18 4.0
Mary 19 3.8
Bill 20 3.9
Joe 18 3.8

A = LOAD 'data' AS (name:chararray, age:int, gpa);

DESCRIBE A;
A: {name: chararray,age: int,gpa: bytearray}

DUMP A;
(John,18,4.0)
(Mary,19,3.8)
(Bill,20,3.9)
(Joe,18,3.8)

Schemas for Complex Data Types

Complex data types include tuples, bags, and maps.

Tuple Schemas

A tuple is an ordered set of fields.

Syntax
alias[:tuple] (alias[:type]) [, (alias[:type]) …] )

Terms
alias The name assigned to the tuple.
:tuple (Optional) The data type, tuple (case insensitive).
( ) The designation for a tuple, a set of parentheses.
alias[:type] The constituents of the tuple, where the schema definition rules for the corresponding type applies to the constituents of the tuple:
  • alias – the name assigned to the field
  • type (optional) – the simple or complex data type assigned to the field

Examples
In this example the schema defines one tuple. The load statements are equivalent.
cat data;
(3,8,9)
(1,4,7)
(2,5,8)

A = LOAD 'data' AS (T: tuple (f1:int, f2:int, f3:int));
A = LOAD 'data' AS (T: (f1:int, f2:int, f3:int));

DESCRIBE A;
A: {T: (f1: int,f2: int,f3: int)}

DUMP A;
((3,8,9))
((1,4,7))
((2,5,8))
In this example the schema defines two tuples.
cat data;
(3,8,9) (mary,19)
(1,4,7) (john,18)
(2,5,8) (joe,18)

A = LOAD data AS (F:tuple(f1:int,f2:int,f3:int),T:tuple(t1:chararray,t2:int));

DESCRIBE A;
A: {F: (f1: int,f2: int,f3: int),T: (t1: chararray,t2: int)}

DUMP A;
((3,8,9),(mary,19))
((1,4,7),(john,18))
((2,5,8),(joe,18))

Bag Schemas

A bag is a collection of tuples.

Syntax
alias[:bag] {tuple}

Terms
alias The name assigned to the bag.
:bag (Optional) The data type, bag (case insensitive).
{ } The designation for a bag, a set of curly brackets.
tuple A tuple (see Tuple Schema).

Examples
In this example the schema defines a bag. The two load statements are equivalent.
cat data;
{(3,8,9)}
{(1,4,7)}
{(2,5,8)}

A = LOAD 'data' AS (B: bag {T: tuple(t1:int, t2:int, t3:int)});
A = LOAD 'data' AS (B: {T: (t1:int, t2:int, t3:int)});

DESCRIBE A:
A: {B: {T: (t1: int,t2: int,t3: int)}}

DUMP A;
({(3,8,9)})
({(1,4,7)})
({(2,5,8)})

Map Schemas

A map is a set of key value pairs.

Syntax (<> demotes optional)
alias<:map> [ <type> ]

Terms
alias The name assigned to the map.
:map (Optional) The data type, map (case insensitive).
[ ] The designation for a map, a set of straight brackets [ ].
type (Optional) The datatype (all types allowed, bytearray is the default).
The type applies to the map value only; the map key is always type chararray (see Map).
If a type is declared then ALL values in the map must be of this type.

Examples
In this example the schema defines an untyped map (the map values default to bytearray). The load statements are equivalent.
cat data;
[open#apache]
[apache#hadoop]

A = LOAD 'data' AS (M:map []);
A = LOAD 'data' AS (M:[]);

DESCRIBE A;
a: {M: map[ ]}

DUMP A;
([open#apache])
([apache#hadoop])
This example shows the use of a typed maps.
/* Map types are declared*/
a = load '1.txt' as(map[int]); --Map value is int
b = foreach a generate (map[(i:int)])a0; -- Map value is tuple
b = stream a through `cat` as (m:map[{(i:int,j:chararray)}]); -- Map value is bag

/* The MapLookup of a typed map will result in a datatype of the map value */
a = load '1.txt' as(map[int]);
b = foreach a generate $0#'key';

/* Schema for b */
b: {int}


Schemas for Multiple Types

You can define schemas for data that includes multiple types.

Example
In this example the schema defines a tuple, bag, and map.
A = LOAD 'mydata' AS (T1:tuple(f1:int, f2:int), B:bag{T2:tuple(t1:float,t2:float)}, M:map[] );

A = LOAD 'mydata' AS (T1:(f1:int, f2:int), B:{T2:(t1:float,t2:float)}, M:[] );

Previous Relation Shortcut
There is a shortcut form to reference the relation on the previous line of a pig script or grunt session:
a = load 'thing' as (x:int);
b = foreach @ generate x;
c = foreach @ generate x;
d = foreach @ generate x;

Arithmetic Operators and More


Arithmetic Operators


Description

Operator Symbol  Notes
addition +
subtraction -
multiplication *
division /
modulo % Returns the remainder of a divided by b (a%b).
Works with integral numbers (int, long).
bincond ? : (condition ? value_if_true : value_if_false)
The bincond should be enclosed in parenthesis.
The schemas for the two conditional outputs of the bincond should match.
Use expressions only (relational operators are not allowed).
case CASE WHEN THEN ELSE END CASE expression [ WHEN value THEN value ]+ [ ELSE value ]? END
CASE [ WHEN condition THEN value ]+ [ ELSE value ]? END
Case operator is equivalent to nested bincond operators.
The schemas for all the outputs of the when/else branches should match.
Use expressions only (relational operators are not allowed).

Examples
Suppose we have relation A.
A = LOAD 'data' AS (f1:int, f2:int, B:bag{T:tuple(t1:int,t2:int)});

DUMP A;
(10,1,{(2,3),(4,6)})
(10,3,{(2,3),(4,6)})
(10,6,{(2,3),(4,6),(5,7)})
In this example the modulo operator is used with fields f1 and f2.
X = FOREACH A GENERATE f1, f2, f1%f2;

DUMP X;
(10,1,0)
(10,3,1)
(10,6,4)
In this example the bincond operator is used with fields f2 and B. The condition is "f2 equals 1"; if the condition is true, return 1; if the condition is false, return the count of the number of tuples in B.
X = FOREACH A GENERATE f2, (f2==1?1:COUNT(B));

DUMP X;
(1,1L)
(3,2L)
(6,3L)
In this example the case operator is used with field f2. The expression is "f2 % 2"; if the expression is equal to 0, return 'even'; if the expression is equal to 1, return 'odd'.
X = FOREACH A GENERATE f2, (
  CASE f2 % 2
    WHEN 0 THEN 'even'
    WHEN 1 THEN 'odd'
  END
);
DUMP X;
(1,odd)
(3,odd)
(6,even)
This can be also written as follows:
X = FOREACH A GENERATE f2, (
  CASE
    WHEN f2 % 2 == 0 THEN 'even'
    WHEN f2 % 2 == 1 THEN 'odd'
  END
);
DUMP X;
(1,odd)
(3,odd)
(6,even)

Types Table: addition (+) and subtraction (-) operators
* bytearray cast as this data type
bag tuple map int long float double chararray bytearray
bag error error error error error error error error error
tuple not yet error error error error error error error
map error error error error error error error
int int long float double error cast as int
long long float double error cast as long
float float double error cast as float
double double error cast as double  
chararray error error
bytearray cast as double

Types Table: multiplication (*) and division (/) operators
* bytearray cast as this data type
bag tuple map int long float double chararray bytearray
bag error error error not yet not yet not yet not yet error error
tuple error error not yet not yet not yet not yet error error
map error error error error error error error
int int long float double error cast as int
long long float double error cast as long
float float double error cast as float
double double error cast as double
chararray error error
bytearray cast as double

Types Table: modulo (%) operator
int long bytearray
int int long cast as int
long long cast as long
bytearray error

Boolean Operators


Description

Operator Symbol  Notes
AND       and
OR or
IN in IN operator is equivalent to nested OR operators.
NOT not
The result of a boolean expression (an expression that includes boolean and comparison operators) is always of type boolean (true or false).

Example
X = FILTER A BY (f1==8) OR (NOT (f2+f3 > f1)) OR (f1 IN (9, 10, 11));

Cast Operators


Description

Pig Latin supports casts as shown in this table.
from / to bag tuple map int long float double chararray bytearray boolean
bag error error error error error error error error error
tuple error error error error error error error error error
map error error error error error error error error error
int error error error yes yes yes yes error error
long error error error yes yes yes yes error error
float error error error yes yes yes yes error error
double error error error yes yes yes yes error error
chararray error error error yes yes yes yes error yes
bytearray yes yes yes yes yes yes yes yes yes
boolean error error error error error error error yes error

Syntax  
{(data_type) |  (tuple(data_type))  | (bag{tuple(data_type)}) | (map[]) } field

Terms
(data_type) The data type you want to cast to, enclosed in parentheses. You can cast to any data type except bytearray (see the table above).
field The field whose type you want to change.
The field can be represented by positional notation or by name (alias). For example, if f1 is the first field and type int, you can cast to type long using (long)$0 or (long)f1.

Usage
Cast operators enable you to cast or convert data from one type to another, as long as conversion is supported (see the table above). For example, suppose you have an integer field, myint, which you want to convert to a string. You can cast this field from int to chararray using (chararray)myint.
Please note the following:
  • A field can be explicitly cast. Once cast, the field remains that type (it is not automatically cast back). In this example $0 is explicitly cast to int.
    B = FOREACH A GENERATE (int)$0 + 1;
    
  • Where possible, Pig performs implicit casts. In this example $0 is cast to int (regardless of underlying data) and $1 is cast to double.
    B = FOREACH A GENERATE $0 + 1, $1 + 1.0
    
  • When two bytearrays are used in arithmetic expressions or a bytearray expression is used with built in aggregate functions (such as SUM) they are implicitly cast to double. If the underlying data is really int or long, you’ll get better performance by declaring the type or explicitly casting the data.
  • Downcasts may cause loss of data. For example casting from long to int may drop bits.

Examples

In this example an int is cast to type chararray (see relation X).
A = LOAD 'data' AS (f1:int,f2:int,f3:int);

DUMP A;
(1,2,3)
(4,2,1)
(8,3,4)
(4,3,3)
(7,2,5)
(8,4,3)

B = GROUP A BY f1;

DUMP B;
(1,{(1,2,3)})
(4,{(4,2,1),(4,3,3)})
(7,{(7,2,5)})
(8,{(8,3,4),(8,4,3)})

DESCRIBE B;
B: {group: int,A: {f1: int,f2: int,f3: int}}

X = FOREACH B GENERATE group, (chararray)COUNT(A) AS total;
(1,1)
(4,2)
(7,1)
(8,2)

DESCRIBE X;
X: {group: int,total: chararray}
In this example a bytearray (fld in relation A) is cast to type tuple.
cat data;
(1,2,3)
(4,2,1)
(8,3,4)

A = LOAD 'data' AS fld:bytearray;

DESCRIBE A;
a: {fld: bytearray}

DUMP A;
((1,2,3))
((4,2,1))
((8,3,4))

B = FOREACH A GENERATE (tuple(int,int,float))fld;

DESCRIBE B;
b: {(int,int,float)}

DUMP B;
((1,2,3))
((4,2,1))
((8,3,4))
In this example a bytearray (fld in relation A) is cast to type bag.
cat data;
{(4829090493980522200L)}
{(4893298569862837493L)}
{(1297789302897398783L)}

A = LOAD 'data' AS fld:bytearray;

DESCRIBE A;
A: {fld: bytearray}

DUMP A;
({(4829090493980522200L)})
({(4893298569862837493L)})
({(1297789302897398783L)})

B = FOREACH A GENERATE (bag{tuple(long)})fld; 

DESCRIBE B;
B: {{(long)}}

DUMP B;
({(4829090493980522200L)})
({(4893298569862837493L)})
({(1297789302897398783L)})
In this example a bytearray (fld in relation A) is cast to type map.
cat data;
[open#apache]
[apache#hadoop]
[hadoop#pig]
[pig#grunt]

A = LOAD 'data' AS fld:bytearray;

DESCRIBE A;
A: {fld: bytearray}

DUMP A;
([open#apache])
([apache#hadoop])
([hadoop#pig])
([pig#grunt])

B = FOREACH A GENERATE ((map[])fld;

DESCRIBE B;
B: {map[ ]}

DUMP B;
([open#apache])
([apache#hadoop])
([hadoop#pig])
([pig#grunt])

Casting Relations to Scalars

Pig allows you to cast the elements of a single-tuple relation into a scalar value. The tuple can be a single-field or multi-field tulple. If the relation contains more than one tuple, however, a runtime error is generated: "Scalar has more than one row in the output".
The cast relation can be used in any place where an expression of the type would make sense, including FOREACH, FILTER, and SPLIT. Note that if an explicit cast is not used an implict cast will be inserted according to Pig rules. Also, when the schema can't be inferred bytearray is used.
The primary use case for casting relations to scalars is the ability to use the values of global aggregates in follow up computations.
In this example the percentage of clicks belonging to a particular user are computed. For the FOREACH statement, an explicit cast is used. If the SUM is not given a name, a position can be used as well (userid, clicks/(double)C.$0).
A = load 'mydata' as (userid, clicks); 
B = group A all; 
C = foreach B genertate SUM(A.clicks) as total; 
D = foreach A generate userid, clicks/(double)C.total; 
dump D;
In this example a multi-field tuple is used. For the FILTER statement, Pig performs an implicit cast. For the FOREACH statement, an explicit cast is used.
A = load 'mydata' as (userid, clicks); 
B = group A all; 
C = foreach B genertate SUM(A.clicks) as total, COUNT(A) as cnt; 
D = FILTER A by clicks > C.total/3 
E = foreach D generate userid, clicks/(double)C.total, cnt; 
dump E; 

Comparison Operators


Description

Operator Symbol  Notes
equal ==
not equal !=
less than <
greater than >
less than or equal to <=
greater than or equal to >=
pattern matching matches Takes an expression on the left and a string constant on the right.
expression matches string-constant
Use the Java format for regular expressions.
Use the comparison operators with numeric and string data.

Examples

Numeric Example
X = FILTER A BY (f1 == 8);
String Example
X = FILTER A BY (f2 == 'apache');
Matches Example
X = FILTER A BY (f1 matches '.*apache.*');

Types Table: equal (==) operator

bag tuple map int long float double chararray bytearray boolean datetime biginteger bigdecimal
bag error error error error error error error error error error error error error
tuple boolean
(see Note 1)
error error error error error error error error error error error
map boolean
(see Note 2)
error error error error error error error error error error
int boolean boolean boolean boolean error cast as boolean error error error error
long boolean boolean boolean error cast as boolean error error error error
float boolean boolean error cast as boolean error error error error
double boolean error cast as boolean error error error error
chararray boolean cast as boolean error error error error
bytearray boolean error error error error
boolean boolean error error error
datetime boolean error error
biginteger boolean error
bigdecimal boolean
Note 1: boolean (Tuple A is equal to tuple B if they have the same size s, and for all 0 <= i < s A[i] == B[i])
Note 2: boolean (Map A is equal to map B if A and B have the same number of entries, and for every key k1 in A with a value of v1, there is a key k2 in B with a value of v2, such that k1 == k2 and v1 == v2)

Types Table: not equal (!=) operator

bag tuple map int long float double chararray bytearray boolean datetime biginteger bigdecimal
bag error error error error error error error error error error error error error
tuple error error error error error error error error error error error error
map error error error error error error error error error error error
int boolean boolean boolean boolean error boolean (bytearray cast as int) error error error error
long boolean boolean boolean error boolean (bytearray cast as long) error error error error
float boolean boolean error boolean (bytearray cast as float) error error error error
double boolean error boolean (bytearray cast as double) error error error error
chararray boolean boolean (bytearray cast as chararray) error error error error
bytearray boolean error error error error
boolean boolean error error error
datetime boolean error error
biginteger boolean error
bigdecimal boolean

Types Table: matches operator

*Cast as chararray (the second argument must be chararray)
chararray bytearray*
chararray boolean boolean
bytearray boolean boolean

Type Construction Operators


Description

Operator Symbol  Notes
tuple constructor ( ) Use to construct a tuple from the specified elements. Equivalent to TOTUPLE.
bag constructor { } Use to construct a bag from the specified elements. Equivalent to TOBAG.
map constructor [ ] Use to construct a map from the specified elements. Equivalent to TOMAP.
Note the following:
  • These operators can be used anywhere where the expression of the corresponding type is acceptable including FOREACH GENERATE, FILTER, etc.
  • A single element enclosed in parens ( ) like (5) is not considered to be a tuple but rather an arithmetic operator.
  • For bags, every element is put in the bag; if the element is not a tuple Pig will create a tuple for it:
    • Given this {$1, $2} Pig creates this {($1), ($2)} a bag with two tuples ... neither $1 and $2 are tuples so Pig creates a tuple around each item
    • Given this {($1), $2} Pig creates this {($1), ($2)} a bag with two tuples ... since ($1) is treated as $1 (one cannot create a single element tuple using this syntax), {($1), $2} becomes {$1, $2} and Pig creates a tuple around each item
    • Given this {($1, $2)} Pig creates this {($1, $2)} a bag with a single tuple ... Pig creates a tuple ($1, $2) and then puts this tuple into the bag

Examples

Tuple Construction
A = load 'students' as (name:chararray, age:int, gpa:float);
B = foreach A generate (name, age);
store B into 'results';

Input (students):
joe smith  20  3.5
amy chen   22  3.2
leo allen  18  2.1

Output (results):
(joe smith,20)
(amy chen,22)
(leo allen,18)
Bag Construction
A = load 'students' as (name:chararray, age:int, gpa:float);
B = foreach A generate {(name, age)}, {name, age};
store B into 'results';

Input (students):
joe smith  20  3.5
amy chen   22  3.2
leo allen  18  2.1

Output (results):
{(joe smith,20)}   {(joe smith),(20)}
{(amy chen,22)}    {(amy chen),(22)}
{(leo allen,18)}   {(leo allen),(18)}
Map Construction
A = load 'students' as (name:chararray, age:int, gpa:float);
B = foreach A generate [name, gpa];
store B into 'results';

Input (students):
joe smith  20  3.5
amy chen   22  3.2
leo allen  18  2.1

Output (results):
[joe smith#3.5]
[amy chen#3.2]
[leo allen#2.1]

Dereference Operators


Description

Operator Symbol  Notes
tuple dereference     tuple.id or tuple.(id,…) Tuple dereferencing can be done by name (tuple.field_name) or position (mytuple.$0). If a set of fields are dereferenced (tuple.(name1, name2) or tuple.($0, $1)), the expression represents a tuple composed of the specified fields. Note that if the dot operator is applied to a bytearray, the bytearray will be assumed to be a tuple.
bag dereference bag.id or bag.(id,…) Bag dereferencing can be done by name (bag.field_name) or position (bag.$0). If a set of fields are dereferenced (bag.(name1, name2) or bag.($0, $1)), the expression represents a bag composed of the specified fields.
map dereference map#'key' Map dereferencing must be done by key (field_name#key or $0#key). If the pound operator is applied to a bytearray, the bytearray is assumed to be a map. If the key does not exist, the empty string is returned.

Examples

Tuple Example
Suppose we have relation A.
A = LOAD 'data' as (f1:int, f2:tuple(t1:int,t2:int,t3:int));

DUMP A;
(1,(1,2,3))
(2,(4,5,6))
(3,(7,8,9))
(4,(1,4,7))
(5,(2,5,8))
In this example dereferencing is used to retrieve two fields from tuple f2.
X = FOREACH A GENERATE f2.t1,f2.t3;

DUMP X;
(1,3)
(4,6)
(7,9)
(1,7)
(2,8)
Bag Example
Suppose we have relation B, formed by grouping relation A (see the GROUP operator for information about the field names in relation B).
A = LOAD 'data' AS (f1:int, f2:int,f3:int);

DUMP A;
(1,2,3)
(4,2,1)
(8,3,4)
(4,3,3)
(7,2,5)
(8,4,3)

B = GROUP A BY f1;

DUMP B;
(1,{(1,2,3)})
(4,{(4,2,1),(4,3,3)})
(7,{(7,2,5)})
(8,{(8,3,4),(8,4,3)})

ILLUSTRATE B;
etc …
----------------------------------------------------------
| b   | group: int | a: bag({f1: int,f2: int,f3: int}) |
----------------------------------------------------------
In this example dereferencing is used with relation X to project the first field (f1) of each tuple in the bag (a).
X = FOREACH B GENERATE a.f1;

DUMP X;
({(1)})
({(4),(4)})
({(7)})
({(8),(8)})
Tuple/Bag Example
Suppose we have relation B, formed by grouping relation A  (see the GROUP operator for information about the field names in relation B).
A = LOAD 'data' AS (f1:int, f2:int, f3:int);

DUMP A;
(1,2,3)
(4,2,1)
(8,3,4)
(4,3,3)
(7,2,5)
(8,4,3)

B = GROUP A BY (f1,f2);

DUMP B;
((1,2),{(1,2,3)})
((4,2),{(4,2,1)})
((4,3),{(4,3,3)})
((7,2),{(7,2,5)})
((8,3),{(8,3,4)})
((8,4),{(8,4,3)})

ILLUSTRATE B;
etc …
-------------------------------------------------------------------------------
| b     | group: tuple({f1: int,f2: int}) | a: bag({f1: int,f2: int,f3: int}) |
-------------------------------------------------------------------------------
|       | (8, 3)                                | {(8, 3, 4), (8, 3, 4)} |
-------------------------------------------------------------------------------
In this example dereferencing is used to project a field (f1) from a tuple (group) and a field (f1) from a bag (a).
X = FOREACH B GENERATE group.f1, a.f1;

DUMP X;
(1,{(1)})
(4,{(4)})
(4,{(4)})
(7,{(7)})
(8,{(8)})
(8,{(8)})
Map Example
Suppose we have relation A.
A = LOAD 'data' AS (f1:int, f2:map[]);

DUMP A;
(1,[open#apache])
(2,[apache#hadoop])
(3,[hadoop#pig])
(4,[pig#grunt])
In this example dereferencing is used to look up the value of key 'open'.
X = FOREACH A GENERATE f2#'open';

DUMP X;
(apache)
()
()
()

Disambiguate Operator

After JOIN, COGROUP, CROSS, or FLATTEN operations, the field names have the orginial alias and the disambiguate operator ( :: ) prepended in the schema. The disambiguate operator is used to identify field names in case there is a ambiguity.
In this example, to disambiguate y, use A::y or B::y. In cases where there is no ambiguity, such as z, the :: is not necessary but is still supported.
A = load 'data1' as (x, y);
B = load 'data2' as (x, y, z);
C = join A by x, B by x;
D = foreach C generate A::y, z; -- Cannot simply refer to y as it can refer to A::y or B::y
In cases where the schema is stored as part of the StoreFunc like PigStorage, JsonStorage, AvroStorage or OrcStorage, users generally have to use an extra FOREACH before STORE to rename the field names and remove the disambiguate operator from the names. To automatically remove the disambiguate operator from the schema for the STORE operation, the pig.store.schema.disambiguate Pig property can be set to "false". It is the responsibility of the user to make sure that there is no conflict in the field names when using this setting.

Flatten Operator

The FLATTEN operator looks like a UDF syntactically, but it is actually an operator that changes the structure of tuples and bags in a way that a UDF cannot. Flatten un-nests tuples, bags and maps. The idea is the same, but the operation and result is different for each type of structure.
For tuples, flatten substitutes the fields of a tuple in place of the tuple. For example, consider a relation that has a tuple of the form (a, (b, c)). The expression GENERATE $0, flatten($1), will cause that tuple to become (a, b, c).
For bags, the situation becomes more complicated. When we un-nest a bag, we create new tuples. If we have a relation that is made up of tuples of the form ({(b,c),(d,e)}) and we apply GENERATE flatten($0), we end up with two tuples (b,c) and (d,e). When we remove a level of nesting in a bag, sometimes we cause a cross product to happen. For example, consider a relation that has a tuple of the form (a, {(b,c), (d,e)}), commonly produced by the GROUP operator. If we apply the expression GENERATE $0, flatten($1) to this tuple, we will create new tuples: (a, b, c) and (a, d, e).
For maps, flatten creates a tuple with two fields containing the key and value. If we have a map field named kvpair with input as (m[k1#v1, k2#v2]) and we apply GENERATE flatten(kvpair), it will generate two tuples (k1,v1) and (k2,v2) which can be accessed as kvpair::key and kvpair::value. When there are additional projections in the expression, a cross product will happen similar to bags. For example, if we apply the expression GENERATE $0, FLATTEN($1) to the input tuple (a, m[k1#1, k2#2, k3#3]), we will see (a,k1,1), (a,k2,2) and (a,k3,3) as the result.
Also note that the flatten of empty bag will result in that row being discarded; no output is generated. (See also Drop Nulls Before a Join.)
grunt> cat empty.bag
{}      1
grunt> A = LOAD 'empty.bag' AS (b : bag{}, i : int);
grunt> B = FOREACH A GENERATE flatten(b), i;
grunt> DUMP B;
grunt>
For examples using the FLATTEN operator, see FOREACH.

Null Operators


Description

Operator Symbol  Notes

is null 
is null

is not null  
is not null
For a detailed discussion of nulls see Nulls and Pig Latin.

Examples

In this example, values that are not null are obtained.
X = FILTER A BY f1 is not null;

Types Table

The null operators can be applied to all data types (see Nulls and Pig Latin).

Sign Operators


Description

Operator Symbol  Notes
positive       +  Has no effect.
negative (negation)  -  Changes the sign of a positive or negative number.

Examples

In this example, the negation operator is applied to the "x" values.
A = LOAD 'data' as (x, y, z);

B = FOREACH A GENERATE -x, y;

Types Table: negative ( - ) operator

bag error
tuple error
map error
int int
long long
float float
double double
chararray error
bytearray double (as double)
datetime error
biginteger biginteger
bigdecimal bigdecimal

Relational Operators


ASSERT

Assert a condition on the data.

Syntax

ASSERT alias BY expression [, message];

Terms

alias The name of the relation.
BY Required keyword.
expression A boolean expression.
message Error message when assertion fails.

Usage

Use assert to ensure a condition is true on your data. Processing fails if any of the records voilate the condition.

Examples

Suppose we have relation A.
A = LOAD 'data' AS (a0:int,a1:int,a2:int);

DUMP A;
(1,2,3)
(4,2,1)
(8,3,4)
(4,3,3)
(7,2,5)
(8,4,3)
Now, you can assert that a0 column in your data is >0, fail if otherwise
ASSERT A by a0 > 0, 'a0 should be greater than 0';

COGROUP

See the GROUP operator.

CROSS

Computes the cross product of two or more relations.

Syntax

alias = CROSS alias, alias [, alias …] [PARTITION BY partitioner] [PARALLEL n];

Terms

alias The name of a relation.

PARTITION BY partitioner
Use this feature to specify the Hadoop Partitioner. The partitioner controls the partitioning of the keys of the intermediate map-outputs.
PARALLEL n Increase the parallelism of a job by specifying the number of reduce tasks, n.
For more information, see Use the Parallel Features.

Usage

Use the CROSS operator to compute the cross product (Cartesian product) of two or more relations.
CROSS is an expensive operation and should be used sparingly.

Example

Suppose we have relations A and B.
A = LOAD 'data1' AS (a1:int,a2:int,a3:int);

DUMP A;
(1,2,3)
(4,2,1)

B = LOAD 'data2' AS (b1:int,b2:int);

DUMP B;
(2,4)
(8,9)
(1,3)
In this example the cross product of relation A and B is computed.
X = CROSS A, B;

DUMP X;
(1,2,3,2,4)
(1,2,3,8,9)
(1,2,3,1,3)
(4,2,1,2,4)
(4,2,1,8,9)
(4,2,1,1,3)

CUBE

Performs cube/rollup operations.

Cube operation

Cube operation computes aggregates for all possbile combinations of specified group by dimensions. The number of group by combinations generated by cube for n dimensions will be 2^n.

Rollup operation

Rollup operations computes multiple levels of aggregates based on hierarchical ordering of specified group by dimensions. Rollup is useful when there is hierarchical ordering on the dimensions. The number of group by combinations generated by rollup for n dimensions will be n+1.

Syntax

alias = CUBE alias BY { CUBE expression | ROLLUP expression }, [ CUBE expression | ROLLUP expression ] [PARALLEL n];

Terms

alias The name of the relation.
CUBE Keyword
BY Keyword
expression Projections (dimensions) of the relation. Supports field, star and project-range expressions.
ROLLUP Keyword
PARALLEL n Increase the parallelism of a job by specifying the number of reduce tasks, n.
For more information, see Use the Parallel Features.

Example


Basic usage of CUBE operation

salesinp = LOAD '/pig/data/salesdata' USING PigStorage(',') AS
    (product:chararray, year:int, region:chararray, state:chararray, city:chararray, sales:long);
cubedinp = CUBE salesinp BY CUBE(product,year);
result = FOREACH cubedinp GENERATE FLATTEN(group), SUM(cube.sales) AS totalsales;
For a sample input tuple (car, 2012, midwest, ohio, columbus, 4000), the above query with cube operation will output
(car,2012,4000)
(car,,4000)
(,2012,4000)
(,,4000)

Output schema

grunt> describe cubedinp;
cubedinp: {group: (product: chararray,year: int),cube: {(product: chararray,year: int,region: chararray,
state: chararray,city: chararray,sales: long)}}
Note the second column, ‘cube’ field which is a bag of all tuples that belong to ‘group’. Also note that the measure attribute ‘sales’ along with other unused dimensions in load statement are pushed down so that it can be referenced later while computing aggregates on the measure, like in this case SUM(cube.sales).

Basic usage of ROLLUP operation

salesinp = LOAD '/pig/data/salesdata' USING PigStorage(',') AS
    (product:chararray, year:int, region:chararray, state:chararray, city:chararray, sales:long);
rolledup = CUBE salesinp BY ROLLUP(region,state,city);
result = FOREACH rolledup GENERATE FLATTEN(group), SUM(cube.sales) AS totalsales;
For a sample input tuple (car, 2012, midwest, ohio, columbus, 4000), the above query with rollup operation will output
(midwest,ohio,columbus,4000)
(midwest,ohio,,4000)
(midwest,,,4000)
(,,,4000)

Output schema

grunt> describe rolledup;
rolledup: {group: (region: chararray,state: chararray,city: chararray),cube: {(region: chararray,
state: chararray,city: chararray,product: chararray,year: int,sales: long)}}

Basic usage of CUBE and ROLLUP operation combined

If CUBE and ROLLUP operations are used together, the output groups will be the cross product of all groups generated by cube and rollup operation. If there are m dimensions in cube operations and n dimensions in rollup operation then overall number of combinations will be (2^m) * (n+1).
salesinp = LOAD '/pig/data/salesdata' USING PigStorage(',') AS
    (product:chararray, year:int, region:chararray, state:chararray, city:chararray, sales:long);
cubed_and_rolled = CUBE salesinp BY CUBE(product,year), ROLLUP(region, state, city);
result = FOREACH cubed_and_rolled GENERATE FLATTEN(group), SUM(cube.sales) AS totalsales;
For a sample input tuple (car, 2012, midwest, ohio, columbus, 4000), the above query with cube and rollup operation will output
(car,2012,midwest,ohio,columbus,4000)
(car,2012,midwest,ohio,,4000)
(car,2012,midwest,,,4000)
(car,2012,,,,4000)
(car,,midwest,ohio,columbus,4000)
(car,,midwest,ohio,,4000)
(car,,midwest,,,4000)
(car,,,,,4000)
(,2012,midwest,ohio,columbus,4000)
(,2012,midwest,ohio,,4000)
(,2012,midwest,,,4000)
(,2012,,,,4000)
(,,midwest,ohio,columbus,4000)
(,,midwest,ohio,,4000)
(,,midwest,,,4000)
(,,,,,4000)

Output schema

grunt> describe cubed_and_rolled;
cubed_and_rolled: {group: (product: chararray,year: int,region: chararray,
state: chararray,city: chararray),cube: {(product: chararray,year: int,region: chararray,
state: chararray,city: chararray,sales: long)}}

Handling null values in dimensions

Since null values are used to represent subtotals in cube and rollup operation, in order to differentiate the legitimate null values that already exists as dimension values, CUBE operator converts any null values in dimensions to "unknown" value before performing cube or rollup operation. For example, for CUBE(product,location) with a sample tuple (car,) the output will be
(car,unknown)
(car,)
(,unknown)
(,)

DEFINE

See:

DISTINCT

Removes duplicate tuples in a relation.

Syntax

alias = DISTINCT alias [PARTITION BY partitioner] [PARALLEL n];       

Terms

alias The name of the relation.

PARTITION BY partitioner
Use this feature to specify the Hadoop Partitioner. The partitioner controls the partitioning of the keys of the intermediate map-outputs.
PARALLEL n Increase the parallelism of a job by specifying the number of reduce tasks, n.
For more information, see Use the Parallel Features.

Usage

Use the DISTINCT operator to remove duplicate tuples in a relation. DISTINCT does not preserve the original order of the contents (to eliminate duplicates, Pig must first sort the data). You cannot use DISTINCT on a subset of fields; to do this, use FOREACH and a nested block to first select the fields and then apply DISTINCT (see Example: Nested Block).

Example

Suppose we have relation A.
A = LOAD 'data' AS (a1:int,a2:int,a3:int);

DUMP A;
(8,3,4)
(1,2,3)        
(4,3,3)        
(4,3,3)        
(1,2,3) 
In this example all duplicate tuples are removed.
X = DISTINCT A;

DUMP X;
(1,2,3)
(4,3,3)
(8,3,4)

FILTER

Selects tuples from a relation based on some condition.

Syntax

alias = FILTER alias  BY expression;

Terms

alias The name of the relation.
BY Required keyword.
expression A boolean expression.

Usage

Use the FILTER operator to work with tuples or rows of data (if you want to work with columns of data, use the FOREACH...GENERATE operation).
FILTER is commonly used to select the data that you want; or, conversely, to filter out (remove) the data you don’t want.

Examples

Suppose we have relation A.
A = LOAD 'data' AS (a1:int,a2:int,a3:int);

DUMP A;
(1,2,3)
(4,2,1)
(8,3,4)
(4,3,3)
(7,2,5)
(8,4,3)
In this example the condition states that if the third field equals 3, then include the tuple with relation X.
X = FILTER A BY f3 == 3;

DUMP X;
(1,2,3)
(4,3,3)
(8,4,3)
In this example the condition states that if the first field equals 8 or if the sum of fields f2 and f3 is not greater than first field, then include the tuple relation X.
X = FILTER A BY (f1 == 8) OR (NOT (f2+f3 > f1));

DUMP X;
(4,2,1)
(8,3,4)
(7,2,5)
(8,4,3)

FOREACH

Generates data transformations based on columns of data.

Syntax

alias  = FOREACH { block | nested_block };

Terms

alias The name of relation (outer bag).
block FOREACH…GENERATE block used with a relation (outer bag). Use this syntax:
alias = FOREACH alias GENERATE expression [AS schema] [expression [AS schema]….];
See Schemas

nested_block
Nested FOREACH...GENERATE block used with a inner bag. Use this syntax:
alias = FOREACH nested_alias {
   alias = {nested_op | nested_exp}; [{alias = {nested_op | nested_exp}; …]
   GENERATE expression [AS schema] [expression [AS schema]….]
};
Where:
The nested block is enclosed in opening and closing brackets { … }.
The GENERATE keyword must be the last statement within the nested block.
See Schemas
Macros are NOT alllowed inside a nested block.
expression An expression.
nested_alias The name of the inner bag.
nested_op Allowed operations are CROSS, DISTINCT, FILTER, FOREACH, LIMIT, and ORDER BY.
Note: FOREACH statements can be nested to two levels only. FOREACH statements that are nested to three or more levels will result in a grammar error.
You can also perform projections within the nested block.
For examples, see Example: Nested Block.
nested_exp Any arbitrary, supported expression.
AS Keyword
schema A schema using the AS keyword (see Schemas).
  • If the FLATTEN operator is used, enclose the schema in parentheses.
  • If the FLATTEN operator is not used, don't enclose the schema in parentheses.

Usage

Use the FOREACH…GENERATE operation to work with columns of data (if you want to work with tuples or rows of data, use the FILTER operation).
FOREACH...GENERATE works with relations (outer bags) as well as inner bags:
  • If A is a relation (outer bag), a FOREACH statement could look like this.
    X = FOREACH A GENERATE f1;
    
  • If A is an inner bag, a FOREACH statement could look like this.
    X = FOREACH B {
            S = FILTER A BY 'xyz';
            GENERATE COUNT (S.$0);
    }
    

Example: Projection

In this example the asterisk (*) is used to project all fields from relation A to relation X. Relation A and X are identical.
X = FOREACH A GENERATE *;

DUMP X;
(1,2,3)
(4,2,1)
(8,3,4)
(4,3,3)
(7,2,5)
(8,4,3)
In this example two fields from relation A are projected to form relation X.
X = FOREACH A GENERATE a1, a2;

DUMP X;
(1,2)
(4,2)
(8,3)
(4,3)
(7,2)
(8,4)

Example: Nested Projection

In this example if one of the fields in the input relation is a tuple, bag or map, we can perform a projection on that field (using a deference operator).
X = FOREACH C GENERATE group, B.b2;

DUMP X;
(1,{(3)})
(4,{(6),(9)})
(8,{(9)})
In this example multiple nested columns are retained.
X = FOREACH C GENERATE group, A.(a1, a2);

DUMP X;
(1,{(1,2)})
(4,{(4,2),(4,3)})
(8,{(8,3),(8,4)})

Example: Schema

In this example two fields in relation A are summed to form relation X. A schema is defined for the projected field.
X = FOREACH A GENERATE a1+a2 AS f1:int;

DESCRIBE X;
x: {f1: int}

DUMP X;
(3)
(6)
(11)
(7)
(9)
(12)

Y = FILTER X BY f1 > 10;

DUMP Y;
(11)
(12)

Example: Applying Functions

In this example the built in function SUM() is used to sum a set of numbers in a bag.
X = FOREACH C GENERATE group, SUM (A.a1);

DUMP X;
(1,1)
(4,8)
(8,16)

Example: Flatten

In this example the FLATTEN operator is used to eliminate nesting.
X = FOREACH C GENERATE group, FLATTEN(A);

DUMP X;
(1,1,2,3)
(4,4,2,1)
(4,4,3,3)
(8,8,3,4)
(8,8,4,3)
Another FLATTEN example.
X = FOREACH C GENERATE GROUP, FLATTEN(A.a3);

DUMP X;
(1,3)
(4,1)
(4,3)
(8,4)
(8,3)
Another FLATTEN example. Note that for the group '4' in C, there are two tuples in each bag. Thus, when both bags are flattened, the cross product of these tuples is returned; that is, tuples (4, 2, 6), (4, 3, 6), (4, 2, 9), and (4, 3, 9).
X = FOREACH C GENERATE FLATTEN(A.(a1, a2)), FLATTEN(B.$1);

DUMP X;
(1,2,3)
(4,2,6)
(4,2,9)
(4,3,6)
(4,3,9)
(8,3,9)
(8,4,9)
Another FLATTEN example. Here, relations A and B both have a column x. When forming relation E, you need to use the :: operator to identify which column x to use - either relation A column x (A::x) or relation B column x (B::x). This example uses relation A column x (A::x).
A = LOAD 'data' AS (x, y);
B = LOAD 'data' AS (x, z);
C = COGROUP A BY x, B BY x;
D = FOREACH C GENERATE flatten(A), flatten(b);
E = GROUP D BY A::x;
……
A FLATTEN example on a map type. Here we load an integer and map (of integer values) into A. Then m gets flattened, and finally we are filtering the result to only include tuples where the value among the un-nested map entries was 5.
A = LOAD 'data' AS (a:int, m:map[int]);
B = FOREACH A GENERATE a, FLATTEN(m);
C = FILTER B by m::value == 5;
……

Example: Nested Block

In this example a CROSS is performed within the nested block.
user = load 'user' as (uid, age, gender, region);
session = load 'session' as (uid, region);
C = cogroup user by uid, session by uid;
D = foreach C {
    crossed = cross user, session;
    generate crossed;
}
dump D;  
In this example FOREACH is nested to the second level.
a = load '1.txt' as (a0, a1:chararray, a2:chararray); 
b = group a by a0; 
c = foreach b { 
    c0 = foreach a generate TOMAP(a1,a2); 
    generate c0; 
} 
dump c; 
This example shows a CROSS and FOREACH nested to the second level.
a = load '1.txt' as (a0, a1, a2); 
b = load '2.txt' as (b0, b1); 
c = cogroup a by a0, b by b0; 
d = foreach c { 
    d0 = cross a, b; 
    d1 = foreach d0 generate a1+b1; 
    generate d1; 
} 
dump d;
Suppose we have relations A and B. Note that relation B contains an inner bag.
A = LOAD 'data' AS (url:chararray,outlink:chararray);

DUMP A;
(www.ccc.com,www.hjk.com)
(www.ddd.com,www.xyz.org)
(www.aaa.com,www.cvn.org)
(www.www.com,www.kpt.net)
(www.www.com,www.xyz.org)
(www.ddd.com,www.xyz.org)

B = GROUP A BY url;

DUMP B;
(www.aaa.com,{(www.aaa.com,www.cvn.org)})
(www.ccc.com,{(www.ccc.com,www.hjk.com)})
(www.ddd.com,{(www.ddd.com,www.xyz.org),(www.ddd.com,www.xyz.org)})
(www.www.com,{(www.www.com,www.kpt.net),(www.www.com,www.xyz.org)})
In this example we perform two of the operations allowed in a nested block, FILTER and DISTINCT. Note that the last statement in the nested block must be GENERATE. Also, note the use of projection (PA = FA.outlink;) to retrieve a field. DISTINCT can be applied to a subset of fields (as opposed to a relation) only within a nested block.
X = FOREACH B {
        FA= FILTER A BY outlink == 'www.xyz.org';
        PA = FA.outlink;
        DA = DISTINCT PA;
        GENERATE group, COUNT(DA);
}

DUMP X;
(www.aaa.com,0)
(www.ccc.com,0)
(www.ddd.com,1)
(www.www.com,1)

GROUP

Groups the data in one or more relations.
Note: The GROUP and COGROUP operators are identical. Both operators work with one or more relations. For readability GROUP is used in statements involving one relation and COGROUP is used in statements involving two or more relations. You can COGROUP up to but no more than 127 relations at a time.

Syntax

alias = GROUP alias { ALL | BY expression} [, alias ALL | BY expression …] [USING 'collected' | 'merge'] [PARTITION BY partitioner] [PARALLEL n];

Terms

alias The name of a relation.
You can COGROUP up to but no more than 127 relations at a time.
ALL Keyword. Use ALL if you want all tuples to go to a single group; for example, when doing aggregates across entire relations.
B = GROUP A ALL;
BY Keyword. Use this clause to group the relation by field, tuple or expression.
B = GROUP A BY f1;
expression A tuple expression. This is the group key or key field. If the result of the tuple expression is a single field, the key will be the value of the first field rather than a tuple with one field. To group using multiple keys, enclose the keys in parentheses:
B = GROUP A BY (key1,key2);
USING Keyword
'collected' Use the ‘collected’ clause with the GROUP operation (works with one relation only).
The following conditions apply:
  • The loader must implement the {CollectableLoader} interface.
  • Data must be sorted on the group key.
If your data and loaders satisfy these conditions, use the ‘collected’ clause to perform an optimized version of GROUP; the operation will execute on the map side and avoid running the reduce phase.
'merge' Use the ‘merge’ clause with the COGROUP operation (works with two or more relations only).
The following conditions apply:
  • No other operations can be done between the LOAD and COGROUP statements.
  • Data must be sorted on the COGROUP key for all tables in ascending (ASC) order.
  • Nulls are considered smaller than evertyhing. If data contains null keys, they should occur before anything else.
  • Left-most loader must implement the {CollectableLoader} interface as well as {OrderedLoadFunc} interface.
  • All other loaders must implement IndexableLoadFunc.
  • Type information must be provided in the schema for all the loaders.
If your data and loaders satisfy these conditions, the ‘merge’ clause to perform an optimized version of COGROUP; the operation will execute on the map side and avoid running the reduce phase.

PARTITION BY partitioner
Use this feature to specify the Hadoop Partitioner. The partitioner controls the partitioning of the keys of the intermediate map-outputs.
PARALLEL n Increase the parallelism of a job by specifying the number of reduce tasks, n.
For more information, see Use the Parallel Features.

Usage

The GROUP operator groups together tuples that have the same group key (key field). The key field will be a tuple if the group key has more than one field, otherwise it will be the same type as that of the group key. The result of a GROUP operation is a relation that includes one tuple per group. This tuple contains two fields:
  • The first field is named "group" (do not confuse this with the GROUP operator) and is the same type as the group key.
  • The second field takes the name of the original relation and is type bag.
  • The names of both fields are generated by the system as shown in the example below.
Note the following about the GROUP/COGROUP and JOIN operators:
  • The GROUP and JOIN operators perform similar functions. GROUP creates a nested set of output tuples while JOIN creates a flat set of output tuples
  • The GROUP/COGROUP and JOIN operators handle null values differently (see Nulls and GROUP/COGROUP Operataors).

Example

Suppose we have relation A.
A = load 'student' AS (name:chararray,age:int,gpa:float);

DESCRIBE A;
A: {name: chararray,age: int,gpa: float}

DUMP A;
(John,18,4.0F)
(Mary,19,3.8F)
(Bill,20,3.9F)
(Joe,18,3.8F)
Now, suppose we group relation A on field "age" for form relation B. We can use the DESCRIBE and ILLUSTRATE operators to examine the structure of relation B. Relation B has two fields. The first field is named "group" and is type int, the same as field "age" in relation A. The second field is name "A"  after relation A and is type bag.
B = GROUP A BY age;

DESCRIBE B;
B: {group: int, A: {name: chararray,age: int,gpa: float}}

ILLUSTRATE B;
etc ... 
----------------------------------------------------------------------
| B     | group: int | A: bag({name: chararray,age: int,gpa: float}) |
----------------------------------------------------------------------
|       | 18         | {(John, 18, 4.0), (Joe, 18, 3.8)}             |
|       | 20         | {(Bill, 20, 3.9)}                             |
----------------------------------------------------------------------

DUMP B;
(18,{(John,18,4.0F),(Joe,18,3.8F)})
(19,{(Mary,19,3.8F)})
(20,{(Bill,20,3.9F)})
Continuing on, as shown in these FOREACH statements, we can refer to the fields in relation B by names "group" and "A" or by positional notation.
C = FOREACH B GENERATE group, COUNT(A);

DUMP C;
(18,2L)
(19,1L)
(20,1L)

C = FOREACH B GENERATE $0, $1.name;

DUMP C;
(18,{(John),(Joe)})
(19,{(Mary)})
(20,{(Bill)})

Example

Suppose we have relation A.
A = LOAD 'data' as (f1:chararray, f2:int, f3:int);

DUMP A;
(r1,1,2)
(r2,2,1)
(r3,2,8)
(r4,4,4)
In this example the tuples are grouped using an expression, f2*f3.
X = GROUP A BY f2*f3;

DUMP X;
(2,{(r1,1,2),(r2,2,1)})
(16,{(r3,2,8),(r4,4,4)})

Example

Suppose we have two relations, A and B.
A = LOAD 'data1' AS (owner:chararray,pet:chararray);

DUMP A;
(Alice,turtle)
(Alice,goldfish)
(Alice,cat)
(Bob,dog)
(Bob,cat)

B = LOAD 'data2' AS (friend1:chararray,friend2:chararray);

DUMP B;
(Cindy,Alice)
(Mark,Alice)
(Paul,Bob)
(Paul,Jane)
In this example tuples are co-grouped using field “owner” from relation A and field “friend2” from relation B as the key fields. The DESCRIBE operator shows the schema for relation X, which has three fields, "group", "A" and "B" (see the GROUP operator for information about the field names).
X = COGROUP A BY owner, B BY friend2;

DESCRIBE X;
X: {group: chararray,A: {owner: chararray,pet: chararray},B: {friend1: chararray,friend2: chararray}}
Relation X looks like this. A tuple is created for each unique key field. The tuple includes the key field and two bags. The first bag is the tuples from the first relation with the matching key field. The second bag is the tuples from the second relation with the matching key field. If no tuples match the key field, the bag is empty.
(Alice,{(Alice,turtle),(Alice,goldfish),(Alice,cat)},{(Cindy,Alice),(Mark,Alice)})
(Bob,{(Bob,dog),(Bob,cat)},{(Paul,Bob)})
(Jane,{},{(Paul,Jane)})

Example

This example shows how to group using multiple keys.
 A = LOAD 'allresults' USING PigStorage() AS (tcid:int, tpid:int, date:chararray, result:chararray, tsid:int, tag:chararray);
 B = GROUP A BY (tcid, tpid); 

Example: PARTITION BY

To use the Hadoop Partitioner add PARTITION BY clause to the appropriate operator:
A = LOAD 'input_data'; 
B = GROUP A BY $0 PARTITION BY org.apache.pig.test.utils.SimpleCustomPartitioner PARALLEL 2;
Here is the code for SimpleCustomPartitioner:
public class SimpleCustomPartitioner extends Partitioner <PigNullableWritable, Writable> { 
     //@Override 
    public int getPartition(PigNullableWritable key, Writable value, int numPartitions) { 
        if(key.getValueAsPigType() instanceof Integer) { 
            int ret = (((Integer)key.getValueAsPigType()).intValue() % numPartitions); 
            return ret; 
       } 
       else { 
            return (key.hashCode()) % numPartitions; 
        } 
    } 
}

IMPORT

See IMPORT (macros)

JOIN (inner)

Performs an inner join of two or more relations based on common field values.

Syntax

alias = JOIN alias BY {expression|'('expression [, expression …]')'} (, alias BY {expression|'('expression [, expression …]')'} …) [USING 'replicated' | 'bloom' | 'skewed' | 'merge' | 'merge-sparse'] [PARTITION BY partitioner] [PARALLEL n]; 

Terms

alias The name of a relation.
BY Keyword
expression A field expression.
Example: X = JOIN A BY fieldA, B BY fieldB, C BY fieldC;
USING Keyword
'replicated' Use to perform replicated joins (see Replicated Joins).
'bloom' Use to perform bloom joins (see Bloom Joins).
'skewed' Use to perform skewed joins (see Skewed Joins).
'merge' Use to perform merge joins (see Merge Joins).
'merge-sparse' Use to perform merge-sparse joins (see Merge-Sparse Joins).

PARTITION BY partitioner
Use this feature to specify the Hadoop Partitioner. The partitioner controls the partitioning of the keys of the intermediate map-outputs.
This feature CANNOT be used with skewed joins.
PARALLEL n Increase the parallelism of a job by specifying the number of reduce tasks, n.
For more information, see Use the Parallel Features.

Usage

Use the JOIN operator to perform an inner, equijoin join of two or more relations based on common field values. Inner joins ignore null keys, so it makes sense to filter them out before the join.
Note the following about the GROUP/COGROUP and JOIN operators:
  • The GROUP and JOIN operators perform similar functions. GROUP creates a nested set of output tuples while JOIN creates a flat set of output tuples.
  • The GROUP/COGROUP and JOIN operators handle null values differently (see Nulls and JOIN Operator).

Self Joins
To perform self joins in Pig load the same data multiple times, under different aliases, to avoid naming conflicts.
In this example the same data is loaded twice using aliases A and B.
grunt> A = load 'mydata';
grunt> B = load 'mydata';
grunt> C = join A by $0, B by $0;
grunt> explain C;

Example

Suppose we have relations A and B.
A = LOAD 'data1' AS (a1:int,a2:int,a3:int);

DUMP A;
(1,2,3)
(4,2,1)
(8,3,4)
(4,3,3)
(7,2,5)
(8,4,3)

B = LOAD 'data2' AS (b1:int,b2:int);

DUMP B;
(2,4)
(8,9)
(1,3)
(2,7)
(2,9)
(4,6)
(4,9)
In this example relations A and B are joined by their first fields.
X = JOIN A BY a1, B BY b1;

DUMP X;
(1,2,3,1,3)
(4,2,1,4,6)
(4,3,3,4,6)
(4,2,1,4,9)
(4,3,3,4,9)
(8,3,4,8,9)
(8,4,3,8,9)

JOIN (outer)

Performs an outer join of two relations based on common field values.

Syntax

alias = JOIN left-alias BY left-alias-column [LEFT|RIGHT|FULL] [OUTER], right-alias BY right-alias-column [USING 'replicated' | 'bloom' | 'skewed' | 'merge'] [PARTITION BY partitioner] [PARALLEL n]; 

Terms

alias The name of a relation. Applies to alias, left-alias and right-alias.
alias-column The name of the join column for the corresponding relation. Applies to left-alias-column and right-alias-column.
BY Keyword
LEFT Left outer join.
RIGHT Right outer join.
FULL Full outer join.
OUTER (Optional) Keyword
USING Keyword
'replicated' Use to perform replicated joins (see Replicated Joins).
Only left outer join is supported for replicated joins.
'bloom' Use to perform bloom joins (see Bloom Joins).
Full outer join is not supported for bloom joins.
'skewed' Use to perform skewed joins (see Skewed Joins).
'merge' Use to perform merge joins (see Merge Joins).

PARTITION BY partitioner
Use this feature to specify the Hadoop Partitioner. The partitioner controls the partitioning of the keys of the intermediate map-outputs.
This feature CANNOT be used with skewed joins.
PARALLEL n Increase the parallelism of a job by specifying the number of reduce tasks, n.
For more information, see Use the Parallel Features.

Usage

Use the JOIN operator with the corresponding keywords to perform left, right, or full outer joins. The keyword OUTER is optional for outer joins; the keywords LEFT, RIGHT and FULL will imply left outer, right outer and full outer joins respectively when OUTER is omitted. The Pig Latin syntax closely adheres to the SQL standard.
Please note the following:
  • Outer joins will only work provided the relations which need to produce nulls (in the case of non-matching keys) have schemas.
  • Outer joins will only work for two-way joins; to perform a multi-way outer join, you will need to perform multiple two-way outer join statements.

Examples

This example shows a left outer join.
A = LOAD 'a.txt' AS (n:chararray, a:int); 
B = LOAD 'b.txt' AS (n:chararray, m:chararray);
C = JOIN A by $0 LEFT OUTER, B BY $0;
This example shows a full outer join.
A = LOAD 'a.txt' AS (n:chararray, a:int); 
B = LOAD 'b.txt' AS (n:chararray, m:chararray);
C = JOIN A BY $0 FULL, B BY $0;
This example shows a replicated left outer join.
A = LOAD 'large';
B = LOAD 'tiny';
C= JOIN A BY $0 LEFT, B BY $0 USING 'replicated';
This example shows a bloom right outer join.
A = LOAD 'large';
B = LOAD 'small';
C= JOIN A BY $0 RIGHT, B BY $0 USING 'bloom';
This example shows a skewed full outer join.
A = LOAD 'studenttab' as (name, age, gpa);
B = LOAD 'votertab' as (name, age, registration, contribution);
C = JOIN A BY name FULL, B BY name USING 'skewed';

LIMIT

Limits the number of output tuples.

Syntax

alias = LIMIT alias  n;

Terms

alias The name of a relation.
n The number of output tuples, either:
  • a constant (for example, 3)
  • a scalar used in an expression (for example, c.sum/100)
Note: The expression can consist of constants or scalars; it cannot contain any columns from the input relation.
Note: Using a scalar instead of a constant in LIMIT automatically disables most optimizations (only push-before-foreach is performed).

Usage

Use the LIMIT operator to limit the number of output tuples.
If the specified number of output tuples is equal to or exceeds the number of tuples in the relation, all tuples in the relation are returned.
If the specified number of output tuples is less than the number of tuples in the relation, then n tuples are returned. There is no guarantee which n tuples will be returned, and the tuples that are returned can change from one run to the next. A particular set of tuples can be requested using the ORDER operator followed by LIMIT.
Note: The LIMIT operator allows Pig to avoid processing all tuples in a relation. In most cases a query that uses LIMIT will run more efficiently than an identical query that does not use LIMIT. It is always a good idea to use limit if you can.

Examples

In this example the limit is expressed as a scalar.
a = load 'a.txt';
b = group a all;
c = foreach b generate COUNT(a) as sum;
d = order a by $0;
e = limit d c.sum/100;
Suppose we have relation A.
A = LOAD 'data' AS (a1:int,a2:int,a3:int);

DUMP A;
(1,2,3)
(4,2,1)
(8,3,4)
(4,3,3)
(7,2,5)
(8,4,3)
In this example output is limited to 3 tuples. Note that there is no guarantee which three tuples will be output.
X = LIMIT A 3;

DUMP X;
(1,2,3)
(4,3,3)
(7,2,5)
In this example the ORDER operator is used to order the tuples and the LIMIT operator is used to output the first three tuples.
B = ORDER A BY f1 DESC, f2 ASC;

DUMP B;
(8,3,4) 
(8,4,3) 
(7,2,5) 
(4,2,1)
(4,3,3)
(1,2,3)

X = LIMIT B 3;

DUMP X;
(8,3,4)
(8,4,3) 
(7,2,5) 

LOAD

Loads data from the file system.

Syntax

LOAD 'data' [USING function] [AS schema];       

Terms

'data' The name of the file or directory, in single quotes.
If you specify a directory name, all the files in the directory are loaded.
You can use Hadoop globing to specify files at the file system or directory levels (see Hadoop globStatus for details on globing syntax).

Note: Pig uses Hadoop globbing so the functionality is IDENTICAL. However, when you run from the command line using the Hadoop fs command (rather than the Pig LOAD operator), the Unix shell may do some of the substitutions; this could alter the outcome giving the impression that globing works differently for Pig and Hadoop. For example:
  • This works hadoop fs -ls /mydata/20110423{00,01,02,03,04,05,06,07,08,09,{10..23}}00//part
  • This does not work LOAD '/mydata/20110423{00,01,02,03,04,05,06,07,08,09,{10..23}}00//part '
USING Keyword.
If the USING clause is omitted, the default load function PigStorage is used.
function The load function.
  • You can use a built in function (see Load/Store Functions). PigStorage is the default load function and does not need to be specified (simply omit the USING clause).
  • You can write your own load function if your data is in a format that cannot be processed by the built in functions (see User Defined Functions).
AS Keyword.
schema A schema using the AS keyword, enclosed in parentheses (see Schemas).
The loader produces the data of the type specified by the schema. If the data does not conform to the schema, depending on the loader, either a null value or an error is generated.
Note: For performance reasons the loader may not immediately convert the data to the specified format; however, you can still operate on the data assuming the specified type.

Usage

Use the LOAD operator to load data from the file system.

Examples

Suppose we have a data file called myfile.txt. The fields are tab-delimited. The records are newline-separated.
1 2 3
4 2 1
8 3 4
In this example the default load function, PigStorage, loads data from myfile.txt to form relation A. The two LOAD statements are equivalent. Note that, because no schema is specified, the fields are not named and all fields default to type bytearray.
A = LOAD 'myfile.txt';

A = LOAD 'myfile.txt' USING PigStorage('\t');

DUMP A;
(1,2,3)
(4,2,1)
(8,3,4)
In this example a schema is specified using the AS keyword. The two LOAD statements are equivalent. You can use the DESCRIBE and ILLUSTRATE operators to view the schema.
A = LOAD 'myfile.txt' AS (f1:int, f2:int, f3:int);

A = LOAD 'myfile.txt' USING PigStorage('\t') AS (f1:int, f2:int, f3:int);

DESCRIBE A;
a: {f1: int,f2: int,f3: int}

ILLUSTRATE A;
---------------------------------------------------------
| a     | f1: bytearray | f2: bytearray | f3: bytearray |
---------------------------------------------------------
|       | 4             | 2             | 1             |
---------------------------------------------------------

---------------------------------------
| a     | f1: int | f2: int | f3: int |
---------------------------------------
|       | 4       | 2       | 1       |
---------------------------------------
For examples of how to specify more complex schemas for use with the LOAD operator, see Schemas for Complex Data Types and Schemas for Multiple Types.

NATIVE

Executes native MapReduce/Tez jobs inside a Pig script.

Syntax

alias1 = NATIVE 'native.jar' STORE alias2 INTO 'inputLocation' USING storeFunc LOAD 'outputLocation' USING loadFunc AS schema [`params, ... `];

Terms

alias1, alias2 The names of relations.
native.jar The jar file containing MapReduce or Tez program (enclosed in single quotes).
You can specify any MapReduce/Tez jar file that can be run through the hadoop jar native.jar params command.
The values for inputLocation and outputLocation can be passed in the params.
STORE ... INTO ... USING See STORE
Store alias2 into the inputLocation using storeFunc, which is then used by the MapReduce/Tez job to read its data.
LOAD ... USING ... AS See LOAD
After running native.jar's MapReduce/Tez job, load back the data from outputLocation into alias1 using loadFunc as schema.
`params, ...` Extra parameters required for the mapreduce/tez job (enclosed in back tics).

Usage

Use the NATIVE operator to run native MapReduce/Tez jobs from inside a Pig script.
The input and output locations for the MapReduce/Tez program are conveyed to Pig using the STORE/LOAD clauses. Pig, however, does not pass this information (nor require that this information be passed) to the MapReduce/Tez program. If you want to pass the input and output locations to the MapReduce/Tez program you can use the params clause or you can hardcode the locations in the MapReduce/Tez program.

Example

This example demonstrates how to run the wordcount MapReduce progam from Pig. Note that the files specified as input and output locations in the NATIVE statement will NOT be deleted by Pig automatically. You will need to delete them manually.
A = LOAD 'WordcountInput.txt';
B = NATIVE 'wordcount.jar' STORE A INTO 'inputDir' LOAD 'outputDir'
    AS (word:chararray, count: int) `org.myorg.WordCount inputDir outputDir`;

ORDER BY

Sorts a relation based on one or more fields.

Syntax

alias = ORDER alias BY { * [ASC|DESC] | field_alias [ASC|DESC] [, field_alias [ASC|DESC] …] } [PARALLEL n];

Terms

alias The name of a relation.
* The designator for a tuple.
field_alias A field in the relation. The field must be a simple type.
ASC Sort in ascending order.
DESC Sort in descending order.
PARALLEL n Increase the parallelism of a job by specifying the number of reduce tasks, n.
For more information, see Use the Parallel Features.

Usage

Note: ORDER BY is NOT stable; if multiple records have the same ORDER BY key, the order in which these records are returned is not defined and is not guarantted to be the same from one run to the next.
In Pig, relations are unordered (see Relations, Bags, Tuples, Fields):
  • If you order relation A to produce relation X (X = ORDER A BY * DESC;) relations A and X still contain the same data.
  • If you retrieve relation X (DUMP X;) the data is guaranteed to be in the order you specified (descending).
  • However, if you further process relation X (Y = FILTER X BY $0 > 1;) there is no guarantee that the data will be processed in the order you originally specified (descending).
Pig currently supports ordering on fields with simple types or by tuple designator (*). You cannot order on fields with complex types or by expressions.
A = LOAD 'mydata' AS (x: int, y: map[]);     
B = ORDER A BY x; -- this is allowed because x is a simple type
B = ORDER A BY y; -- this is not allowed because y is a complex type
B = ORDER A BY y#'id'; -- this is not allowed because y#'id' is an expression

Examples

Suppose we have relation A.
A = LOAD 'data' AS (a1:int,a2:int,a3:int);

DUMP A;
(1,2,3)
(4,2,1)
(8,3,4)
(4,3,3)
(7,2,5)
(8,4,3)
In this example relation A is sorted by the third field, f3 in descending order. Note that the order of the three tuples ending in 3 can vary.
X = ORDER A BY a3 DESC;

DUMP X;
(7,2,5)
(8,3,4)
(1,2,3)
(4,3,3)
(8,4,3)
(4,2,1)

RANK

Returns each tuple with the rank within a relation.

Syntax

alias = RANK alias [ BY { * [ASC|DESC] | field_alias [ASC|DESC] [, field_alias [ASC|DESC] …] } [DENSE] ];

Terms

alias The name of a relation.
* The designator for a tuple.
field_alias A field in the relation. The field must be a simple type.
ASC Sort in ascending order.
DESC Sort in descending order.
DENSE No gap in the ranking values.

Usage

When specifying no field to sort on, the RANK operator simply prepends a sequential value to each tuple.
Otherwise, the RANK operator uses each field (or set of fields) to sort the relation. The rank of a tuple is one plus the number of different rank values preceding it. If two or more tuples tie on the sorting field values, they will receive the same rank.
NOTE: When using the option DENSE, ties do not cause gaps in ranking values.

Examples

Suppose we have relation A.
A = load 'data' AS (f1:chararray,f2:int,f3:chararray);
   
DUMP A;
(David,1,N)
(Tete,2,N)
(Ranjit,3,M)
(Ranjit,3,P)
(David,4,Q)
(David,4,Q)
(Jillian,8,Q)
(JaePak,7,Q)
(Michael,8,T)
(Jillian,8,Q)
(Jose,10,V)
            
In this example, the RANK operator does not change the order of the relation and simply prepends to each tuple a sequential value.
B = rank A;

dump B;
(1,David,1,N)
(2,Tete,2,N)
(3,Ranjit,3,M)
(4,Ranjit,3,P)
(5,David,4,Q)
(6,David,4,Q)
(7,Jillian,8,Q)
(8,JaePak,7,Q)
(9,Michael,8,T)
(10,Jillian,8,Q)
(11,Jose,10,V)
            
In this example, the RANK operator works with f1 and f2 fields, and each one with different sorting order. RANK sorts the relation on these fields and prepends the rank value to each tuple. Otherwise, the RANK operator uses each field (or set of fields) to sort the relation. The rank of a tuple is one plus the number of different rank values preceding it. If two or more tuples tie on the sorting field values, they will receive the same rank.
C = rank A by f1 DESC, f2 ASC;
                                
dump C;
(1,Tete,2,N)
(2,Ranjit,3,M)
(2,Ranjit,3,P)
(4,Michael,8,T)
(5,Jose,10,V)
(6,Jillian,8,Q)
(6,Jillian,8,Q)
(8,JaePak,7,Q)
(9,David,1,N)
(10,David,4,Q)
(10,David,4,Q)                
            
Same example as previous, but DENSE. In this case there are no gaps in ranking values.
C = rank A by f1 DESC, f2 ASC DENSE;

dump C;
(1,Tete,2,N)
(2,Ranjit,3,M)
(2,Ranjit,3,P)
(3,Michael,8,T)
(4,Jose,10,V)
(5,Jillian,8,Q)
(5,Jillian,8,Q)
(6,JaePak,7,Q)
(7,David,1,N)
(8,David,4,Q)
(8,David,4,Q)
            

SAMPLE

Selects a random sample of data based on the specified sample size.

Syntax

SAMPLE alias size;

Terms

alias The name of a relation.
size Sample size, either
  • a constant, range 0 to 1 (for example, enter 0.1 for 10%)
  • a scalar used in an expression
Note: The expression can consist of constants or scalars; it cannot contain any columns from the input relation.

Usage

Use the SAMPLE operator to select a random data sample with the stated sample size. SAMPLE is a probabalistic operator; there is no guarantee that the exact same number of tuples will be returned for a particular sample size each time the operator is used.

Example

In this example relation X will contain 1% of the data in relation A.
A = LOAD 'data' AS (f1:int,f2:int,f3:int);

X = SAMPLE A 0.01;
In this example, a scalar expression is used (it will sample approximately 1000 records from the input).
a = LOAD 'a.txt';
b = GROUP a ALL;
c = FOREACH b GENERATE COUNT_STAR(a) AS num_rows;
d = SAMPLE a (double)1000/c.num_rows;

SPLIT

Partitions a relation into two or more relations.

Syntax

SPLIT alias INTO alias IF expression, alias IF expression [, alias IF expression …] [, alias OTHERWISE];

Terms

alias The name of a relation.
INTO Required keyword.
IF Required keyword.
expression An expression.
OTHERWISE Optional keyword. Designates a default relation.

Usage

Use the SPLIT operator to partition the contents of a relation into two or more relations based on some expression. Depending on the conditions stated in the expression:
  • A tuple may be assigned to more than one relation.
  • A tuple may not be assigned to any relation.

Example

In this example relation A is split into three relations, X, Y, and Z.
A = LOAD 'data' AS (f1:int,f2:int,f3:int);

DUMP A;                
(1,2,3)
(4,5,6)
(7,8,9)        

SPLIT A INTO X IF f1<7, Y IF f2==5, Z IF (f3<6 OR f3>6);

DUMP X;
(1,2,3)
(4,5,6)

DUMP Y;
(4,5,6)

DUMP Z;
(1,2,3)
(7,8,9)

Example

In this example, the SPLIT and FILTER statements are essentially equivalent. However, because SPLIT is implemented as "split the data stream and then apply filters" the SPLIT statement is more expensive than the FILTER statement because Pig needs to filter and store two data streams.
SPLIT input_var INTO output_var IF (field1 is not null), ignored_var IF (field1 is null);  
-- where ignored_var is not used elsewhere
   
output_var = FILTER input_var BY (field1 is not null);
   

STORE

Stores or saves results to the file system.

Syntax

STORE alias INTO 'directory' [USING function];

Terms

alias The name of a relation.
INTO Required keyword.
'directory' The name of the storage directory, in quotes. If the directory already exists, the STORE operation will fail.
The output data files, named part-nnnnn, are written to this directory.
USING Keyword. Use this clause to name the store function.
If the USING clause is omitted, the default store function PigStorage is used.
function The store function.
  • You can use a built in function (see the Load/Store Functions). PigStorage is the default store function and does not need to be specified (simply omit the USING clause).
  • You can write your own store function if your data is in a format that cannot be processed by the built in functions (see User Defined Functions).

Usage

Use the STORE operator to run (execute) Pig Latin statements and save (persist) results to the file system. Use STORE for production scripts and batch mode processing.
Note: To debug scripts during development, you can use DUMP to check intermediate results.

Examples

In this example data is stored using PigStorage and the asterisk character (*) as the field delimiter.
A = LOAD 'data' AS (a1:int,a2:int,a3:int);

DUMP A;
(1,2,3)
(4,2,1)
(8,3,4)
(4,3,3)
(7,2,5)
(8,4,3)

STORE A INTO 'myoutput' USING PigStorage ('*');

CAT myoutput;
1*2*3
4*2*1
8*3*4
4*3*3
7*2*5
8*4*3
In this example, the CONCAT function is used to format the data before it is stored.
A = LOAD 'data' AS (a1:int,a2:int,a3:int);

DUMP A;
(1,2,3)
(4,2,1)
(8,3,4)
(4,3,3)
(7,2,5)
(8,4,3)

B = FOREACH A GENERATE CONCAT('a:',(chararray)f1), CONCAT('b:',(chararray)f2), CONCAT('c:',(chararray)f3);

DUMP B;
(a:1,b:2,c:3)
(a:4,b:2,c:1)
(a:8,b:3,c:4)
(a:4,b:3,c:3)
(a:7,b:2,c:5)
(a:8,b:4,c:3)

STORE B INTO 'myoutput' using PigStorage(',');

CAT myoutput;
a:1,b:2,c:3
a:4,b:2,c:1
a:8,b:3,c:4
a:4,b:3,c:3
a:7,b:2,c:5
a:8,b:4,c:3

STREAM

Sends data to an external script or program.

Syntax

alias = STREAM alias [, alias …] THROUGH {`command` | cmd_alias } [AS schema] ;

Terms

alias The name of a relation.
THROUGH Keyword.
`command` A command, including the arguments, enclosed in back tics (where a command is anything that can be executed).
cmd_alias The name of a command created using the DEFINE operator (see DEFINE (UDFs, streaming) for additional streaming examples).
AS Keyword.
schema A schema using the AS keyword, enclosed in parentheses (see Schemas).

Usage

Use the STREAM operator to send data through an external script or program. Multiple stream operators can appear in the same Pig script. The stream operators can be adjacent to each other or have other operations in between.
When used with a command, a stream statement could look like this:
A = LOAD 'data';

B = STREAM A THROUGH `stream.pl -n 5`;
When used with a cmd_alias, a stream statement could look like this, where mycmd is the defined alias.
A = LOAD 'data';

DEFINE mycmd `stream.pl –n 5`;

B = STREAM A THROUGH mycmd;

About Data Guarantees

Data guarantees are determined based on the position of the streaming operator in the Pig script.
  • Unordered data – No guarantee for the order in which the data is delivered to the streaming application.
  • Grouped data – The data for the same grouped key is guaranteed to be provided to the streaming application contiguously
  • Grouped and ordered data – The data for the same grouped key is guaranteed to be provided to the streaming application contiguously. Additionally, the data within the group is guaranteed to be sorted by the provided secondary key.
In addition to position, data grouping and ordering can be determined by the data itself. However, you need to know the property of the data to be able to take advantage of its structure.

Example: Data Guarantees

In this example the data is unordered.
A = LOAD 'data';

B = STREAM A THROUGH `stream.pl`;
In this example the data is grouped.
A = LOAD 'data';

B = GROUP A BY $1;

C = FOREACH B FLATTEN(A);

D = STREAM C THROUGH `stream.pl`;
In this example the data is grouped and ordered.
A = LOAD 'data';

B = GROUP A BY $1;

C = FOREACH B {
      D = ORDER A BY ($3, $4);
      GENERATE D;
}

E = STREAM C THROUGH `stream.pl`;

Example: Schemas

In this example a schema is specified as part of the STREAM statement.
X = STREAM A THROUGH `stream.pl` as (f1:int, f2:int, f3:int);

UNION

Computes the union of two or more relations.

Syntax

alias = UNION [ONSCHEMA] alias, alias [, alias …] [PARALLEL n];

Terms

alias The name of a relation.

ONSCHEMA
Use the ONSCHEMA clause to base the union on named fields (rather than positional notation). All inputs to the union must have a non-unknown (non-null) schema.
PARALLEL n This is only applicable for Tez execution mode and will not work with Mapreduce mode. Specifying PARALLEL will introduce an extra reduce step that will slightly degrade performance. The primary purpose in this case is to control the number of output files.
For more information, see Use the Parallel Features.

Usage

Use the UNION operator to merge the contents of two or more relations. The UNION operator:
  • Does not preserve the order of tuples. Both the input and output relations are interpreted as unordered bags of tuples.
  • Does not ensure (as databases do) that all tuples adhere to the same schema or that they have the same number of fields. In a typical scenario, however, this should be the case; therefore, it is the user's responsibility to either (1) ensure that the tuples in the input relations have the same schema or (2) be able to process varying tuples in the output relation.
  • Does not eliminate duplicate tuples.
Schema Behavior
The behavior of schemas for UNION (positional notation / data types) and UNION ONSCHEMA (named fields / data types) is the same, except where noted.
Union on relations with two different sizes result in a null schema (union only):
A: (a1:long, a2:long) 
B: (b1:long, b2:long, b3:long) 
A union B: null 
Union columns with incompatible types results in a failure. (See Types Table for addition and subtraction for incompatible types.)
A: (a1:long)
B: (a1:chararray)
A union B: ERROR: Cannot cast from long to bytearray
Union columns of compatible type will produce an "escalate" type. The priority is:
  • double > float > long > int > bytearray
  • tuple|bag|map|chararray > bytearray
A: (a1:int, a2:bytearray, a3:int) 
B: (b1:float, b2:chararray, b3:bytearray) 
A union B: (a1:float, a2:chararray, a3:int) 
Union of different inner types results in an empty complex type:
A: (a1:(a11:long, a12:int), a2:{(a21:charray, a22:int)}) 
B: (b1:(b11:int, b12:int), b2:{(b21:int, b22:int)}) 
A union B: (a1:(), a2:{()}) 
The alias of the first relation is always taken as the alias of the unioned relation field.

Example

In this example the union of relation A and B is computed.
A = LOAD 'data' AS (a1:int,a2:int,a3:int);

DUMP A;
(1,2,3)
(4,2,1)

B = LOAD 'data' AS (b1:int,b2:int);

DUMP A;
(2,4)
(8,9)
(1,3)

X = UNION A, B;

DUMP X;
(1,2,3)
(4,2,1)
(2,4)
(8,9)
(1,3)

Example

This example shows the use of ONSCHEMA.
L1 = LOAD 'f1' USING (a : int, b : float);
DUMP L1;
(11,12.0)
(21,22.0)

L2 = LOAD  'f1' USING (a : long, c : chararray);
DUMP L2;
(11,a)
(12,b)
(13,c)

U = UNION ONSCHEMA L1, L2;
DESCRIBE U ;
U : {a : long, b : float, c : chararray}

DUMP U;
(11,12.0,)
(21,22.0,)
(11,,a)
(12,,b)
(13,,c)

UDF Statements


DEFINE (UDFs, streaming)

Assigns an alias to a UDF or streaming command.

Syntax: UDF and streaming

DEFINE alias {function | [`command` [input] [output] [ship] [cache] [stderr] ] };

Terms

alias The name for a UDF function or the name for a streaming command (the cmd_alias for the STREAM operator).
function For use with functions.
The name of a UDF function.
`command` For use with streaming.
A command, including the arguments, enclosed in back tics (where a command is anything that can be executed).
The clauses (input, output, ship, cache, stderr) are described below. Note the following:
  • All clauses are optional.
  • The clauses can be specified in any order (for example, stderr can appear before input)
  • Each clause can be specified at most once (for example, multiple inputs are not allowed)
input For use with streaming.
INPUT ( {stdin | 'path'} [USING serializer] [, {stdin | 'path'} [USING serializer] …] )
Where:
  • INPUT – Keyword.
  • 'path' – A file path, enclosed in single quotes.
  • USING – Keyword.
  • serializer – PigStreaming is the default serializer.
output For use with streaming.
OUTPUT ( {stdout | stderr | 'path'} [USING deserializer] [, {stdout | stderr | 'path'} [USING deserializer] …] )
Where:
  • OUTPUT – Keyword.
  • 'path' – A file path, enclosed in single quotes.
  • USING – Keyword.
  • deserializer – PigStreaming is the default deserializer.

ship
For use with streaming.
SHIP('path' [, 'path' …])
Where:
  • SHIP – Keyword.
  • 'path' – A file path, enclosed in single quotes.

cache
For use with streaming.
CACHE('dfs_path#dfs_file' [, 'dfs_path#dfs_file' …])
Where:
  • CACHE – Keyword.
  • 'dfs_path#dfs_file' – A file path/file name on the distributed file system, enclosed in single quotes. Example: '/mydir/mydata.txt#mydata.txt'
stderr For use with streaming.
STDERR( '/dir') or STDERR( '/dir' LIMIT n)
Where:
  • '/dir' is the log directory, enclosed in single quotes.
  • (optional) LIMIT n is the error threshold where n is an integer value. If not specified, the default error threshold is unlimited.

Usage

Use the DEFINE statement to assign a name (alias) to a UDF function or to a streaming command.
Use DEFINE to specify a UDF function when:
  • The function has a long package name that you don't want to include in a script, especially if you call the function several times in that script.
  • The constructor for the function takes string parameters. If you need to use different constructor parameters for different calls to the function you will need to create multiple defines – one for each parameter set.
Use DEFINE to specify a streaming command when:
  • The streaming command specification is complex.
  • The streaming command specification requires additional parameters (input, output, and so on).

About Input and Output for Streaming
Serialization is needed to convert data from tuples to a format that can be processed by the streaming application. Deserialization is needed to convert the output from the streaming application back into tuples. PigStreaming is the default serialization/deserialization function.
Streaming uses the same default format as PigStorage to serialize/deserialize the data. If you want to explicitly specify a format, you can do it as show below (see more examples in the Examples: Input/Output section).
DEFINE CMD `perl PigStreaming.pl - nameMap` input(stdin using PigStreaming(',')) output(stdout using PigStreaming(','));
A = LOAD 'file';
B = STREAM B THROUGH CMD;
If you need an alternative format, you will need to create a custom serializer/deserializer by implementing the following interfaces.
interface PigToStream {

    /**
     * Given a tuple, produce an array of bytes to be passed to the streaming
     * executable.
     */
    public byte[] serialize(Tuple t) throws IOException;
}

interface StreamToPig {

    /**
     *  Given a byte array from a streaming executable, produce a tuple.
     */
    public Tuple deserialize(byte[]) throws IOException;

    /**
     * This will be called on both the front end and the back
     * end during execution.
     *
     * @return the {@link LoadCaster} associated with this object.
     * @throws IOException if there is an exception during LoadCaster
     */
    public LoadCaster getLoadCaster() throws IOException;
}

About Ship
Use the ship option to send streaming binary and supporting files, if any, from the client node to the compute nodes. Pig does not automatically ship dependencies; it is your responsibility to explicitly specify all the dependencies and to make sure that the software the processing relies on (for instance, perl or python) is installed on the cluster. Supporting files are shipped to the task's current working directory and only relative paths should be specified. Any pre-installed binaries should be specified in the PATH.
Only files, not directories, can be specified with the ship option. One way to work around this limitation is to tar all the dependencies into a tar file that accurately reflects the structure needed on the compute nodes, then have a wrapper for your script that un-tars the dependencies prior to execution.
Note that the ship option has two components: the source specification, provided in the ship( ) clause, is the view of your machine; the command specification is the view of the actual cluster. The only guarantee is that the shipped files are available in the current working directory of the launched job and that your current working directory is also on the PATH environment variable.
Shipping files to relative paths or absolute paths is not supported since you might not have permission to read/write/execute from arbitrary paths on the clusters.
Note the following:
  • It is safe only to ship files to be executed from the current working directory on the task on the cluster.
    OP = stream IP through 'script';
    or
    DEFINE CMD 'script' ship('/a/b/script');
    OP = stream IP through CMD;
    
  • Shipping files to relative paths or absolute paths is undefined and mostly will fail since you may not have permissions to read/write/execute from arbitraty paths on the actual clusters.

About Cache
The ship option works with binaries, jars, and small datasets. However, loading larger datasets at run time for every execution can severely impact performance. Instead, use the cache option to access large files already moved to and available on the compute nodes. Only files, not directories, can be specified with the cache option.

About Auto-Ship
If the ship and cache options are not specified, Pig will attempt to auto-ship the binary in the following way:
  • If the first word on the streaming command is perl or python, Pig assumes that the binary is the first non-quoted string it encounters that does not start with dash.
  • Otherwise, Pig will attempt to ship the first string from the command line as long as it does not come from /bin, /usr/bin, /usr/local/bin. Pig will determine this by scanning the path if an absolute path is provided or by executing which. The paths can be made configurable using the set stream.skippath option (you can use multiple set commands to specify more than one path to skip).
If you don't supply a DEFINE for a given streaming command, then auto-shipping is turned off.
Note the following:
  • If Pig determines that it needs to auto-ship an absolute path it will not ship it at all since there is no way to ship files to the necessary location (lack of permissions and so on).
    OP = stream IP through `/a/b/c/script`;
    or 
    OP = stream IP through `perl /a/b/c/script.pl`;
    
  • Pig will not auto-ship files in the following system directories (this is determined by executing 'which <file>' command).
    /bin /usr/bin /usr/local/bin /sbin /usr/sbin /usr/local/sbin
    
  • To auto-ship, the file in question should be present in the PATH. So if the file is in the current working directory then the current working directory should be in the PATH.

Examples: Input/Output

In this example PigStreaming is the default serialization/deserialization function. The tuples from relation A are converted to tab-delimited lines that are passed to the script.
X = STREAM A THROUGH `stream.pl`;
In this example PigStreaming is used as the serialization/deserialization function, but a comma is used as the delimiter.
DEFINE Y 'stream.pl' INPUT(stdin USING PigStreaming(',')) OUTPUT (stdout USING PigStreaming(','));

X = STREAM A THROUGH Y;
In this example user defined serialization/deserialization functions are used with the script.
DEFINE Y 'stream.pl' INPUT(stdin USING MySerializer) OUTPUT (stdout USING MyDeserializer);

X = STREAM A THROUGH Y;

Examples: Ship/Cache

In this example ship is used to send the script to the cluster compute nodes.
DEFINE Y 'stream.pl' SHIP('/work/stream.pl');

X = STREAM A THROUGH Y;
In this example cache is used to specify a file located on the cluster compute nodes.
DEFINE Y 'stream.pl data.gz' SHIP('/work/stream.pl') CACHE('/input/data.gz#data.gz');

X = STREAM A THROUGH Y;

Example: DEFINE with STREAM

In this example a command is defined for use with the STREAM operator.
A = LOAD 'data';

DEFINE mycmd 'stream_cmd –input file.dat';

B = STREAM A through mycmd;

Examples: Logging

In this example the streaming stderr is stored in the _logs/<dir> directory of the job's output directory. Because the job can have multiple streaming applications associated with it, you need to ensure that different directory names are used to avoid conflicts. Pig stores up to 100 tasks per streaming job.
DEFINE Y 'stream.pl' stderr('<dir>' limit 100);

X = STREAM A THROUGH Y;

Examples: DEFINE a function

In this example a function is defined for use with the FOREACH …GENERATE operator.
REGISTER /src/myfunc.jar

DEFINE myFunc myfunc.MyEvalfunc('foo');

A = LOAD 'students';

B = FOREACH A GENERATE myFunc($0);

REGISTER (a jar/script)

Registers a JAR file so that the UDFs in the file can be used.

Syntax

REGISTER path;

Terms

path The path to the JAR file (the full location URI is required). Do not place the name in quotes.

Usage

Pig Scripts
Use the REGISTER statement inside a Pig script to specify a JAR file or a Python/JavaScript module. Pig supports JAR files and modules stored in local file systems as well as remote, distributed file systems such as HDFS and Amazon S3 (see Pig Scripts).

Additionally, JAR files stored in local file systems can be specified as a glob pattern using “*”. Pig will search for matching jars in the local file system, either the relative path (relative to your working directory) or the absolute path. Pig will pick up all JARs that match the glob.
Command Line
You can register additional files (to use with your Pig script) via PIG_OPTS environment variable using the -Dpig.additional.jars.uris option. For more information see User Defined Functions.

Examples

In this example REGISTER states that the JavaScript module, myfunc.js, is located in the /src directory.
/src $ java -jar pig.jar –

REGISTER /src/myfunc.js;
A = LOAD 'students';
B = FOREACH A GENERATE myfunc.MyEvalFunc($0);
In this example additional JAR files are registered via PIG_OPTS environment variable.
export PIG_OPTS="-Dpig.additional.jars.uris=my.jar,your.jar"
In this example a JAR file stored in HDFS and a local JAR file are registered.
export PIG_OPTS="-Dpig.additional.jars.uris=hdfs://nn.mydomain.com:9020/myjars/my.jar,file:///home/root/pig/your.jar"
Note, the legacy property pig.additional.jars which use colon as separator is still supported. But we recommend to use pig.additional.jars.uris since colon is also used in URL scheme, and thus we cannot use full scheme in the list. We will deprecate pig.additional.jar in future releases.
This example shows how to specify a glob pattern using either a relative path or an absolute path.
register /homes/user/pig/myfunc*.jar
register count*.jar
register jars/*.jar

REGISTER (an artifact)

Instead of figuring out the dependencies manually, downloading them and registering each jar using the above register command, you can specify the artifact's coordinates and expect pig to automatically fetch the required dependencies, download and register them.

Syntax

To download an Artifact (and its dependencies), you need to specify the artifact's group, module and version following the syntax shown below. This command will download the Jar specified and all its dependencies and load it into the classpath.
REGISTER ivy://group:module:version?querystring

Terms

group Which module group the module comes from. Translates directly to a Maven groupId or an Ivy Organization.
module The name of the module to load. Translated directly to a Maven artifactId or an Ivy artifact.
version The version of the module to use. You can specify a specific version or use "+" or "*" to use the latest version.
querystring This will contain "&" separated key-value pairs to help us exclude all or specific dependencies etc.

Usage

The Register artifact command is an extension to the above register command used to register a jar. In addition to registering a jar from a local system or from hdfs, you can now specify the coordinates of the artifact and pig will download the artifact (and its dependencies if needed) from the configured repository.

Parameters Supported in the Query String
  • Transitive Transitive helps specifying if you need the dependencies along with the registering jar. By setting transitive to false in the querystring we can tell pig to register only the artifact without its dependencies. This will download only the artifact specified and will not download the dependencies of the jar. The default value of transitive is true. Syntax
    REGISTER ivy://org:module:version?transitive=false
  • Exclude While registering an artifact if you wish to exclude some dependencies you can specify them using the exclude key. Suppose you want to use a specific version of a dependent jar which doesn't match the version of the jar when automatically fetched, then you could exclude such dependencies by specifying a comma separated list of dependencies and register the dependent jar separately. Syntax
    REGISTER ivy://org:module:version?exclude=org:mod,org:mod,...
  • Classifier Some maven dependencies need classifiers in order to be able to resolve. You can specify them using a classifier key. Syntax
    REGISTER ivy://org:module:version?classifier=value

Other properties
  • An optional pig property, pig.artifacts.download.location, can be used to configure the location where the artifacts should be downloaded. By default, they will be downloaded to ~/.groovy/grapes
  • This command can be used or can replace the register jar command wherever used including macros.
  • Group/Organization and Version are optional fields. In such cases you can leave them blank.
  • The repositories can be configured using an ivysettings file. Pig will search for an ivysettings.xml file in the following locations in order. PIG_CONF_DIR > PIG_HOME > Classpath

Examples

  • Registering an Artifact and all its dependencies.
                    -- Both are the same
    
                    REGISTER ivy://org.apache.avro:avro:1.5.1
    
                    REGISTER ivy://org.apache.avro:avro:1.5.1?transitive=true
  • Registering an artifact without getting its dependencies.
                   REGISTER ivy://org.apache.avro:avro:1.5.1?transitive=false
  • Registering the latest artifact.
                    -- Both of the following syntaxes work.
    
                    REGISTER ivy://org.apache.avro:avro:+
    
                    REGISTER ivy://org.apache.avro:avro:*
  • Registering an artifact by excluding specific dependencies.
                    REGISTER ivy://org.apache.pig:pig:0.10.0?exclude=commons-cli:commons-cli,commons-codec:commons-codec
  • Specifying a classifier
                    REGISTER ivy://net.sf.json-lib:json-lib:2.4?classifier=jdk15
  • Registering an artifact without a group or organization. Just skip them.
                    REGISTER ivy://:module:

Introduction

Pig comes with a set of built in functions (the eval, load/store, math, string, bag and tuple functions). Two main properties differentiate built in functions from user defined functions (UDFs). First, built in functions don't need to be registered because Pig knows where they are. Second, built in functions don't need to be qualified when they are used because Pig knows where to find them.

Dynamic Invokers

Often you may need to use a simple function that is already provided by standard Java libraries, but for which a user defined functions (UDF) has not been written. Dynamic invokers allow you to refer to Java functions without having to wrap them in custom UDFs, at the cost of doing some Java reflection on every function call.
...
DEFINE UrlDecode InvokeForString('java.net.URLDecoder.decode', 'String String'); 
encoded_strings = LOAD 'encoded_strings.txt' as (encoded:chararray); 
decoded_strings = FOREACH encoded_strings GENERATE UrlDecode(encoded, 'UTF-8'); 
...
Currently, dynamic invokers can be used for any static function that:
  • Accepts no arguments or accepts some combination of strings, ints, longs, doubles, floats, or arrays with these same types
  • Returns a string, an int, a long, a double, or a float
Only primitives can be used for numbers; no capital-letter numeric classes can be used as arguments. Depending on the return type, a specific kind of invoker must be used: InvokeForString, InvokeForInt, InvokeForLong, InvokeForDouble, or InvokeForFloat.
The DEFINE statement is used to bind a keyword to a Java method, as above. The first argument to the InvokeFor* constructor is the full path to the desired method. The second argument is a space-delimited ordered list of the classes of the method arguments. This can be omitted or an empty string if the method takes no arguments. Valid class names are string, long, float, double, and int. Invokers can also work with array arguments, represented in Pig as DataBags of single-tuple elements. Simply refer to string[], for example. Class names are not case sensitive.
The ability to use invokers on methods that take array arguments makes methods like those in org.apache.commons.math.stat.StatUtils available (for processing the results of grouping your datasets, for example). This is helpful, but a word of caution: the resulting UDF will not be optimized for Hadoop, and the very significant benefits one gains from implementing the Algebraic and Accumulator interfaces are lost here. Be careful if you use invokers this way.

Eval Functions


AVG

Computes the average of the numeric values in a single-column bag.

Syntax

AVG(expression)

Terms

expression Any expression whose result is a bag. The elements of the bag should be data type int, long, float, double, bigdecimal, biginteger or bytearray.

Usage

Use the AVG function to compute the average of the numeric values in a single-column bag. AVG requires a preceding GROUP ALL statement for global averages and a GROUP BY statement for group averages.
The AVG function ignores NULL values.

Example

In this example the average GPA for each student is computed (see the GROUP operator for information about the field names in relation B).
A = LOAD 'student.txt' AS (name:chararray, term:chararray, gpa:float);

DUMP A;
(John,fl,3.9F)
(John,wt,3.7F)
(John,sp,4.0F)
(John,sm,3.8F)
(Mary,fl,3.8F)
(Mary,wt,3.9F)
(Mary,sp,4.0F)
(Mary,sm,4.0F)

B = GROUP A BY name;

DUMP B;
(John,{(John,fl,3.9F),(John,wt,3.7F),(John,sp,4.0F),(John,sm,3.8F)})
(Mary,{(Mary,fl,3.8F),(Mary,wt,3.9F),(Mary,sp,4.0F),(Mary,sm,4.0F)})

C = FOREACH B GENERATE A.name, AVG(A.gpa);

DUMP C;
({(John),(John),(John),(John)},3.850000023841858)
({(Mary),(Mary),(Mary),(Mary)},3.925000011920929)

Types Tables

int long float double bigdecimal biginteger chararray bytearray
AVG double double double double bigdecimal * bigdecimal * error cast as double
* Average values for datatypes bigdecimal and biginteger have precision setting java.math.MathContext.DECIMAL128.

BagToString

Concatenate the elements of a Bag into a chararray string, placing an optional delimiter between each value.

Syntax

BagToString(vals:bag [, delimiter:chararray])

Terms

vals A bag of arbitrary values. They will each be cast to chararray if they are not already.
delimiter A chararray value to place between elements of the bag; defaults to underscore '_'.

Usage

BagToString creates a single string from the elements of a bag, similar to SQL's GROUP_CONCAT function. Keep in mind the following:
  • Bags can be of arbitrary size, while strings in Java cannot: you will either exhaust available memory or exceed the maximum number of characters (about 2 billion). One of the worst features a production job can have is thresholding behavior: everything will seem nearly fine until the data size of your largest bag grows from nearly-too-big to just-barely-too-big.
  • Bags are disordered unless you explicitly apply a nested ORDER BY operation as demonstrated below. A nested FOREACH will preserve ordering, letting you order by one combination of fields then project out just the values you'd like to concatenate.
  • The default string conversion is applied to each element. If the bags contents are not atoms (tuple, map, etc), this may be not be what you want. Use a nested FOREACH to format values and then compose them with BagToString as shown below
Examples:
vals delimiter BagToString(vals, delimiter) Notes
{('BOS'),('NYA'),('BAL')} BOS_NYA_BAL If only one argument is given, the field is delimited with underscore characters
{('BOS'),('NYA'),('BAL')} '|' BOS|NYA|BAL But you can supply your own delimiter
{('BOS'),('NYA'),('BAL')} '' BOSNYABAL Use an explicit empty string to just smush everything together
{(1),(2),(3)} '|' 1|2|3 Elements are type-converted for you (but see examples below)

Examples

Simple delimited strings are simple:
team_parks = LOAD 'team_parks' AS (team_id:chararray, park_id:chararray, years:bag{(year_id:int)});

-- BOS     BOS07   {(1995),(1997),(1996),(1998),(1999)}
-- NYA     NYC16   {(1995),(1999),(1998),(1997),(1996)}
-- NYA     NYC17   {(1998)}
-- SDN     HON01   {(1997)}
-- SDN     MNT01   {(1996),(1999)}
-- SDN     SAN01   {(1999),(1997),(1998),(1995),(1996)}

team_parkslist = FOREACH (GROUP team_parks BY team_id) GENERATE
  group AS team_id, BagToString(team_parks.park_id, ';');

-- BOS     BOS07
-- NYA     NYC17;NYC16
-- SDN     SAN01;MNT01;HON01
The default handling of complex elements works, but probably isn't what you want.
team_parkyearsugly = FOREACH (GROUP team_parks BY team_id) GENERATE
  group AS team_id,
  BagToString(team_parks.(park_id, years));

-- BOS     BOS07_{(1995),(1997),(1996),(1998),(1999)}
-- NYA     NYC17_{(1998)}_NYC16_{(1995),(1999),(1998),(1997),(1996)}
-- SDN     SAN01_{(1999),(1997),(1998),(1995),(1996)}_MNT01_{(1996),(1999)}_HON01_{(1997)}
Instead, assemble it in pieces. In step 2, we sort on one field but process another; it remains in the sorted order.
team_park_yearslist = FOREACH team_parks {
  years_o = ORDER years BY year_id;
  GENERATE team_id, park_id, SIZE(years_o) AS n_years, BagToString(years_o, '/') AS yearslist;
};
team_parkyearslist = FOREACH (GROUP team_park_yearslist BY team_id) {
  tpy_o = ORDER team_park_yearslist BY n_years DESC, park_id ASC;
  tpy_f = FOREACH tpy_o GENERATE CONCAT(park_id, ':', yearslist);
  GENERATE group AS team_id, BagToString(tpy_f, ';');
  };

-- BOS     BOS07:1995/1996/1997/1998/1999
-- NYA     NYC16:1995/1996/1997/1998/1999;NYC17:1998
-- SDN     SAN01:1995/1996/1997/1998/1999;MNT01:1996/1999;HON01:1997

BagToTuple

Un-nests the elements of a bag into a tuple.

Syntax

BagToTuple(expression)

Terms

expression An expression with data type bag.

Usage

BagToTuple creates a tuple from the elements of a bag. It removes only the first level of nesting; it does not recursively un-nest nested bags. Unlike FLATTEN, BagToTuple will not generate multiple output records per input record.

Examples

In this example, a bag containing tuples with one field is converted to a tuple.
A = LOAD 'bag_data' AS (B1:bag{T1:tuple(f1:chararray)});

DUMP A;
({('a'),('b'),('c')})
({('d'),('e'),('f')})

X = FOREACH A GENERATE BagToTuple(B1);

DUMP X;
(('a','b','c'))
(('d','e','f'))
In this example, a bag containing tuples with two fields is converted to a tuple.
A = LOAD 'bag_data' AS (B1:bag{T1:tuple(f1:int,f2:int)});

DUMP A;
({(4,1),(7,8),(4,9)})
({(5,8),(4,3),(3,8)})

X = FOREACH A GENERATE BagToTuple(B1);

DUMP X;
((4,1,7,8,4,9))
((5,8,4,3,3,8))

Bloom

Bloom filters are a common way to select a limited set of records before moving data for a join or other heavy weight operation.

Syntax

BuildBloom(String hashType, String mode, String vectorSize, String nbHash)
Bloom(String filename)

Terms

hashtype The type of hash function to use. Valid values for the hash functions are 'jenkins' and 'murmur'.
mode Will be ignored, though by convention it should be "fixed" or "fixedsize"
vectorSize The number of bits in the bloom filter.
nbHash The number of hash functions used in constructing the bloom filter.
filename File containing the serialized Bloom filter.
See Bloom Filter for a discussion of how to select the number of bits and the number of hash functions.

Usage

Bloom filters are a common way to select a limited set of records before moving data for a join or other heavy weight operation. For example, if one wanted to join a very large data set L with a smaller set S, and it was known that the number of keys in L that will match with S is small, building a bloom filter on S and then applying it to L before the join can greatly reduce the number of records from L that have to be moved from the map to the reduce, thus speeding the join.
The implementation uses Hadoop's bloom filters (org.apache.hadoop.util.bloom.BloomFilter) internally.

Examples

  define bb BuildBloom('128', '3', 'jenkins');
  small = load 'S' as (x, y, z);
  grpd = group small all;
  fltrd = foreach grpd generate bb(small.x);
  store fltrd in 'mybloom';
  exec;
  define bloom Bloom('mybloom');
  large = load 'L' as (a, b, c);
  flarge = filter large by bloom(L.a);
  joined = join small by x, flarge by a;
  store joined into 'results';

CONCAT

Concatenates two or more expressions of identical type.

Syntax

CONCAT (expression, expression, [...expression])

Terms

expression Any expression.

Usage

Use the CONCAT function to concatenate two or more expressions. The result values of the expressions must have identical types.
If any subexpression is null, the resulting expression is null.

Example

In this example, fields f1, an underscore string literal, f2 and f3 are concatenated.
A = LOAD 'data' as (f1:chararray, f2:chararray, f3:chararray);

DUMP A;
(apache,open,source)
(hadoop,map,reduce)
(pig,pig,latin)

X = FOREACH A GENERATE CONCAT(f1, '_', f2,f3);

DUMP X;
(apache_opensource)
(hadoop_mapreduce)
(pig_piglatin)

COUNT

Computes the number of elements in a bag.

Syntax

COUNT(expression)

Terms

expression An expression with data type bag.

Usage

Use the COUNT function to compute the number of elements in a bag. COUNT requires a preceding GROUP ALL statement for global counts and a GROUP BY statement for group counts.
The COUNT function follows syntax semantics and ignores nulls. What this means is that a tuple in the bag will not be counted if the FIRST FIELD in this tuple is NULL. If you want to include NULL values in the count computation, use COUNT_STAR.
Note: You cannot use the tuple designator (*) with COUNT; that is, COUNT(*) will not work.

Example

In this example the tuples in the bag are counted (see the GROUP operator for information about the field names in relation B).
A = LOAD 'data' AS (f1:int,f2:int,f3:int);

DUMP A;
(1,2,3)
(4,2,1)
(8,3,4)
(4,3,3)
(7,2,5)
(8,4,3)

B = GROUP A BY f1;

DUMP B;
(1,{(1,2,3)})
(4,{(4,2,1),(4,3,3)})
(7,{(7,2,5)})
(8,{(8,3,4),(8,4,3)})

X = FOREACH B GENERATE COUNT(A);

DUMP X;
(1L)
(2L)
(1L)
(2L)

Types Tables

int long float double chararray bytearray
COUNT long long long long long long

COUNT_STAR

Computes the number of elements in a bag.

Syntax

COUNT_STAR(expression) 

Terms

expression An expression with data type bag.

Usage

Use the COUNT_STAR function to compute the number of elements in a bag. COUNT_STAR requires a preceding GROUP ALL statement for global counts and a GROUP BY statement for group counts.
COUNT_STAR includes NULL values in the count computation (unlike COUNT, which ignores NULL values).

Example

In this example COUNT_STAR is used to count the tuples in a bag.
X = FOREACH B GENERATE COUNT_STAR(A);

DIFF

Compares two fields in a tuple.

Syntax

DIFF (expression, expression)

Terms

expression An expression with any data type.

Usage

The DIFF function takes two bags as arguments and compares them. Any tuples that are in one bag but not the other are returned in a bag. If the bags match, an empty bag is returned. If the fields are not bags then they will be wrapped in tuples and returned in a bag if they do not match, or an empty bag will be returned if the two records match. The implementation assumes that both bags being passed to the DIFF function will fit entirely into memory simultaneously. If this is not the case the UDF will still function but it will be VERY slow.

Example

In this example DIFF compares the tuples in two bags.
A = LOAD 'bag_data' AS (B1:bag{T1:tuple(t1:int,t2:int)},B2:bag{T2:tuple(f1:int,f2:int)});

DUMP A;
({(8,9),(0,1)},{(8,9),(1,1)})
({(2,3),(4,5)},{(2,3),(4,5)})
({(6,7),(3,7)},{(2,2),(3,7)})

DESCRIBE A;
a: {B1: {T1: (t1: int,t2: int)},B2: {T2: (f1: int,f2: int)}}

X = FOREACH A GENERATE DIFF(B1,B2);

grunt> dump x;
({(0,1),(1,1)})
({})
({(6,7),(2,2)})

IsEmpty

Checks if a bag or map is empty.

Syntax

IsEmpty(expression)

Terms

expression An expression with any data type.

Usage

The IsEmpty function checks if a bag or map is empty (has no data). The function can be used to filter data.

Example

In this example all students with an SSN but no name are located.
SSN = load 'ssn.txt' using PigStorage() as (ssn:long);

SSN_NAME = load 'students.txt' using PigStorage() as (ssn:long, name:chararray);

/* do a cogroup of SSN with SSN_Name */
X = COGROUP SSN by ssn, SSN_NAME by ssn;

/* only keep those ssn's for which there is no name */
Y = filter X by IsEmpty(SSN_NAME);

MAX

Computes the maximum of the numeric values or chararrays in a single-column bag. MAX requires a preceding GROUP ALL statement for global maximums and a GROUP BY statement for group maximums.

Syntax

MAX(expression)       

Terms

expression An expression with data types int, long, float, double, bigdecimal, biginteger, chararray, datetime or bytearray.

Usage

Use the MAX function to compute the maximum of the numeric values or chararrays in a single-column bag.
The MAX function ignores NULL values.

Example

In this example the maximum GPA for all terms is computed for each student (see the GROUP operator for information about the field names in relation B).
A = LOAD 'student' AS (name:chararray, session:chararray, gpa:float);

DUMP A;
(John,fl,3.9F)
(John,wt,3.7F)
(John,sp,4.0F)
(John,sm,3.8F)
(Mary,fl,3.8F)
(Mary,wt,3.9F)
(Mary,sp,4.0F)
(Mary,sm,4.0F)

B = GROUP A BY name;

DUMP B;
(John,{(John,fl,3.9F),(John,wt,3.7F),(John,sp,4.0F),(John,sm,3.8F)})
(Mary,{(Mary,fl,3.8F),(Mary,wt,3.9F),(Mary,sp,4.0F),(Mary,sm,4.0F)})

X = FOREACH B GENERATE group, MAX(A.gpa);

DUMP X;
(John,4.0F)
(Mary,4.0F)

Types Tables

int long float double bigdecimal biginteger chararray datetime bytearray
MAX int long float double bigdecimal biginteger chararray datetime cast as double

MIN

Computes the minimum of the numeric values or chararrays in a single-column bag. MIN requires a preceding GROUP… ALL statement for global minimums and a GROUP … BY statement for group minimums.

Syntax

MIN(expression)

Terms

expression An expression with data types int, long, float, double, bigdecimal, biginteger, chararray, datetime or bytearray.

Usage

Use the MIN function to compute the minimum of a set of numeric values or chararrays in a single-column bag.
The MIN function ignores NULL values.

Example

In this example the minimum GPA for all terms is computed for each student (see the GROUP operator for information about the field names in relation B).
A = LOAD 'student' AS (name:chararray, session:chararray, gpa:float);

DUMP A;
(John,fl,3.9F)
(John,wt,3.7F)
(John,sp,4.0F)
(John,sm,3.8F)
(Mary,fl,3.8F)
(Mary,wt,3.9F)
(Mary,sp,4.0F)
(Mary,sm,4.0F)

B = GROUP A BY name;

DUMP B;
(John,{(John,fl,3.9F),(John,wt,3.7F),(John,sp,4.0F),(John,sm,3.8F)})
(Mary,{(Mary,fl,3.8F),(Mary,wt,3.9F),(Mary,sp,4.0F),(Mary,sm,4.0F)})

X = FOREACH B GENERATE group, MIN(A.gpa);

DUMP X;
(John,3.7F)
(Mary,3.8F)

Types Tables

int long float double bigdecimal biginteger chararray datetime bytearray
MIN int long float double bigdecimal biginteger chararray datetime cast as double

PluckTuple

Allows the user to specify a string prefix, and then filter for the columns in a relation that begin with that prefix or match that regex pattern. Optionally, include flag 'false' to filter for columns that do not match that prefix or match that regex pattern

Syntax

DEFINE pluck PluckTuple(expression1)
DEFINE pluck PluckTuple(expression1,expression3)
pluck(expression2)

Terms

expression1 A prefix to pluck by or an regex pattern to pluck by
expression2 The fields to apply the pluck to, usually '*'
expression3 A boolean flag to indicate whether to include or exclude matching columns

Usage

Example:
a = load 'a' as (x, y);
b = load 'b' as (x, y);
c = join a by x, b by x;
DEFINE pluck PluckTuple('a::');
d = foreach c generate FLATTEN(pluck(*));
describe c;
c: {a::x: bytearray,a::y: bytearray,b::x: bytearray,b::y: bytearray}
describe d;
d: {plucked::a::x: bytearray,plucked::a::y: bytearray}
DEFINE pluckNegative PluckTuple('a::','false');
d = foreach c generate FLATTEN(pluckNegative(*));
describe d;
d: {plucked::b::x: bytearray,plucked::b::y: bytearray}

SIZE

Computes the number of elements based on any Pig data type.

Syntax

SIZE(expression)

Terms

expression An expression with any data type.

Usage

Use the SIZE function to compute the number of elements based on the data type (see the Types Tables below). SIZE includes NULL values in the size computation. SIZE is not algebraic.
If the tested object is null, the SIZE function returns null.

Example

In this example the number of characters in the first field is computed.
A = LOAD 'data' as (f1:chararray, f2:chararray, f3:chararray);
(apache,open,source)
(hadoop,map,reduce)
(pig,pig,latin)

X = FOREACH A GENERATE SIZE(f1);

DUMP X;
(6L)
(6L)
(3L)

Types Tables

int returns 1
long returns 1
float returns 1
double returns 1
chararray returns number of characters in the array
bytearray returns number of bytes in the array
tuple returns number of fields in the tuple
bag returns number of tuples in bag
map returns number of key/value pairs in map

SUBTRACT

Bags subtraction, SUBTRACT(bag1, bag2) = bags composed of bag1 elements not in bag2

Syntax

SUBTRACT(expression, expression)

Terms

expression An expression with data type bag.

Usage

SUBTRACT takes two bags as arguments and returns a new bag composed of the tuples of first bag are not in the second bag.
If null, bag arguments are replaced by empty bags. If arguments are not bags, an IOException is thrown.
The implementation assumes that both bags being passed to the SUBTRACT function will fit entirely into memory simultaneously, if this is not the case, SUBTRACT will still function but will be very slow.

Example

In this example, SUBTRACT creates a new bag composed of B1 elements that are not in B2.
A = LOAD 'bag_data' AS (B1:bag{T1:tuple(t1:int,t2:int)},B2:bag{T2:tuple(f1:int,f2:int)});

DUMP A;
({(8,9),(0,1),(1,2)},{(8,9),(1,1)})
({(2,3),(4,5)},{(2,3),(4,5)})
({(6,7),(3,7),(3,7)},{(2,2),(3,7)})

DESCRIBE A;
A: {B1: {T1: (t1: int,t2: int)},B2: {T2: (f1: int,f2: int)}}

X = FOREACH A GENERATE SUBTRACT(B1,B2);

DUMP X;
({(0,1),(1,2)})
({})
({(6,7)})

SUM

Computes the sum of the numeric values in a single-column bag. SUM requires a preceding GROUP ALL statement for global sums and a GROUP BY statement for group sums.

Syntax

SUM(expression)       

Terms

expression An expression with data types int, long, float, double, bigdecimal, biginteger or bytearray cast as double.

Usage

Use the SUM function to compute the sum of a set of numeric values in a single-column bag.
The SUM function ignores NULL values.

Example

In this example the number of pets is computed. (see the GROUP operator for information about the field names in relation B).
A = LOAD 'data' AS (owner:chararray, pet_type:chararray, pet_num:int);

DUMP A;
(Alice,turtle,1)
(Alice,goldfish,5)
(Alice,cat,2)
(Bob,dog,2)
(Bob,cat,2) 

B = GROUP A BY owner;

DUMP B;
(Alice,{(Alice,turtle,1),(Alice,goldfish,5),(Alice,cat,2)})
(Bob,{(Bob,dog,2),(Bob,cat,2)})

X = FOREACH B GENERATE group, SUM(A.pet_num);
DUMP X;
(Alice,8L)
(Bob,4L)

Types Tables

int long float double bigdecimal biginteger chararray bytearray
SUM long long double double bigdecimal biginteger error cast as double

IN

IN operator allows you to easily test if an expression matches any value in a list of values. It is used to reduce the need for multiple OR conditions.

Syntax

IN (expression)

Terms

expression An expression with data types chararray, int, long, float, double, bigdecimal, biginteger or bytearray.

Usage

IN operator allows you to easily test if an expression matches any value in a list of values. It is used to help reduce the need for multiple OR conditions.

Example

In this example we filter out ID 4 and 6.
A = load 'data' using PigStorage(',') AS (id:int, first:chararray, last:chararray, gender:chararray);

DUMP A;
(1,Christine,Romero,Female)
(2,Sara,Hansen,Female)
(3,Albert,Rogers,Male)
(4,Kimberly,Morrison,Female)
(5,Eugene,Baker,Male)
(6,Ann,Alexander,Female)
(7,Kathleen,Reed,Female)
(8,Todd,Scott,Male)
(9,Sharon,Mccoy,Female)
(10,Evelyn,Rice,Female)

X = FILTER A BY id IN (4, 6);
DUMP X;
(4,Kimberly,Morrison,Female)
(6,Ann,Alexander,Female)
In this example, we're passing a BigInteger and using NOT operator, thereby negating the passed list of fields in the IN clause
A = load 'data' using PigStorage(',') AS (id:biginteger, first:chararray, last:chararray, gender:chararray); 
X = FILTER A BY NOT id IN (1, 3, 5, 7, 9); 
DUMP X;
 
(2,Sara,Hansen,Female)
(4,Kimberly,Morrison,Female)
(6,Ann,Alexander,Female)
(8,Todd,Scott,Male)
(10,Evelyn,Rice,Female)

TOKENIZE

Splits a string and outputs a bag of words.

Syntax

TOKENIZE(expression [, 'field_delimiter'])       

Terms

expression An expression with data type chararray.
'field_delimiter' An optional field delimiter (in single quotes).
If field_delimiter is null or not passed, the following will be used as delimiters: space [ ], double quote [ " ], coma [ , ] parenthesis [ () ], star [ * ].

Usage

Use the TOKENIZE function to split a string of words (all words in a single tuple) into a bag of words (each word in a single tuple).

Example

In this example the strings in each row are split.
A  = LOAD 'data' AS (f1:chararray);

DUMP A;
(Here is the first string.)
(Here is the second string.)
(Here is the third string.)

X = FOREACH A GENERATE TOKENIZE(f1);

DUMP X;
({(Here),(is),(the),(first),(string.)})
({(Here),(is),(the),(second),(string.)})
({(Here),(is),(the),(third),(string.)})
In this example a field delimiter is specified.
{code}
A = LOAD 'data' AS (f1:chararray);
B = FOREACH A GENERATE TOKENIZE (f1,'||');
DUMP B;
{code} 

Load/Store Functions

Load/store functions determine how data goes into Pig and comes out of Pig. Pig provides a set of built-in load/store functions, described in the sections below. You can also write your own load/store functions (see User Defined Functions).

Handling Compression

Support for compression is determined by the load/store function. PigStorage and TextLoader support gzip and bzip compression for both read (load) and write (store). BinStorage does not support compression.
To work with gzip compressed files, input/output files need to have a .gz extension. Gzipped files cannot be split across multiple maps; this means that the number of maps created is equal to the number of part files in the input location.
A = load 'myinput.gz';
store A into 'myoutput.gz';
To work with bzip compressed files, the input/output files need to have a .bz or .bz2 extension. Because the compression is block-oriented, bzipped files can be split across multiple maps.
A = load 'myinput.bz';
store A into 'myoutput.bz';
Note: PigStorage and TextLoader correctly read compressed files as long as they are NOT CONCATENATED bz/bz2 FILES generated in this manner:
  • cat *.bz > text/concat.bz
  • cat *.bz2 > text/concat.bz2
If you use concatenated bzip files with your Pig jobs, you will NOT see a failure but the results will be INCORRECT.

BinStorage

Loads and stores data in machine-readable format.

Syntax

BinStorage()       

Terms

none no parameters

Usage

Pig uses BinStorage to load and store the temporary data that is generated between multiple MapReduce jobs.
  • BinStorage works with data that is represented on disk in machine-readable format. BinStorage does NOT support compression.
  • BinStorage supports multiple locations (files, directories, globs) as input.
Occasionally, users use BinStorage to store their own data. However, because BinStorage is a proprietary binary format, the original data is never in BinStorage - it is always a derivation of some other data.
We have seen several examples of users doing something like this:
a = load 'b.txt' as (id, f);
b = group a by id;
store b into 'g' using BinStorage();
And then later:
a = load 'g/part*' using BinStorage() as (id, d:bag{t:(v, s)});
b = foreach a generate (double)id, flatten(d);
dump b;
There is a problem with this sequence of events. The first script does not define data types and, as the result, the data is stored as a bytearray and a bag with a tuple that contains two bytearrays. The second script attempts to cast the bytearray to double; however, since the data originated from a different loader, it has no way to know the format of the bytearray or how to cast it to a different type. To solve this problem, Pig:
  • Sends an error message when the second script is executed: "ERROR 1118: Cannot cast bytes loaded from BinStorage. Please provide a custom converter."
  • Allows you to use a custom converter to perform the casting.
    a = load 'g/part*' using BinStorage('Utf8StorageConverter') as (id, d:bag{t:(v, s)});
    b = foreach a generate (double)id, flatten(d);
    dump b;
    

Examples

In this example BinStorage is used with the LOAD and STORE functions.
A = LOAD 'data' USING BinStorage();

STORE X into 'output' USING BinStorage(); 
In this example BinStorage is used to load multiple locations.
A = LOAD 'input1.bin, input2.bin' USING BinStorage();
BinStorage does not track data lineage. When Pig uses BinStorage to move data between MapReduce jobs, Pig can figure out the correct cast function to use and apply it. However, as shown in the example below, when you store data using BinStorage and then use a separate Pig Latin script to read data (thus loosing the type information), it is your responsibility to correctly cast the data before storing it using BinStorage.
raw = load 'sampledata' using BinStorage() as (col1,col2, col3);
--filter out null columns
A = filter raw by col1#'bcookie' is not null;

B = foreach A generate col1#'bcookie'  as reqcolumn;
describe B;
--B: {regcolumn: bytearray}
X = limit B 5;
dump X;
(36co9b55onr8s)
(36co9b55onr8s)
(36hilul5oo1q1)
(36hilul5oo1q1)
(36l4cj15ooa8a)

B = foreach A generate (chararray)col1#'bcookie'  as convertedcol;
describe B;
--B: {convertedcol: chararray}
X = limit B 5;
dump X; 
()
()
()
()
()

JsonLoader, JsonStorage

Load or store JSON data.

Syntax

JsonLoader( ['schema'] )
JsonStorage( )

Terms

schema An optional Pig schema, in single quotes.

Usage

Use JsonLoader to load JSON data.
Use JsonStorage to store JSON data.
Note that there is no concept of delimit in JsonLoader or JsonStorage. The data is encoded in standard JSON format. JsonLoader optionally takes a schema as the construct argument.

Examples

In this example data is loaded with a schema.
a = load 'a.json' using JsonLoader('a0:int,a1:{(a10:int,a11:chararray)},a2:(a20:double,a21:bytearray),a3:[chararray]');  
In this example data is loaded without a schema; it assumes there is a .pig_schema (produced by JsonStorage) in the input directory.
a = load 'a.json' using JsonLoader(); 

PigDump

Stores data in UTF-8 format.

Syntax

PigDump()       

Terms

none no parameters

Usage

PigDump stores data as tuples in human-readable UTF-8 format.

Example

In this example PigDump is used with the STORE function.
STORE X INTO 'output' USING PigDump();

PigStorage

Loads and stores data as structured text files.

Syntax

PigStorage( [field_delimiter] , ['options'] )

Terms


field_delimiter
The default field delimiter is tab ('\t').
You can specify other characters as field delimiters; however, be sure to encase the characters in single quotes.

'options'
A string that contains space-separated options ('optionA optionB optionC')
Currently supported options are:
  • ('schema') - Stores the schema of the relation using a hidden JSON file.
  • ('noschema') - Ignores a stored schema during the load.
  • ('tagsource') - (deprecated, Use tagPath instead) Add a first column indicates the input file of the record.
  • ('tagPath') - Add a first column indicates the input path of the record.
  • ('tagFile') - Add a first column indicates the input file name of the record.

Usage

PigStorage is the default function used by Pig to load/store the data. PigStorage supports structured text files (in human-readable UTF-8 format) in compressed or uncompressed form (see Handling Compression). All Pig data types (both simple and complex) can be read/written using this function. The input data to the load can be a file, a directory or a glob.
Load/Store Statements
Load statements – PigStorage expects data to be formatted using field delimiters, either the tab character ('\t') or other specified character.
Store statements – PigStorage outputs data using field delimiters, either the tab character ('\t') or other specified character, and the line feed record delimiter ('\n').
Field/Record Delimiters
Field Delimiters – For load and store statements the default field delimiter is the tab character ('\t'). You can use other characters as field delimiters, but separators such as ^A or Ctrl-A should be represented in Unicode (\u0001) using UTF-16 encoding (see Wikipedia ASCII, Unicode, and UTF-16).
Record Deliminters – For load statements Pig interprets the line feed ( '\n' ), carriage return ( '\r' or CTRL-M) and combined CR + LF ( '\r\n' ) characters as record delimiters (do not use these characters as field delimiters). For store statements Pig uses the line feed ('\n') character as the record delimiter.
Schemas
If the schema option is specified, a hidden ".pig_schema" file is created in the output directory when storing data. It is used by PigStorage (with or without -schema) during loading to determine the field names and types of the data without the need for a user to explicitly provide the schema in an as clause, unless noschema is specified. No attempt to merge conflicting schemas is made during loading. The first schema encountered during a file system scan is used.
Additionally, if the schema option is specified, a ".pig_headers" file is created in the output directory. This file simply lists the delimited aliases. This is intended to make export to tools that can read files with header lines easier (just cat the header to your data).
If the schema option is NOT specified, a schema will not be written when storing data.
If the noschema option is NOT specified, and a schema is found, it gets loaded when loading data.
Note that regardless of whether or not you store the schema, you always need to specify the correct delimiter to read your data. If you store using delimiter "#" and then load using the default delimiter, your data will not be parsed correctly.
Record Provenance
If tagPath or tagFile option is specified, PigStorage will add a pseudo-column INPUT_FILE_PATH or INPUT_FILE_NAME respectively to the beginning of the record. As the name suggests, it is the input file path/name containing this particular record. Please note tagsource is deprecated.
Complex Data Types
The formats for complex data types are shown here:
  • Tuple: enclosed by (), items separated by ","
    • Non-empty tuple: (item1,item2,item3)
    • Empty tuple is valid: ()
  • Bag: enclosed by {}, tuples separated by ","
    • Non-empty bag: {code}{(tuple1),(tuple2),(tuple3)}{code}
    • Empty bag is valid: {}
  • Map: enclosed by [], items separated by ",", key and value separated by "#"
    • Non-empty map: [key1#value1,key2#value2]
    • Empty map is valid: []
If load statement specify a schema, Pig will convert the complex type according to schema. If conversion fails, the affected item will be null (see Nulls and Pig Latin).

Examples

In this example PigStorage expects input.txt to contain tab-separated fields and newline-separated records. The statements are equivalent.
A = LOAD 'student' USING PigStorage('\t') AS (name: chararray, age:int, gpa: float); 

A = LOAD 'student' AS (name: chararray, age:int, gpa: float);
In this example PigStorage stores the contents of X into files with fields that are delimited with an asterisk ( * ). The STORE statement specifies that the files will be located in a directory named output and that the files will be named part-nnnnn (for example, part-00000).
STORE X INTO  'output' USING PigStorage('*');
In this example, PigStorage loads data with complex data type, a bag of map and double.
a = load '1.txt' as (a0:{t:(m:map[int],d:double)});

{([foo#1,bar#2],34.0),([white#3,yellow#4],45.0)} : valid
{([foo#badint],baddouble)} : conversion fail for badint/baddouble, get {([foo#],)}
{} : valid, empty bag

TextLoader

Loads unstructured data in UTF-8 format.

Syntax

TextLoader()

Terms

none no parameters

Usage

TextLoader works with unstructured data in UTF8 format. Each resulting tuple contains a single field with one line of input text. TextLoader also supports compression.
Currently, TextLoader support for compression is limited.
TextLoader cannot be used to store data.

Example

In this example TextLoader is used with the LOAD function.
A = LOAD 'data' USING TextLoader();

HBaseStorage

Loads and stores data from an HBase table.

Syntax

HBaseStorage('columns', ['options'])

Terms

columns A list of qualified HBase columns to read data from or store data to. The column family name and column qualifier are seperated by a colon (:). Only the columns used in the Pig script need to be specified. Columns are specified in one of three different ways as described below.
  • Explicitly specify a column family and column qualifier (e.g., user_info:id). This will produce a scalar in the resultant tuple.
  • Specify a column family and a portion of column qualifier name as a prefix followed by an asterisk (i.e., user_info:address_*). This approach is used to read one or more columns from the same column family with a matching descriptor prefix. The datatype for this field will be a map of column descriptor name to field value. Note that combining this style of prefix with a long list of fully qualified column descriptor names could cause perfomance degredation on the HBase scan. This will produce a Pig map in the resultant tuple with column descriptors as keys.
  • Specify all the columns of a column family using the column family name followed by an asterisk (i.e., user_info:*). This will produce a Pig map in the resultant tuple with column descriptors as keys.
'options' A string that contains space-separated options (‘-optionA=valueA -optionB=valueB -optionC=valueC’)
Currently supported options are:
  • -loadKey=(true|false) Load the row key as the first value in every tuple returned from HBase (default=false)
  • -gt=minKeyVal Return rows with a rowKey greater than minKeyVal
  • -lt=maxKeyVal Return rows with a rowKey less than maxKeyVal
  • -regex=regex Return rows with a rowKey that match this regex on KeyVal
  • -gte=minKeyVal Return rows with a rowKey greater than or equal to minKeyVal
  • -lte=maxKeyVal Return rows with a rowKey less than or equal to maxKeyVal
  • -limit=numRowsPerRegion Max number of rows to retrieve per region
  • -caching=numRows Number of rows to cache (faster scans, more memory)
  • -delim=delimiter Column delimiter in columns list (default is whitespace)
  • -ignoreWhitespace=(true|false) When delim is set to something other than whitespace, ignore spaces when parsing column list (default=true)
  • -caster=(HBaseBinaryConverter|Utf8StorageConverter) Class name of Caster to use to convert values (default=Utf8StorageConverter). The default caster can be overridden with the pig.hbase.caster config param. Casters must implement LoadStoreCaster.
  • -noWAL=(true|false) During storage, sets the write ahead to false for faster loading into HBase (default=false). To be used with extreme caution since this could result in data loss (see http://hbase.apache.org/book.html#perf.hbase.client.putwal).
  • -minTimestamp=timestamp Return cell values that have a creation timestamp greater or equal to this value
  • -maxTimestamp=timestamp Return cell values that have a creation timestamp less than this value
  • -timestamp=timestamp Return cell values that have a creation timestamp equal to this value
  • -includeTimestamp=Record will include the timestamp after the rowkey on store (rowkey, timestamp, ...)
  • -includeTombstone=Record will include a tombstone marker on store after the rowKey and timestamp (if included) (rowkey, [timestamp,] tombstone, ...)

Usage

HBaseStorage stores and loads data from HBase. The function takes two arguments. The first argument is a space seperated list of columns. The second optional argument is a space seperated list of options. Column syntax and available options are listed above. Note that HBaseStorage always disable split combination.

Load Example

In this example HBaseStorage is used with the LOAD function with an explicit schema.
raw = LOAD 'hbase://SomeTableName'
      USING org.apache.pig.backend.hadoop.hbase.HBaseStorage(
      'info:first_name info:last_name tags:work_* info:*', '-loadKey=true -limit=5') AS
      (id:bytearray, first_name:chararray, last_name:chararray, tags_map:map[], info_map:map[]);
The datatypes of the columns are declared with the "AS" clause. The first_name and last_name columns are specified as fully qualified column names with a chararray datatype. The third specification of tags:work_* requests a set of columns in the tags column family that begin with "work_". There can be zero, one or more columns of that type in the HBase table. The type is specified as tags_map:map[]. This indicates that the set of column values returned will be accessed as a map, where the key is the column name and the value is the cell value of the column. The fourth column specification is also a map of column descriptors to cell values.
When the type of the column is specified as a map in the "AS" clause, the map keys are the column descriptor names and the data type is chararray. The datatype of the columns values can be declared explicitly as shown in the examples below:
  • tags_map[chararray] - In this case, the column values are all declared to be of type chararray
  • tags_map[int] - In this case, the column values are all declared to be of type int.

Store Example

In this example HBaseStorage is used to store a relation into HBase.
A = LOAD 'hdfs_users' AS (id:bytearray, first_name:chararray, last_name:chararray);
STORE A INTO 'hbase://users_table' USING org.apache.pig.backend.hadoop.hbase.HBaseStorage(
    'info:first_name info:last_name');
In the example above relation A is loaded from HDFS and stored in HBase. Note that the schema of relation A is a tuple of size 3, but only two column descriptor names are passed to the HBaseStorage constructor. This is because the first entry in the tuple is used as the HBase rowKey.

AvroStorage

Loads and stores data from Avro files.

Syntax

AvroStorage(['schema|record name'], ['options'])

Terms

schema A JSON string specifying the Avro schema for the input. You may specify an explicit schema when storing data or when loading data. When you manually provide a schema, Pig will use the provided schema for serialization and deserialization. This means that you can provide an explicit schema when saving data to simplify the output (for example by removing nullable unions), or rename fields. This also means that you can provide an explicit schema when reading data to only read a subset of the fields in each record.
See the Apache Avro Documentation for more details on how to specify a valid schema.
record name When storing a bag of tuples with AvroStorage, if you do not want to specify the full schema, you may specify the avro record name instead. (AvroStorage will determine that the argument isn't a valid schema definition and use it as a variable name instead.)
'options' A string that contains space-separated options (‘-optionA valueA -optionB valueB -optionC ’)
Currently supported options are:
  • -namespace nameSpace or -n nameSpace Explicitly specify the namespace field in Avro records when storing data
  • -schemfile schemaFile or -f schemaFile Specify the input (or output) schema from an external file. Pig assumes that the file is located on the default filesystem, but you may use an explicity URL to unambigously specify the location. (For example, if the data was on your local file system in /stuff/schemafile.avsc, you could specify "-f file:///stuff/schemafile.avsc" to specify the location. If the data was on HDFS under /yourdirectory/schemafile.avsc, you could specify "-f hdfs:///yourdirectory/schemafile.avsc"). Pig expects this to be a text file, containing a valid avro schema.
  • -examplefile exampleFile or -e exampleFile Specify the input (or output) schema using another Avro file as an example. Pig assumes that the file is located on the default filesystem, but you may use and explicity URL to specify the location. Pig expects this to be an Avro data file.
  • -allowrecursive or -r Specify whether to allow recursive schema definitions (the default is to throw an exception if Pig encounters a recursive schema). When reading objects with recursive definitions, Pig will translate Avro records to schema-less tuples; the Pig Schema for the object may not match the data exactly.
  • -doublecolons or -d Specify how to handle Pig schemas that contain double-colons when writing data in Avro format. (When you join two bags in Pig, Pig will automatically label the fields in the output Tuples with names that contain double-colons). If you select this option, AvroStorage will translate names with double colons into names with double underscores.

Usage

AvroStorage stores and loads data from Avro files. Often, you can load and store data using AvroStorage without knowing much about the Avros serialization format. AvroStorage will attempt to automatically translate a pig schema and pig data to avro data, or avro data to pig data.
By default, when you use AvoStorage to load data, AvroStorage will use depth first search to find a valid Avro file on the input path, then use the schema from that file to load the data. When you use AvroStorage to store data, AvroStorage will attempt to translate the Pig schema to an equivalent Avro schema. You can manually specify the schema by providing an explicit schema in Pig, loading a schema from an external schema file, or explicitly telling Pig to read the schema from a specific avro file.
To compress your output with AvroStorage, you need to use the correct Avro properties for compression. For example, to enable compression using deflate level 5, you would specify
SET avro.output.codec 'deflate'
SET avro.mapred.deflate.level 5
Valid values for avro.output.codec include deflate, snappy, and null.
There are a few key differences between Avro and Pig data, and in some cases it helps to understand the differences between the Avro and Pig data models. Before writing Pig data to Avro (or creating Avro files to use in Pig), keep in mind that there might not be an equivalent Avro Schema for every Pig Schema (and vice versa):
  • Recursive schema definitions You cannot define schemas recursively in Pig, but you can define schemas recursively in Avro.
  • Allowed characters Pig schemas may sometimes contain characters like colons (":") that are illegal in Avro names.
  • Unions In Avro, you can define an object that may be one of several different types (including complex types such as records). In Pig, you cannot.
  • Enums Avro allows you to define enums to efficiently and abstractly represent categorical variable, but Pig does not.
  • Fixed Length Byte Arrays Avro allows you to define fixed length byte arrays, but Pig does not.
  • Nullable values In Pig, all types are nullable. In Avro, they are not.
Here is how AvroStorage translates Pig values to Avro:
Original Pig Type Translated Avro Type
Integers int ["int","null"]
Longs long ["long,"null"]
Floats float ["float","null"]
Doubles double ["double","null"]
Strings chararray ["string","null"]
Bytes bytearray ["bytes","null"]
Booleans boolean ["boolean","null"]
Tuples tuple The Pig Tuple schema will be translated to an union of and Avro record with an equivalent schem and null.
Bags of Tuples bag The Pig Tuple schema will be translated to a union of an array of records with an equivalent schema and null.
Maps map The Pig Tuple schema will be translated to a union of a map of records with an equivalent schema and null.
Here is how AvroStorage translates Avro values to Pig:
Original Avro Types Translated Pig Type
Integers ["int","null"] or "int" int
Longs ["long,"null"] or "long" long
Floats ["float","null"] or "float" float
Doubles ["double","null"] or "double" double
Strings ["string","null"] or "string" chararray
Enums Either an enum or a union of an enum and null chararray
Bytes ["bytes","null"] or "bytes" bytearray
Fixes Either a fixed length byte array, or a union of a fixed length array and null bytearray
Booleans ["boolean","null"] or "boolean" boolean
Tuples Either a record type, or a union or a record and null tuple
Bags of Tuples Either an array, or a union of an array and null bag
Maps Either a map, or a union of a map and null map
In many cases, AvroStorage will automatically translate your data correctly and you will not need to provide any more information to AvroStorage. But sometimes, it may be convenient to manually provide a schema to AvroStorge. See the example selection below for examples on manually specifying a schema with AvroStorage.

Load Examples

Suppose that you were provided with a file of avro data (located in 'stuff') with the following schema:
{"type" : "record",
 "name" : "stuff",
 "fields" : [
   {"name" : "label", "type" : "string"}, 
   {"name" : "value", "type" : "int"},
   {"name" : "marketingPlans", "type" : ["string", "bytearray", "null"]}
  ]
}
Additionally, suppose that you don't need the value of the field "marketingPlans." (That's a good thing, because AvroStorage doesn't know how to translate that Avro schema to a Pig schema). To load only the fieds "label" and "value" into Pig, you can manually specify the schema passed to AvroStorage:
measurements = LOAD 'stuff' USING AvroStorage(
  '{"type":"record","name":"measurement","fields":[{"name":"label","type":"string"},{"name":"value","type":"int"}]}'
  );

Store Examples

Suppose that you are saving a bag called measurements with the schema:
measurements:{measurement:(label:chararray,value:int)}
To store this bag into a file called "measurements", you can use a statement like:
STORE measurements INTO 'measurements' USING AvroStorage('measurement');
AvroStorage will translate this to the Avro schema
{"type":"record", 
 "name":"measurement",
 "fields" : [
   {"name" : "label", "type" : ["string", "null"]}, 
   {"name" : "value", "type" : ["int", "null"]}
  ]
} 
But suppose that you knew that the label and value fields would never be null. You could define a more precise schema manually using a statement like:
STORE measurements INTO 'measurements' USING AvroStorage(
  '{"type":"record","name":"measurement","fields":[{"name":"label","type":"string"},{"name":"value","type":"int"}]}'
  );

TrevniStorage

Loads and stores data from Trevni files.

Syntax

TrevniStorage(['schema|record name'], ['options'])
Trevni is a column-oriented storage format that is part of the Apache Avro project. Trevni is closely related to Avro.
Likewise, TrevniStorage is very closely related to AvroStorage, and shares the same options as AvroStorage. See AvroStorage for a detailed description of the arguments for TrevniStorage.

AccumuloStorage

Loads or stores data from an Accumulo table. The first element in a Tuple is equivalent to the "row" from the Accumulo Key, while the columns in that row are can be grouped in various static or wildcarded ways. Basic wildcarding functionality exists to group various columns families/qualifiers into a Map for LOADs, or serialize a Map into some group of column families or qualifiers on STOREs.

Syntax

AccumuloStorage(['columns'[, 'options']])

Arguments

'columns' A comma-separated list of "columns" to read data from to write data to. Each of these columns can be considered one of three different types:
  1. Literal
  2. Column family prefix
  3. Column qualifier prefix
Literal: this is the simplest specification which is a colon-delimited string that maps to a column family and column qualifier. This will read/write a simple scalar from/to Accumulo.
Column family prefix: When reading data, this will fetch data from Accumulo Key-Values in the current row whose column family match the given prefix. This will result in a Map being placed into the Tuple. When writing data, a Map is also expected at the given offset in the Tuple whose Keys will be appended to the column family prefix, an empty column qualifier is used, and the Map value will be placed in the Accumulo Value. A valid column family prefix is a literal asterisk (*) in which case the Map Key will be equivalent to the Accumulo column family.
Column qualifier prefix: Similar to the column family prefix except it operates on the column qualifier. On reads, Accumulo Key-Values in the same row that match the given column family and column qualifier prefix will be placed into a single Map. On writes, the provided column family from the column specification will be used, the Map key will be appended to the column qualifier provided in the specification, and the Map Value will be the Accumulo Value.
When "columns" is not provided or is a blank String, it is treated equivalently to "*". This is to say that when a column specification string is not provided, for reads, all columns in the given Accumulo row will be placed into a single Map (with the Map keys being colon delimited to preserve the column family/qualifier from Accumulo). For writes, the Map keys will be placed into the column family and the column qualifier will be empty.
'options' A string that contains space-separated options ("optionA valueA -optionB valueB -optionC valueC")
The currently supported options are:
  • (-c|--caster) LoadStoreCasterImpl An implementation of a LoadStoreCaster to use when serializing types into Accumulo, usually AccumuloBinaryConverter or UTF8StringConverter, defaults to UTF8StorageConverter.
  • (-auths|--authorizations) auth1,auth2... A comma-separated list of Accumulo authorizations to use when reading data from Accumulo. Defaults to the empty set of authorizations (none).
  • (-s|--start) start_row The Accumulo row to begin reading from, inclusive
  • (-e|--end) end_row The Accumulo row to read until, inclusive
  • (-buff|--mutation-buffer-size) num_bytes The number of bytes to buffer when writing data to Accumulo. A higher value requires more memory
  • (-wt|--write-threads) num_threads The number of threads used to write data to Accumulo.
  • (-ml|--max-latency) milliseconds Maximum time in milliseconds before data is flushed to Accumulo.
  • (-sep|--separator) str The separator character used when parsing the column specification, defaults to comma (,)
  • (-iw|--ignore-whitespace) (true|false) Should whitespace be stripped from the column specification, defaults to true

Usage

AccumuloStorage has the functionality to store or fetch data from Accumulo. Its goal is to provide a simple, widely applicable table schema compatible with Pig's API. Each Tuple contains some subset of the columns stored within one row of the Accumulo table, which depends on the columns provided as an argument to the function. If '*' is provided, all columns in the table will be returned. The second argument provides control over a variety of options that can be used to change various properties.
When invoking Pig Scripts that use AccumuloStorage, it's important to ensure that Pig has the Accumulo jars on its classpath. This is easily achieved using the ACCUMULO_HOME environment variable.
PIG_CLASSPATH="$ACCUMULO_HOME/lib/*:$PIG_CLASSPATH" pig my_script.pig

Load Example

It is simple to fetch all columns from Airport codes that fall between Boston and San Francisco that can be viewed with 'auth1' and/or 'auth2' Accumulo authorizations.
raw = LOAD 'accumulo://airports?instance=accumulo&user=root&password=passwd&zookeepers=localhost'
      USING org.apache.pig.backend.hadoop.accumulo.AccumuloStorage(
      '*', '-a auth1,auth2 -s BOS -e SFO') AS
      (code:chararray, all_columns:map[]);
The datatypes of the columns are declared with the "AS" clause. In this example, the row key, which is the unique airport code is assigned to the "code" variable while all of the other columns are placed into the map. When there is a non-empty column qualifier, the key in that map will have a colon which separates which portion of the key came from the column family and which portion came from the column qualifier. The Accumulo value is placed in the Map value.
Most times, it is not necessary, nor desired for performance reasons, to fetch all columns.
raw = LOAD 'accumulo://airports?instance=accumulo&user=root&password=passwd&zookeepers=localhost'
      USING org.apache.pig.backend.hadoop.accumulo.AccumuloStorage(
      'name,building:num_terminals,carrier*,reviews:transportation*') AS
      (code:chararray name:bytearray carrier_map:map[] transportion_reviews_map:map[]);
An asterisk can be used when requesting columns to group a collection of columns into a single Map instead of enumerating each column.

Store Example

Data can be easily stored into Accumulo.
A = LOAD 'flights.txt' AS (id:chararray, carrier_name:chararray, src_airport:chararray, dest_airport:chararray, tail_number:int);
STORE A INTO 'accumulo://flights?instance=accumulo&user=root&password=passwd&zookeepers=localhost' USING 
    org.apache.pig.backend.hadoop.accumulo.AccumuloStorage('carrier_name,src_airport,dest_airport,tail_number');
Here, we read the file 'flights.txt' out of HDFS and store the results into the relation A. We extract a unique ID for the flight, its source and destination and the tail number from the given file. When STORE'ing back into Accumulo, we specify the column specifications (in this case, just a column family). It is also important to note that four elements are provided as columns because the first element in the Tuple is used as the row in Accumulo.

OrcStorage

Loads from or stores data to Orc file.

Syntax

OrcStorage(['options'])

Options

A string that contains space-separated options (‘-optionA valueA -optionB valueB -optionC ’). Current options are only applicable with STORE operation and not for LOAD.
Currently supported options are:
  • --stripeSize or -s Set the stripe size for the file. Default is 268435456(256 MB).
  • --rowIndexStride or -r Set the distance between entries in the row index. Default is 10000.
  • --bufferSize or -b Set the size of the memory buffers used for compressing and storing the stripe in memory. Default is 262144 (256K).
  • --blockPadding or -p Sets whether the HDFS blocks are padded to prevent stripes from straddling blocks. Default is true.
  • --compress or -c Sets the generic compression that is used to compress the data. Valid codecs are: NONE, ZLIB, SNAPPY, LZO. Default is ZLIB.
  • --version or -v Sets the version of the file that will be written

Example

OrcStorage as a StoreFunc.
A = LOAD 'student.txt' as (name:chararray, age:int, gpa:double);
store A into 'student.orc' using OrcStorage('-c SNAPPY'); -- store student.txt into data.orc with SNAPPY compression
OrcStorage as a LoadFunc.
A = LOAD 'student.orc' USING OrcStorage();
describe A; -- See the schema of student.orc
B = filter A by age > 25 and gpa < 3; -- filter condition will be pushed up to loader
dump B; -- dump the content of student.orc

Data types

Most Orc data type has one to one mapping to Pig data type. Several exceptions are:
Loader side:
  • Orc STRING/CHAR/VARCHAR all map to Pig varchar
  • Orc BYTE/BINARY all map to Pig bytearray
  • Orc TIMESTAMP/DATE all maps to Pig datetime
  • Orc DECIMAL maps to Pig bigdecimal
Storer side:
  • Pig chararray maps to Orc STRING
  • Pig datetime maps to Orc TIMESTAMP
  • Pig bigdecimal/biginteger all map to Orc DECIMAL
  • Pig bytearray maps to Orc BINARY

Predicate pushdown

If there is a filter statement right after OrcStorage, Pig will push the filter condition to the loader. OrcStorage will prune file/stripe/row group which does not satisfy the condition entirely. For the file/stripe/row group contains data that satisfies the filter condition, OrcStorage will load the file/stripe/row group and Pig will evaluate the filter condition again to remove additional data which does not satisfy the filter condition.
OrcStorage predicate pushdown currently support all primitive data types but none of the complex data types. For example, map condition cannot push into OrcStorage:
A = LOAD 'student.orc' USING OrcStorage();
B = filter A by info#'age' > 25; -- map condition cannot push to OrcStorage
dump B;
Currently, the following expressions in filter condition are supported in OrcStorage predicate pushdown: >, >=, <, <=, ==, !=, between, in, and, or, not. The missing expressions are: is null, is not null, matches.

Math Functions

For general information about these functions, see the Java API Specification, Class Math. Note the following:
  • Pig function names are case sensitive and UPPER CASE.
  • Pig may process results differently than as stated in the Java API Specification:
    • If the result value is null or empty, Pig returns null.
    • If the result value is not a number (NaN), Pig returns null.
    • If Pig is unable to process the expression, Pig returns an exception.

ABS

Returns the absolute value of an expression.

Syntax

ABS(expression)

Terms

expression Any expression whose result is type int, long, float, or double.

Usage

Use the ABS function to return the absolute value of an expression. If the result is not negative (x ≥ 0), the result is returned. If the result is negative (x < 0), the negation of the result is returned.

ACOS

Returns the arc cosine of an expression.

Syntax

ACOS(expression)

Terms

expression An expression whose result is type double.

Usage

Use the ACOS function to return the arc cosine of an expression.

ASIN

Returns the arc sine of an expression.

Syntax

ASIN(expression)

Terms

expression An expression whose result is type double.

Usage

Use the ASIN function to return the arc sine of an expression.

ATAN

Returns the arc tangent of an expression.

Syntax

ATAN(expression)

Terms

expression An expression whose result is type double.

Usage

Use the ATAN function to return the arc tangent of an expression.

CBRT

Returns the cube root of an expression.

Syntax

CBRT(expression)

Terms

expression An expression whose result is type double.

Usage

Use the CBRT function to return the cube root of an expression.

CEIL

Returns the value of an expression rounded up to the nearest integer.

Syntax

CEIL(expression)

Terms

expression An expression whose result is type double.

Usage

Use the CEIL function to return the value of an expression rounded up to the nearest integer. This function never decreases the result value.
x CEIL(x)
4.6 5
3.5 4
2.4 3
1.0 1
-1.0 -1
-2.4 -2
-3.5 -3
-4.6 -4

COS

Returns the trigonometric cosine of an expression.

Syntax

COS(expression)

Terms

expression An expression (angle) whose result is type double.

Usage

Use the COS function to return the trigonometric cosine of an expression.

COSH

Returns the hyperbolic cosine of an expression.

Syntax

COSH(expression)

Terms

expression An expression whose result is type double.

Usage

Use the COSH function to return the hyperbolic cosine of an expression.

EXP

Returns Euler's number e raised to the power of x.

Syntax

EXP(expression)

Terms

expression An expression whose result is type double.

Usage

Use the EXP function to return the value of Euler's number e raised to the power of x (where x is the result value of the expression).

FLOOR

Returns the value of an expression rounded down to the nearest integer.

Syntax

FLOOR(expression)

Terms

expression An expression whose result is type double.

Usage

Use the FLOOR function to return the value of an expression rounded down to the nearest integer. This function never increases the result value.
x FLOOR(x)
4.6 4
3.5 3
2.4 2
1.0 1
-1.0 -1
-2.4 -3
-3.5 -4
-4.6 -5

LOG

Returns the natural logarithm (base e) of an expression.

Syntax

LOG(expression)

Terms

expression An expression whose result is type double.

Usage

Use the LOG function to return the natural logarithm (base e) of an expression.

LOG10

Returns the base 10 logarithm of an expression.

Syntax

LOG10(expression)

Terms

expression An expression whose result is type double.

Usage

Use the LOG10 function to return the base 10 logarithm of an expression.

RANDOM

Returns a pseudo random number.

Syntax

RANDOM( )

Terms

N/A No terms.

Usage

Use the RANDOM function to return a pseudo random number (type double) greater than or equal to 0.0 and less than 1.0.

ROUND

Returns the value of an expression rounded to an integer.

Syntax

ROUND(expression)

Terms

expression An expression whose result is type float or double.

Usage

Use the ROUND function to return the value of an expression rounded to an integer (if the result type is float) or rounded to a long (if the result type is double).
Values are rounded towards positive infinity: round(x) = floor(x + 0.5).
x ROUND(x)
4.6 5
3.5 4
2.4 2
1.0 1
-1.0 -1
-2.4 -2
-3.5 -3
-4.6 -5

ROUND_TO

Returns the value of an expression rounded to a fixed number of decimal digits.

Syntax

ROUND_TO(val, digits [, mode])

Terms

val An expression whose result is type float or double: the value to round.
digits An expression whose result is type int: the number of digits to preserve.
mode An optional int specifying the rounding mode, according to the constants Java provides.

Usage

Use the ROUND function to return the value of an expression rounded to a fixed number of digits. Given a float, its result is a float; given a double its result is a double.
The result is a multiple of the digits-th power of ten: 0 leads to no fractional digits; a negative value zeros out correspondingly many places to the left of the decimal point.
When mode is omitted or has the value 6 (RoundingMode.HALF_EVEN), the result is rounded towards the nearest neighbor, and ties are rounded to the nearest even digit. This mode minimizes cumulative error and tends to preserve the average of a set of values.
When mode has the value 4 (RoundingMode.HALF_UP), the result is rounded towards the nearest neighbor, and ties are rounded away from zero. This mode matches the behavior of most SQL systems.
For other rounding modes, consult Java's documentation. There is no rounding mode that matches Math.round's behavior (i.e. round towards positive infinity) -- blame Java, not Pig.
val digits mode ROUND_TO(val, digits)
1234.1789 8 1234.1789
1234.1789 4 1234.1789
1234.1789 1 1234.2
1234.1789 0 1234.0
1234.1789 -1 1230.0
1234.1789 -3 1000.0
1234.1789 -4 0.0
3.25000001 1 3.3
3.25 1 3.2
-3.25 1 -3.2
3.15 1 3.2
-3.15 1 -3.2
3.25 1 4 3.3
-3.25 1 4 -3.3
3.5 0 4.0
-3.5 0 -4.0
2.5 0 2.0
-2.5 0 -2.0
3.5 0 4 4.0
-3.5 0 4 -4.0
2.5 0 4 3.0
-2.5 0 4 -3.0

SIN

Returns the sine of an expression.

Syntax

SIN(expression)

Terms

expression An expression whose result is double.

Usage

Use the SIN function to return the sine of an expession.

SINH

Returns the hyperbolic sine of an expression.

Syntax

SINH(expression)

Terms

expression An expression whose result is double.

Usage

Use the SINH function to return the hyperbolic sine of an expression.

SQRT

Returns the positive square root of an expression.

Syntax

SQRT(expression)

Terms

expression An expression whose result is double.

Usage

Use the SQRT function to return the positive square root of an expression.

TAN

Returns the trignometric tangent of an angle.

Syntax

TAN(expression)

Terms

expression An expression (angle) whose result is double.

Usage

Use the TAN function to return the trignometric tangent of an angle.

TANH

Returns the hyperbolic tangent of an expression.

Syntax

TANH(expression)

Terms

expression An expression whose result is double.

Usage

Use the TANH function to return the hyperbolic tangent of an expression.

String Functions

For general information about these functions, see the Java API Specification, Class String. Note the following:
  • Pig function names are case sensitive and UPPER CASE.
  • Pig string functions have an extra, first parameter: the string to which all the operations are applied.
  • Pig may process results differently than as stated in the Java API Specification. If any of the input parameters are null or if an insufficient number of parameters are supplied, NULL is returned.

ENDSWITH

Tests inputs to determine if the first argument ends with the string in the second.

Syntax

ENDSWITH(string, testAgainst)

Terms

string The string to be tested.
testAgainst The string to test against.

Usage

Use the ENDSWITH function to determine if the first argument ends with the string in the second.
For example, ENDSWITH ('foobar', 'foo') will false, whereas ENDSWITH ('foobar', 'bar') will return true.

EqualsIgnoreCase

Compares two Strings ignoring case considerations.

Syntax

EqualsIgnoreCase(string1, string2)

Terms

string1 The source string.
string2 The string to compare against.

Usage

Use the EqualsIgnoreCase function to determine if two string are equal ignoring case.

INDEXOF

Returns the index of the first occurrence of a character in a string, searching forward from a start index.

Syntax

INDEXOF(string, 'character', startIndex)

Terms

string The string to be searched.
'character' The character being searched for, in quotes.
startIndex The index from which to begin the forward search.
The string index begins with zero (0).

Usage

Use the INDEXOF function to determine the index of the first occurrence of a character in a string. The forward search for the character begins at the designated start index.

LAST_INDEX_OF

Returns the index of the last occurrence of a character in a string, searching backward from the end of the string.

Syntax

LAST_INDEX_OF(string, 'character')

Terms

string The string to be searched.
'character' The character being searched for, in quotes.

Usage

Use the LAST_INDEX_OF function to determine the index of the last occurrence of a character in a string. The backward search for the character begins at the end of the string.

LCFIRST

Converts the first character in a string to lower case.

Syntax

LCFIRST(expression)

Terms

expression An expression whose result type is chararray.

Usage

Use the LCFIRST function to convert only the first character in a string to lower case.

LOWER

Converts all characters in a string to lower case.

Syntax

LOWER(expression)

Terms

expression An expression whose result type is chararray.

Usage

Use the LOWER function to convert all characters in a string to lower case.

LTRIM

Returns a copy of a string with only leading white space removed.

Syntax

LTRIM(expression)

Terms

expression An expression whose result is chararray.

Usage

Use the LTRIM function to remove leading white space from a string.

REGEX_EXTRACT

Performs regular expression matching and extracts the matched group defined by an index parameter.

Syntax

REGEX_EXTRACT (string, regex, index)

Terms

string The string in which to perform the match.
regex The regular expression.
index The index of the matched group to return.

Usage

Use the REGEX_EXTRACT function to perform regular expression matching and to extract the matched group defined by the index parameter (where the index is a 1-based parameter.) The function uses Java regular expression form.
The function returns a string that corresponds to the matched group in the position specified by the index. If there is no matched expression at that position, NULL is returned.

Example

This example will return the string '192.168.1.5'.
REGEX_EXTRACT('192.168.1.5:8020', '(.*):(.*)', 1);

REGEX_EXTRACT_ALL

Performs regular expression matching and extracts all matched groups.

Syntax

REGEX_EXTRACT_ALL (string, regex)

Terms

string The string in which to perform the match.
regex The regular expression.

Usage

Use the REGEX_EXTRACT_ALL function to perform regular expression matching and to extract all matched groups. The function uses Java regular expression form.
The function returns a tuple where each field represents a matched expression. If there is no match, an empty tuple is returned.

Example

This example will return the tuple (192.168.1.5,8020).
REGEX_EXTRACT_ALL('192.168.1.5:8020', '(.*)\:(.*)');

REGEX_SEARCH

Performs regular expression matching and searches all matched characters in a string.

Syntax

REGEX_SEARCH(string, 'regExp');

Terms

string The string in which to perform the match.
'regExp' The regular expression to which the string is to be matched, in quotes.

Usage

Use the REGEX_SEARCH function to perform regular expression matching and to find all matched characters in a string.
The function returns tuples which are placed in a bag. Each tuple only contains one field which represents a matched expression.

Example

This is example will return the bag {(=04 ),(=06 ),(=96 )}.
REGEX_SEARCH('a=04 b=06 c=96 or more', '(=\\d+\\s)');
And this is example will return the bag {(04),(06),(96)}.
REGEX_SEARCH('a=04 b=06 c=96 or more', '=(\\d+)\\s');

REPLACE

Replaces existing characters in a string with new characters.

Syntax

REPLACE(string, 'regExp', 'newChar');

Terms

string The string to be updated.
'regExp' The regular expression to which the string is to be matched, in quotes.
'newChar' The new characters replacing the existing characters, in quotes.

Usage

Use the REPLACE function to replace existing characters in a string with new characters.
For example, to change "open source software" to "open source wiki" use this statement: REPLACE(string,'software','wiki')
Note that the REPLACE function is internally implemented using java.string.replaceAll(String regex, String replacement) where 'regExp' and 'newChar' are passed as the 1st and 2nd argument respectively. If you want to replace special characters such as '[' in the string literal, it is necessary to escape them in 'regExp' by prefixing them with double backslashes (e.g. '\\[').

RTRIM

Returns a copy of a string with only trailing white space removed.

Syntax

RTRIM(expression)

Terms

expression An expression whose result is chararray.

Usage

Use the RTRIM function to remove trailing white space from a string.

SPRINTF

Formats a set of values according to a printf-style template, using the native Java Formatter library.

Syntax

SPRINTF(format, [...vals])

Terms

format The printf-style string describing the template.
vals The values to place in the template. There must be a tuple element for each formatting placeholder, and it must have the correct type: int or long for integer formats such as %d; float or double for decimal formats such as %f; and long for date/time formats such as %t.

Usage

Use the SPRINTF function to format a string according to a template. For example, SPRINTF("part-%05d", 69) will return 'part-00069'.
String format specification arg1 arg2 arg3 SPRINTF(format, arg1, arg2) notes
'%8s|%8d|%-8s' 1234567 1234567 'yay' ' 1234567| 1234567|yay ' Format strings with %s, integers with %d. Types are converted for you where reasonable (here, int -> string).
(null value) 1234567 1234567 'yay' (null value) Returns null (no error or warning) with a null format string.
'%8s|%8d|%-8s' 1234567 (null value) 'yay' (null value) Returns null (no error or warning) if any single argument is null.
'%8.3f|%6x' 123.14159 665568 ' 123.142| a27e0' Format floats/doubles with %f, hexadecimal integers with %x (there are others besides -- see the Java docs)
'%,+10d|%(06d' 1234567 -123 '+1,234,567|(0123)' Numerics take a prefix modifier: , for locale-specific thousands-delimiting, 0 for zero-padding; + to always show a plus sign for positive numbers; space to allow a space preceding positive numbers; ( to indicate negative numbers with parentheses (accountant-style).
'%2$5d: %3$6s %1$3s %2$4x (%<4X)' 'the' 48879 'wheres' '48879: wheres the beef (BEEF)' Refer to args positionally and as many times as you like using %(pos)$.... Use %<... to refer to the previously-specified arg.
'Launch Time: %14d %s' ToMilliSeconds(CurrentTime()) ToString(CurrentTime(), 'yyyy-MM-dd HH:mm:ss Z') 'Launch Time: 1400164132000 2014-05-15 09:28:52 -0500' Instead use ToString to format the date/time portions and SPRINTF to layout the results.
'%8s|%-8s' 1234567 MissingFormatArgumentException: Format specifier '%-8s' You must supply arguments for all specifiers
'%8s' 1234567 'ignored' 'also' 1234567 It's OK to supply too many, though
Note: although the Java formatter (and thus this function) offers the %t specifier for date/time elements, it's best avoided: it's cumbersome, the output and timezone handling may differ from what you expect, and it doesn't accept datetime objects from pig. Instead, just prepare dates usint the ToString UDF as shown.

STARTSWITH

Tests inputs to determine if the first argument starts with the string in the second.

Syntax

STARTSWITH(string, testAgainst)

Terms

string The string to be tested.
testAgainst The string to test against.

Usage

Use the STARTSWITH function to determine if the first argument starts with the string in the second.
For example, STARTSWITH ('foobar', 'foo') will true, whereas STARTSWITH ('foobar', 'bar') will return false.

STRSPLIT

Splits a string around matches of a given regular expression.

Syntax

STRSPLIT(string, regex, limit)

Terms

string The string to be split.
regex The regular expression.
limit If the value is positive, the pattern (the compiled representation of the regular expression) is applied at most limit-1 times, therefore the value of the argument means the maximum length of the result tuple. The last element of the result tuple will contain all input after the last match.
If the value is negative, no limit is applied for the length of the result tuple.
If the value is zero, no limit is applied for the length of the result tuple too, and trailing empty strings (if any) will be removed.

Usage

Use the STRSPLIT function to split a string around matches of a given regular expression.
For example, given the string (open:source:software), STRSPLIT (string, ':',2) will return ((open,source:software)) and STRSPLIT (string, ':',3) will return ((open,source,software)).

STRSPLITTOBAG

Splits a string around matches of a given regular expression and returns a databag

Syntax

STRSPLITTOBAG(string, regex, limit)

Terms

string The string to be split.
regex The regular expression.
limit If the value is positive, the pattern (the compiled representation of the regular expression) is applied at most limit-1 times, therefore the value of the argument means the maximum size of the result bag. The last tuple of the result bag will contain all input after the last match.
If the value is negative, no limit is applied to the size of the result bag.
If the value is zero, no limit is applied to the size of the result bag too, and trailing empty strings (if any) will be removed.

Usage

Use the STRSPLITTOBAG function to split a string around matches of a given regular expression.
For example, given the string (open:source:software), STRSPLITTOBAG (string, ':',2) will return {(open),(source:software)} and STRSPLITTOBAG (string, ':',3) will return {(open),(source),(software)}.

SUBSTRING

Returns a substring from a given string.

Syntax

SUBSTRING(string, startIndex, stopIndex)

Terms

string The string from which a substring will be extracted.
startIndex The index (type integer) of the first character of the substring.
The index of a string begins with zero (0).
stopIndex The index (type integer) of the character following the last character of the substring.

Usage

Use the SUBSTRING function to return a substring from a given string.
Given a field named alpha whose value is ABCDEF, to return substring BCD use this statement: SUBSTRING(alpha,1,4). Note that 1 is the index of B (the first character of the substring) and 4 is the index of E (the character following the last character of the substring).

TRIM

Returns a copy of a string with leading and trailing white space removed.

Syntax

TRIM(expression)

Terms

expression An expression whose result is chararray.

Usage

Use the TRIM function to remove leading and trailing white space from a string.

UCFIRST

Returns a string with the first character converted to upper case.

Syntax

UCFIRST(expression)

Terms

expression An expression whose result type is chararray.

Usage

Use the UCFIRST function to convert only the first character in a string to upper case.

UPPER

Returns a string converted to upper case.

Syntax

UPPER(expression)

Terms

expression An expression whose result type is chararray.

Usage

Use the UPPER function to convert all characters in a string to upper case.

UniqueID

Returns a unique id string for each record in the alias.

Usage

UniqueID generates a unique id for each records. The id takes form "taskindex-sequence"

Datetime Functions

For general information about datetime type operations, see the Java API Specification, Java Date class, and JODA DateTime class. And for the information of ISO date and time formats, please refer to Date and Time Formats.

AddDuration

Returns the result of a DateTime object plus a Duration object.

Syntax

AddDuration(datetime, duration)

Terms

datetime A datetime object.
duration The duration string in ISO 8601 format.

Usage

Use the AddDuration function to created a new datetime object by add some duration to a given datetime object.

CurrentTime

Returns the DateTime object of the current time.

Syntax

CurrentTime()

Usage

Use the CurrentTime function to generate a datetime object of current timestamp with millisecond accuracy.

DaysBetween

Returns the number of days between two DateTime objects.

Syntax

DaysBetween(datetime1, datetime2)

Terms

datetime1 A datetime object.
datetime2 Another datetime object.

Usage

Use the DaysBetween function to get the number of days between the two given datetime objects.

GetDay

Returns the day of a month from a DateTime object.

Syntax

GetDay(datetime)

Terms

datetime A datetime object.

Usage

Use the GetDay function to extract the day of a month from the given datetime object.

GetHour

Returns the hour of a day from a DateTime object.

Syntax

GetHour(datetime)

Terms

datetime A datetime object.

Usage

Use the GetHour function to extract the hour of a day from the given datetime object.

GetMilliSecond

Returns the millisecond of a second from a DateTime object.

Syntax

GetMilliSecond(datetime)

Terms

datetime A datetime object.

Usage

Use the GetMilliSecond function to extract the millsecond of a second from the given datetime object.

GetMinute

Returns the minute of a hour from a DateTime object.

Syntax

GetMinute(datetime)

Terms

datetime A datetime object.

Usage

Use the GetMinute function to extract the minute of a hour from the given datetime object.

GetMonth

Returns the month of a year from a DateTime object.

Syntax

GetMonth(datetime)

Terms

datetime A datetime object.

Usage

Use the GetMonth function to extract the month of a year from the given datetime object.

GetSecond

Returns the second of a minute from a DateTime object.

Syntax

GetSecond(datetime)

Terms

datetime A datetime object.

Usage

Use the GetSecond function to extract the second of a minute from the given datetime object.

GetWeek

Returns the week of a week year from a DateTime object.

Syntax

GetWeek(datetime)

Terms

datetime A datetime object.

Usage

Use the GetWeek function to extract the week of a week year from the given datetime object. Note that week year may be different from year.

GetWeekYear

Returns the week year from a DateTime object.

Syntax

GetWeekYear(datetime)

Terms

datetime A datetime object.

Usage

Use the GetWeekYear function to extract the week year from the given datetime object. Note that week year may be different from year.

GetYear

Returns the year from a DateTime object.

Syntax

GetYear(datetime)

Terms

datetime A datetime object.

Usage

Use the GetYear function to extract the year from the given datetime object.

HoursBetween

Returns the number of hours between two DateTime objects.

Syntax

HoursBetween(datetime1, datetime2)

Terms

datetime1 A datetime object.
datetime2 Another datetime object.

Usage

Use the HoursBetween function to get the number of hours between the two given datetime objects.

MilliSecondsBetween

Returns the number of milliseconds between two DateTime objects.

Syntax

MilliSecondsBetween(datetime1, datetime2)

Terms

datetime1 A datetime object.
datetime2 Another datetime object.

Usage

Use the MilliSecondsBetween function to get the number of millseconds between the two given datetime objects.

MinutesBetween

Returns the number of minutes between two DateTime objects.

Syntax

MinutesBetween(datetime1, datetime2)

Terms

datetime1 A datetime object.
datetime2 Another datetime object.

Usage

Use the MinutsBetween function to get the number of minutes between the two given datetime objects.

MonthsBetween

Returns the number of months between two DateTime objects.

Syntax

MonthsBetween(datetime1, datetime2)

Terms

datetime1 A datetime object.
datetime2 Another datetime object.

Usage

Use the MonthsBetween function to get the number of months between the two given datetime objects.

SecondsBetween

Returns the number of seconds between two DateTime objects.

Syntax

SecondsBetween(datetime1, datetime2)

Terms

datetime1 A datetime object.
datetime2 Another datetime object.

Usage

Use the SecondsBetween function to get the number of seconds between the two given datetime objects.

SubtractDuration

Returns the result of a DateTime object minus a Duration object.

Syntax

SubtractDuration(datetime, duration)

Terms

datetime A datetime object.
duration The duration string in ISO 8601 format.

Usage

Use the AddDuration function to created a new datetime object by add some duration to a given datetime object.

ToDate

Returns a DateTime object according to parameters.

Syntax

ToDate(milliseconds)
ToDate(iosstring)
ToDate(userstring, format)
ToDate(userstring, format, timezone)

Terms

millseconds The offset from 1970-01-01T00:00:00.000Z in terms of the number milliseconds (either positive or negative).
isostring The datetime string in the ISO 8601 format.
userstring The datetime string in the user defined format.
format The date time format pattern string (see Java SimpleDateFormat class).
timezone The timezone string. Either the UTC offset and the location based format can be used as a parameter, while internally the timezone will be converted to the UTC offset format.
Please see the Joda-Time doc for available timezone IDs.

Usage

Use the ToDate function to generate a DateTime object. Note that if the timezone is not specified with the ISO datetime string or by the timezone parameter, the default timezone will be used.

ToMilliSeconds

Returns the number of milliseconds elapsed since January 1, 1970, 00:00:00.000 GMT for a DateTime object.

Syntax

ToMilliSeconds(datetime)

Terms

datetime A datetime object.

Usage

Use the ToMilliSeconds function to convert the DateTime to the number of milliseconds that have passed since January 1, 1970 00:00:00.000 GMT.

ToString

ToString converts the DateTime object to the ISO or the customized string.

Syntax

ToString(datetime [, format string])

Terms

datetime A datetime object.
format string The date time format pattern string (see Java SimpleDateFormat class).

Usage

Use the ToString function to convert the DateTime to the customized string.

ToUnixTime

Returns the Unix Time as long for a DateTime object. UnixTime is the number of seconds elapsed since January 1, 1970, 00:00:00.000 GMT.

Syntax

ToUnixTime(datetime)

Terms

datetime A datetime object.

Usage

Use the ToUnixTime function to convert the DateTime to Unix Time.

WeeksBetween

Returns the number of weeks between two DateTime objects.

Syntax

WeeksBetween(datetime1, datetime2)

Terms

datetime1 A datetime object.
datetime2 Another datetime object.

Usage

Use the WeeksBetween function to get the number of weeks between the two given datetime objects.

YearsBetween

Returns the number of years between two DateTime objects.

Syntax

YearsBetween(datetime1, datetime2)

Terms

datetime1 A datetime object.
datetime2 Another datetime object.

Usage

Use the YearsBetween function to get the number of years between the two given datetime objects.

Tuple, Bag, Map Functions


TOTUPLE

Converts one or more expressions to type tuple.

Syntax

TOTUPLE(expression [, expression ...])

Terms

expression An expression of any datatype.

Usage

Use the TOTUPLE function to convert one or more expressions to a tuple.
See also: Tuple data type and Type Construction Operators

Example

In this example, fields f1, f2 and f3 are converted to a tuple.
a = LOAD 'student' AS (f1:chararray, f2:int, f3:float);
DUMP a;

(John,18,4.0)
(Mary,19,3.8)
(Bill,20,3.9)
(Joe,18,3.8)

b = FOREACH a GENERATE TOTUPLE(f1,f2,f3);
DUMP b;

((John,18,4.0))
((Mary,19,3.8))
((Bill,20,3.9))
((Joe,18,3.8))

TOBAG

Converts one or more expressions to type bag.

Syntax

TOBAG(expression [, expression ...])

Terms

expression An expression with any data type.

Usage

Use the TOBAG function to convert one or more expressions to individual tuples which are then placed in a bag.
See also: Bag data type and Type Construction Operators

Example

In this example, fields f1 and f3 are converted to tuples that are then placed in a bag.
a = LOAD 'student' AS (f1:chararray, f2:int, f3:float);
DUMP a;

(John,18,4.0)
(Mary,19,3.8)
(Bill,20,3.9)
(Joe,18,3.8)

b = FOREACH a GENERATE TOBAG(f1,f3);
DUMP b;

({(John),(4.0)})
({(Mary),(3.8)})
({(Bill),(3.9)})
({(Joe),(3.8)})

TOMAP

Converts key/value expression pairs into a map.

Syntax

TOMAP(key-expression, value-expression [, key-expression, value-expression ...])

Terms

key-expression An expression of type chararray.
value-expression An expression of any type supported by a map.

Usage

Use the TOMAP function to convert pairs of expressions into a map. Note the following:
  • You must supply an even number of expressions as parameters
  • The elements must comply with map type rules:
    • Every odd element (key-expression) must be a chararray since only chararrays can be keys into the map
    • Every even element (value-expression) can be of any type supported by a map.
See also: Map data type and Type Construction Operators

Example

In this example, student names (type chararray) and student GPAs (type float) are used to create three maps.
A = load 'students' as (name:chararray, age:int, gpa:float);
B = foreach A generate TOMAP(name, gpa);
store B into 'results';

Input (students)
joe smith 20 3.5
amy chen 22 3.2
leo allen 18 2.1

Output (results)
[joe smith#3.5]
[amy chen#3.2]
[leo allen#2.1]

TOP

Returns the top-n tuples from a bag of tuples.

Syntax

TOP(topN,column,relation)

Terms

topN The number of top tuples to return (type integer).
column The tuple column whose values are being compared, note 0 denotes the first column.
relation The relation (bag of tuples) containing the tuple column.

Usage

TOP function returns a bag containing top N tuples from the input bag where N is controlled by the first parameter to the function. The tuple comparison is performed based on a single column from the tuple. The column position is determined by the second parameter to the function. The function assumes that all tuples in the bag contain an element of the same type in the compared column.
By default, TOP function uses descending order. But it can be configured via DEFINE statement.
DEFINE asc TOP('ASC'); -- ascending order
DEFINE desc TOP('DESC'); -- descending order

Example

In this example the top 10 occurrences are returned.
DEFINE asc TOP('ASC'); -- ascending order
DEFINE desc TOP('DESC'); -- descending order

A = LOAD 'data' as (first: chararray, second: chararray);
B = GROUP A BY (first, second);
C = FOREACH B generate FLATTEN(group), COUNT(A) as count;
D = GROUP C BY first; -- again group by first
topResults = FOREACH D {
    result = asc(10, 1, C); -- and retain top 10 (in ascending order) occurrences of 'second' in first  
    GENERATE FLATTEN(result);
}

bottomResults = FOREACH D {
    result = desc(10, 1, C); -- and retain top 10  (in descending order) occurrences of 'second' in first  
    GENERATE FLATTEN(result);
}

Hive UDF

Pig invokes all types of Hive UDF, including UDF, GenericUDF, UDAF, GenericUDAF and GenericUDTF. Depending on the Hive UDF you want to use, you need to declare it in Pig with HiveUDF(handles UDF and GenericUDF), HiveUDAF(handles UDAF and GenericUDAF), HiveUDTF(handles GenericUDTF).

Syntax

HiveUDF, HiveUDAF, HiveUDTF share the same syntax.
HiveUDF(name[, constant parameters])

Terms

name Hive UDF name. This can be a fully qualified class name of the Hive UDF/UDTF/UDAF class, or a registered short name in Hive FunctionRegistry (most Hive builtin UDF does that)
constant parameters Optional tuple representing constant parameters of a Hive UDF/UDTF/UDAF. If Hive UDF requires a constant parameter, there is no other way Pig can pass that information to Hive, since Pig schema does not carry the information whether a parameter is constant or not. Null item in the tuple means this field is not a constant. Non-null item represents a constant field. Data type for the item is determined by Pig contant parser.

Example

HiveUDF
define sin HiveUDF('sin');
A = LOAD 'student' as (name:chararray, age:int, gpa:double);
B = foreach A generate sin(gpa);
  
HiveUDTF
define explode HiveUDTF('explode');
A = load 'mydata' as (a0:{(b0:chararray)});
B = foreach A generate flatten(explode(a0));
  
HiveUDAF
define avg HiveUDAF('avg');
A = LOAD 'student' as (name:chararray, age:int, gpa:double);
B = group A by name;
C = foreach B generate group, avg(A.age);
  
HiveUDAF with constant parameter
define in_file HiveUDF('in_file', '(null, "names.txt")');
A = load 'student' as (name:chararray, age:long, gpa:double);
B = foreach A generate in_file(name, 'names.txt');
In this example, we pass (null, "names.txt") to the construct of UDF in_file, meaning the first parameter is regular, the second parameter is a constant. names.txt can be double quoted (unlike other Pig syntax), or quoted in \'. Note we need to pass 'names.txt' again in line 3. This looks stupid but we need to do this to fill the semantic gap between Pig and Hive. We need to pass the constant in the data pipeline in line 3, which is similar Pig UDF. Initialization code in Hive UDF takes ObjectInspector, which capture the data type and whether or not the parameter is a constant. However, initialization code in Pig takes schema, which only capture the former. We need to use additional mechanism (construct parameter) to convey the later.
Note: A few Hive 0.14 UDF contains bug which affects Pig and are fixed in Hive 1.0. Here is a list: compute_stats, context_ngrams, count, ewah_bitmap, histogram_numeric, collect_list, collect_set, ngrams, case, in, named_struct, stack, percentile_approx.

Introduction

Pig provides extensive support for user defined functions (UDFs) as a way to specify custom processing. Pig UDFs can currently be implemented in six languages: Java, Jython, Python, JavaScript, Ruby and Groovy.
The most extensive support is provided for Java functions. You can customize all parts of the processing including data load/store, column transformation, and aggregation. Java functions are also more efficient because they are implemented in the same language as Pig and because additional interfaces are supported such as the Algebraic Interface and the Accumulator Interface.
Limited support is provided for Jython, Python, JavaScript, Ruby and Groovy functions. These functions are new, still evolving, additions to the system. Currently only the basic interface is supported; load/store functions are not supported. Furthermore, JavaScript, Ruby and Groovy are provided as experimental features because they did not go through the same amount of testing as Java or Jython. At runtime note that Pig will automatically detect the usage of a scripting UDF in the Pig script and will automatically ship the corresponding scripting jar, either Jython, Rhino, JRuby or Groovy-all, to the backend. Python does not require any runtime engine since it invoke python command line and stream data in and out of it.
Pig also provides support for Piggy Bank, a repository for JAVA UDFs. Through Piggy Bank you can access Java UDFs written by other users and contribute Java UDFs that you have written.

Writing Java UDFs


Eval Functions


How to Use a Simple Eval Function

Eval is the most common type of function. It can be used in FOREACH statements as shown in this script:
-- myscript.pig
REGISTER myudfs.jar;
A = LOAD 'student_data' AS (name: chararray, age: int, gpa: float);
B = FOREACH A GENERATE myudfs.UPPER(name);
DUMP B;
The command below can be used to run the script. Note that all examples in this document run in local mode for simplicity but the examples can also run in Tez local/Mapreduce/ Tez mode. For more information on how to run Pig, please see the PigTutorial.
pig -x local myscript.pig
The first line of the script provides the location of the jar file that contains the UDF. (Note that there are no quotes around the jar file. Having quotes would result in a syntax error.) To locate the jar file, Pig first checks the classpath. If the jar file can't be found in the classpath, Pig assumes that the location is either an absolute path or a path relative to the location from which Pig was invoked. If the jar file can't be found, an error will be printed: java.io.IOException: Can't read jar file: myudfs.jar.
Multiple register commands can be used in the same script. If the same fully-qualified function is present in multiple jars, the first occurrence will be used consistently with Java semantics.
The name of the UDF has to be fully qualified with the package name or an error will be reported: java.io.IOException: Cannot instantiate:UPPER. Also, the function name is case sensitive (UPPER and upper are not the same). A UDF can take one or more parameters. The exact signature of the function should be clear from its documentation.
The function provided in this example takes an ASCII string and produces its uppercase version. If you are familiar with column transformation functions in SQL, you will recognize that UPPER fits this concept. However, as we will see later in the document, eval functions in Pig go beyond column transformation functions and include aggregate and filter functions.
If you are just a user of UDFs, this is most of what you need to know about UDFs to use them in your code.

How to Write a Simple Eval Function

Let's now look at the implementation of the UPPER UDF.
1  package myudfs;
2  import java.io.IOException;
3  import org.apache.pig.EvalFunc;
4  import org.apache.pig.data.Tuple;
5
6  public class UPPER extends EvalFunc<String>
7  {
8    public String exec(Tuple input) throws IOException {
9        if (input == null || input.size() == 0 || input.get(0) == null)
10            return null;
11        try{
12            String str = (String)input.get(0);
13           return str.toUpperCase();
14        }catch(Exception e){
15            throw new IOException("Caught exception processing input row ", e);
16        }
17    }
18  }
Line 1 indicates that the function is part of the myudfs package. The UDF class extends the EvalFunc class which is the base class for all eval functions. It is parameterized with the return type of the UDF which is a Java String in this case. We will look into the EvalFunc class in more detail later, but for now all we need to do is to implement the exec function. This function is invoked on every input tuple. The input into the function is a tuple with input parameters in the order they are passed to the function in the Pig script. In our example, it will contain a single string field corresponding to the student name.
The first thing to decide is what to do with invalid data. This depends on the format of the data. If the data is of type bytearray it means that it has not yet been converted to its proper type. In this case, if the format of the data does not match the expected type, a NULL value should be returned. If, on the other hand, the input data is of another type, this means that the conversion has already happened and the data should be in the correct format. This is the case with our example and that's why it throws an error (line 15.)
Also, note that lines 9-10 check if the input data is null or empty and if so returns null.
The actual function implementation is on lines 12-13 and is self-explanatory.
Now that we have the function implemented, it needs to be compiled and included in a jar. You will need to build pig.jar to compile your UDF. You can use the following set of commands to checkout the code from SVN repository and create pig.jar:
svn co http://svn.apache.org/repos/asf/pig/trunk
cd trunk
ant
You should see pig.jar in your current working directory. The set of commands below first compiles the function and then creates a jar file that contains it.
cd myudfs
javac -cp pig.jar UPPER.java
cd ..
jar -cf myudfs.jar myudfs
You should now see myudfs.jar in your current working directory. You can use this jar with the script described in the previous section.

Aggregate Functions

Aggregate functions are another common type of eval function. Aggregate functions are usually applied to grouped data, as shown in this script:
-- myscript2.pig
A = LOAD 'student_data' AS (name: chararray, age: int, gpa: float);
B = GROUP A BY name;
C = FOREACH B GENERATE group, COUNT(A);
DUMP C;
The script above uses the COUNT function to count the number of students with the same name. There are a couple of things to note about this script. First, even though we are using a function, there is no register command. Second, the function is not qualified with the package name. The reason for both is that COUNT is a builtin function meaning that it comes with the Pig distribution. These are the only two differences between builtins and UDFs. Builtins are discussed in more detail later in this document.

Algebraic Interface

An aggregate function is an eval function that takes a bag and returns a scalar value. One interesting and useful property of many aggregate functions is that they can be computed incrementally in a distributed fashion. We call these functions algebraic. COUNT is an example of an algebraic function because we can count the number of elements in a subset of the data and then sum the counts to produce a final output. In the Hadoop world, this means that the partial computations can be done by the map and combiner, and the final result can be computed by the reducer.
It is very important for performance to make sure that aggregate functions that are algebraic are implemented as such. Let's look at the implementation of the COUNT function to see what this means. (Error handling and some other code is omitted to save space. The full code can be accessed here.)
public class COUNT extends EvalFunc<Long> implements Algebraic{
    public Long exec(Tuple input) throws IOException {return count(input);}
    public String getInitial() {return Initial.class.getName();}
    public String getIntermed() {return Intermed.class.getName();}
    public String getFinal() {return Final.class.getName();}
    static public class Initial extends EvalFunc<Tuple> {
        public Tuple exec(Tuple input) throws IOException {return TupleFactory.getInstance().newTuple(count(input));}
    }
    static public class Intermed extends EvalFunc<Tuple> {
        public Tuple exec(Tuple input) throws IOException {return TupleFactory.getInstance().newTuple(sum(input));}
    }
    static public class Final extends EvalFunc<Long> {
        public Tuple exec(Tuple input) throws IOException {return sum(input);}
    }
    static protected Long count(Tuple input) throws ExecException {
        Object values = input.get(0);
        if (values instanceof DataBag) return ((DataBag)values).size();
        else if (values instanceof Map) return new Long(((Map)values).size());
    }
    static protected Long sum(Tuple input) throws ExecException, NumberFormatException {
        DataBag values = (DataBag)input.get(0);
        long sum = 0;
        for (Iterator (Tuple) it = values.iterator(); it.hasNext();) {
            Tuple t = it.next();
            sum += (Long)t.get(0);
        }
        return sum;
    }
}
COUNT implements Algebraic interface which looks like this:
public interface Algebraic{
    public String getInitial();
    public String getIntermed();
    public String getFinal();
}
For a function to be algebraic, it needs to implement Algebraic interface that consist of definition of three classes derived from EvalFunc. The contract is that the exec function of the Initial class is called once and is passed the original input tuple. Its output is a tuple that contains partial results. The exec function of the Intermed class can be called zero or more times and takes as its input a tuple that contains partial results produced by the Initial class or by prior invocations of the Intermed class and produces a tuple with another partial result. Finally, the exec function of the Final class is called and produces the final result as a scalar type.
Here's the way to think about this in the Hadoop world. The exec function of the Initial class is invoked once for each input tuple by the map process and produces partial results. The exec function of the Intermed class is invoked once by each combiner invocation (which can happen zero or more times) and also produces partial results. The exec function of the Final class is invoked once by the reducer and produces the final result.
Take a look at the COUNT implementation to see how this is done. Note that the exec function of the Initial and Intermed classes is parameterized with Tuple and the exec of the Final class is parameterized with the real type of the function, which in the case of the COUNT is Long. Also, note that the fully-qualified name of the class needs to be returned from getInitial, getIntermed, and getFinal methods.

Accumulator Interface

In Pig, problems with memory usage can occur when data, which results from a group or cogroup operation, needs to be placed in a bag and passed in its entirety to a UDF.
This problem is partially addressed by Algebraic UDFs that use the combiner and can deal with data being passed to them incrementally during different processing phases (map, combiner, and reduce). However, there are a number of UDFs that are not Algebraic, don't use the combiner, but still don’t need to be given all data at once.
The new Accumulator interface is designed to decrease memory usage by targeting such UDFs. For the functions that implement this interface, Pig guarantees that the data for the same key is passed continuously but in small increments. To work with incremental data, here is the interface a UDF needs to implement:
public interface Accumulator <T> {
   /**
    * Process tuples. Each DataBag may contain 0 to many tuples for current key
    */
    public void accumulate(Tuple b) throws IOException;
    /**
     * Called when all tuples from current key have been passed to the accumulator.
     * @return the value for the UDF for this key.
     */
    public T getValue();
    /**
     * Called after getValue() to prepare processing for next key. 
     */
    public void cleanup();
}
There are several things to note here:
  1. Each UDF must extend the EvalFunc class and implement all necessary functions there.
  2. If a function is algebraic but can be used in a FOREACH statement with accumulator functions, it needs to implement the Accumulator interface in addition to the Algebraic interface.
  3. The interface is parameterized with the return type of the function.
  4. The accumulate function is guaranteed to be called one or more times, passing one or more tuples in a bag, to the UDF. (Note that the tuple that is passed to the accumulator has the same content as the one passed to exec – all the parameters passed to the UDF – one of which should be a bag.)
  5. The getValue function is called after all the tuples for a particular key have been processed to retrieve the final value.
  6. The cleanup function is called after getValue but before the next value is processed.
Here us a code snippet of the integer version of the MAX function that implements the interface:
public class IntMax extends EvalFunc<Integer> implements Algebraic, Accumulator<Integer> {
    …….
    /* Accumulator interface */
    
    private Integer intermediateMax = null;
    
    @Override
    public void accumulate(Tuple b) throws IOException {
        try {
            Integer curMax = max(b);
            if (curMax == null) {
                return;
            }
            /* if bag is not null, initialize intermediateMax to negative infinity */
            if (intermediateMax == null) {
                intermediateMax = Integer.MIN_VALUE;
            }
            intermediateMax = java.lang.Math.max(intermediateMax, curMax);
        } catch (ExecException ee) {
            throw ee;
        } catch (Exception e) {
            int errCode = 2106;
            String msg = "Error while computing max in " + this.getClass().getSimpleName();
            throw new ExecException(msg, errCode, PigException.BUG, e);           
        }
    }

    @Override
    public void cleanup() {
        intermediateMax = null;
    }

    @Override
    public Integer getValue() {
        return intermediateMax;
    }
}

Filter Functions

Filter functions are eval functions that return a boolean value. Filter functions can be used anywhere a Boolean expression is appropriate, including the FILTER operator or bincond expression.
The example below uses the IsEmpy builtin filter function to implement joins.
-- inner join
A = LOAD 'student_data' AS (name: chararray, age: int, gpa: float);
B = LOAD 'voter_data' AS (name: chararray, age: int, registration: chararay, contributions: float);
C = COGROUP A BY name, B BY name;
D = FILTER C BY not IsEmpty(A);
E = FILTER D BY not IsEmpty(B);
F = FOREACH E GENERATE flatten(A), flatten(B);
DUMP F;
Note that, even if filtering is omitted, the same results will be produced because the foreach results is a cross product and cross products get rid of empty bags. However, doing up-front filtering is more efficient since it reduces the input of the cross product.
-- full outer join
A = LOAD 'student_data' AS (name: chararray, age: int, gpa: float);
B = LOAD 'voter_data' AS (name: chararray, age: int, registration: chararay, contributions: float);
C = COGROUP A BY name, B BY name;
D = FOREACH C GENERATE group, flatten((IsEmpty(A) ? null : A)), flatten((IsEmpty(B) ? null : B));
dump D;
The implementation of the IsEmpty function looks like this:
import java.io.IOException;
import java.util.Map;

import org.apache.pig.FilterFunc;
import org.apache.pig.PigException;
import org.apache.pig.backend.executionengine.ExecException;
import org.apache.pig.data.DataBag;
import org.apache.pig.data.Tuple;
import org.apache.pig.data.DataType;

/**
 * Determine whether a bag or map is empty.
 */
public class IsEmpty extends FilterFunc {

    @Override
    public Boolean exec(Tuple input) throws IOException {
        try {
            Object values = input.get(0);
            if (values instanceof DataBag)
                return ((DataBag)values).size() == 0;
            else if (values instanceof Map)
                return ((Map)values).size() == 0;
            else {
                int errCode = 2102;
                String msg = "Cannot test a " +
                DataType.findTypeName(values) + " for emptiness.";
                throw new ExecException(msg, errCode, PigException.BUG);
            }
        } catch (ExecException ee) {
            throw ee;
        }
    }
} 

Implement UDF by Simulation

When implementing more advanced types of EvalFuncs, the simpler implementations can be automatically provided by Pig. Thus if your UDF implements Algebraic then you will get the Accumulator interface and basic the basic EvalFunc exec method for free. Similarly, if your UDF implements Accumulator Interface you will get the basic EvalFunc exec method for free. You will not get the Algebraic implemenation. Note that these free implementations are based on simulation, which might not be the most efficient. If you wish to ensure the efficiency of your Accumulator of EvalFunc exec method, you may still implement them yourself and your implementations will be used.

Pig Types and Native Java Types

The main thing to know about Pig's type system is that Pig uses native Java types for almost all of its types, as shown in this table.
Pig Type Java Class
bytearray DataByteArray
chararray String
int Integer
long Long
float Float
double Double
boolean Boolean
datetime DateTime
bigdecimal BigDecimal
biginteger BigInteger
tuple Tuple
bag DataBag
map Map<Object, Object>
All Pig-specific classes are available here.
Tuple and DataBag are different in that they are not concrete classes but rather interfaces. This enables users to extend Pig with their own versions of tuples and bags. As a result, UDFs cannot directly instantiate bags or tuples; they need to go through factory classes: TupleFactory and BagFactory.
The builtin TOKENIZE function shows how bags and tuples are created. A function takes a text string as input and returns a bag of words from the text. (Note that currently Pig bags always contain tuples.)
package org.apache.pig.builtin;

import java.io.IOException;
import java.util.StringTokenizer;
import org.apache.pig.EvalFunc;
import org.apache.pig.data.BagFactory;
import org.apache.pig.data.DataBag;
import org.apache.pig.data.Tuple;
import org.apache.pig.data.TupleFactory;

public class TOKENIZE extends EvalFunc<DataBag> {
    TupleFactory mTupleFactory = TupleFactory.getInstance();
    BagFactory mBagFactory = BagFactory.getInstance();

    public DataBag exec(Tuple input) throws IOException 
        try {
            DataBag output = mBagFactory.newDefaultBag();
            Object o = input.get(0);
            if (!(o instanceof String)) {
                throw new IOException("Expected input to be chararray, but  got " + o.getClass().getName());
            }
            StringTokenizer tok = new StringTokenizer((String)o, " \",()*", false);
            while (tok.hasMoreTokens()) output.add(mTupleFactory.newTuple(tok.nextToken()));
            return output;
        } catch (ExecException ee) {
            // error handling goes here
        }
    }
}

Schemas and Java UDFs

Pig uses type information for validation and performance. It is important for UDFs to participate in type propagation. Our UDFs generally make no effort to communicate their output schema to Pig. This is because Pig can usually figure out this information by using Java's Reflection. If your UDF returns a scalar or a map, no work is required. However, if your UDF returns a tuple or a bag (of tuples), it needs to help Pig figure out the structure of the tuple.
If a UDF returns a tuple or a bag and schema information is not provided, Pig assumes that the tuple contains a single field of type bytearray. If this is not the case, then not specifying the schema can cause failures. We look at this next.
Let's assume that we have UDF Swap that, given a tuple with two fields, swaps their order. Let's assume that the UDF does not specify a schema and look at the scripts below:
register myudfs.jar;
A = load 'student_data' as (name: chararray, age: int, gpa: float);
B = foreach A generate flatten(myudfs.Swap(name, age)), gpa;
C = foreach B generate $2;
D = limit B 20;
dump D;
This script will result in the following error cause by line 4 ( C = foreach B generate $2;).
java.io.IOException: Out of bound access. Trying to access non-existent column: 2. Schema {bytearray,gpa: float} has 2 column(s).
This is because Pig is only aware of two columns in B while line 4 is requesting the third column of the tuple. (Column indexing in Pig starts with 0.)
The function, including the schema, looks like this:
package myudfs;
import java.io.IOException;
import org.apache.pig.EvalFunc;
import org.apache.pig.data.Tuple;
import org.apache.pig.data.TupleFactory;
import org.apache.pig.impl.logicalLayer.schema.Schema;
import org.apache.pig.data.DataType;

public class Swap extends EvalFunc<Tuple> {
    public Tuple exec(Tuple input) throws IOException {
        if (input == null || input.size()   2
            return null;
        try{
            Tuple output = TupleFactory.getInstance().newTuple(2);
            output.set(0, input.get(1));
            output.set(1, input.get(0));
            return output;
        } catch(Exception e){
            System.err.println("Failed to process input; error - " + e.getMessage());
            return null;
        }
    }
    public Schema outputSchema(Schema input) {
        try{
            Schema tupleSchema = new Schema();
            tupleSchema.add(input.getField(1));
            tupleSchema.add(input.getField(0));
            return new Schema(new Schema.FieldSchema(getSchemaName(this.getClass().getName().toLowerCase(), input),tupleSchema, DataType.TUPLE));
        }catch (Exception e){
                return null;
        }
    }
}
The function creates a schema with a single field (of type FieldSchema) of type tuple. The name of the field is constructed using the getSchemaName function of the EvalFunc class. The name consists of the name of the UDF function, the first parameter passed to it, and a sequence number to guarantee uniqueness. In the previous script, if you replace dump D; with describe B; , you will see the following output:
B: {myudfs.swap_age_3::age: int,myudfs.swap_age_3::name: chararray,gpa: float}
The second parameter to the FieldSchema constructor is the schema representing this field, which in this case is a tuple with two fields. The third parameter represents the type of the schema, which in this case is a TUPLE. All supported schema types are defined in the org.apache.pig.data.DataType class.
public class DataType {
    public static final byte UNKNOWN   =   0;
    public static final byte NULL      =   1;
    public static final byte BOOLEAN   =   5; // internal use only
    public static final byte BYTE      =   6; // internal use only
    public static final byte INTEGER   =  10;
    public static final byte LONG      =  15;
    public static final byte FLOAT     =  20;
    public static final byte DOUBLE    =  25;
    public static final byte DATETIME  =  30;
    public static final byte BYTEARRAY =  50;
    public static final byte CHARARRAY =  55;
    public static final byte BIGINTEGER =  65;
    public static final byte BIGDECIMAL =  70;
    public static final byte MAP       = 100;
    public static final byte TUPLE     = 110;
    public static final byte BAG       = 120;
    public static final byte ERROR     =  -1;
    // more code here
}
You need to import the org.apache.pig.data.DataType class into your code to define schemas. You also need to import the schema class org.apache.pig.impl.logicalLayer.schema.Schema.
The example above shows how to create an output schema for a tuple. Doing this for a bag is very similar. Let's extend the TOKENIZE function to do that.
As you can see in the example below, this is very similar to the output schema definition in the Swap function. One difference is that instead of reusing input schema, we create a brand new field schema to represent the tokens stored in the bag. The other difference is that the type of the schema created is BAG (not TUPLE).
package org.apache.pig.builtin;

import java.io.IOException;
import java.util.StringTokenizer;
import org.apache.pig.EvalFunc;
import org.apache.pig.data.BagFactory;
import org.apache.pig.data.DataBag;
import org.apache.pig.data.Tuple;
import org.apache.pig.data.TupleFactory;
import org.apache.pig.impl.logicalLayer.schema.Schema;
import org.apache.pig.data.DataType;

public class TOKENIZE extends EvalFunc<DataBag> {
    TupleFactory mTupleFactory = TupleFactory.getInstance();
    BagFactory mBagFactory = BagFactory.getInstance();
    public DataBag exec(Tuple input) throws IOException {
        try {
            DataBag output = mBagFactory.newDefaultBag();
            Object o = input.get(0);
            if ((o instanceof String)) {
                throw new IOException("Expected input to be chararray, but  got " + o.getClass().getName());
            }
            StringTokenizer tok = new StringTokenizer((String)o, " \",()*", false);
            while (tok.hasMoreTokens()) output.add(mTupleFactory.newTuple(tok.nextToken()));
            return output;
        } catch (ExecException ee) {
            // error handling goes here
        }
    }
    public Schema outputSchema(Schema input) {
         try{
             Schema bagSchema = new Schema();
             bagSchema.add(new Schema.FieldSchema("token", DataType.CHARARRAY));

             return new Schema(new Schema.FieldSchema(getSchemaName(this.getClass().getName().toLowerCase(), input),
                                                    bagSchema, DataType.BAG));
         }catch (Exception e){
            return null;
         }
    }
}
One more note about schemas and UDFs. Users have requested the ability to examine the input schema of the data before processing the data via a UDF. For example, they would like to know how to convert an input tuple to a map such that the keys in the map are the names of the input columns. Currently there is no way to do this. This is a feature we would like to support in the future.

Error Handling

There are several types of errors that can occur in a UDF:
  1. An error that affects a particular row but is not likely to impact other rows. An example of such an error would be a malformed input value or divide by zero problem. A reasonable handling of this situation would be to emit a warning and return a null value. ABS function in the next section demonstrates this approach. The current approach is to write the warning to stderr. Eventually we would like to pass a logger to the UDFs. Note that returning a NULL value only makes sense if the malformed value is of type bytearray. Otherwise the proper type has been already created and should have an appropriate value. If this is not the case, it is an internal error and should cause the system to fail. Both cases can be seen in the implementation of the ABS function in the next section.
  2. An error that affects the entire processing but can succeed on retry. An example of such a failure is the inability to open a lookup file because the file could not be found. This could be a temporary environmental issue that can go away on retry. A UDF can signal this to Pig by throwing an IOException as with the case of the ABS function below.
  3. An error that affects the entire processing and is not likely to succeed on retry. An example of such a failure is the inability to open a lookup file because of file permission problems. Pig currently does not have a way to handle this case. Hadoop does not have a way to handle this case either. It will be handled the same way as 2 above.

Function Overloading

Before the type system was available in Pig, all values for the purpose of arithmetic calculations were assumed to be doubles as the safest choice. However, this is not very efficient if the data is actually of type integer or long. (We saw about a 2x slowdown of a query when using double where integer could be used.) Now that Pig supports types we can take advantage of the type information and choose the function that is most efficient for the provided operands.
UDF writers are encouraged to provide type-specific versions of a function if this can result in better performance. On the other hand, we don't want the users of the functions to worry about different functions - the right thing should just happen. Pig allows for this via a function table mechanism as shown in the next example.
This example shows the implementation of the ABS function that returns the absolute value of a numeric value passed to it as input.
import java.io.IOException;
import java.util.List;
import java.util.ArrayList;
import org.apache.pig.EvalFunc;
import org.apache.pig.FuncSpec;
import org.apache.pig.data.Tuple;
import org.apache.pig.impl.logicalLayer.FrontendException;
import org.apache.pig.impl.logicalLayer.schema.Schema;
import org.apache.pig.data.DataType;

public class ABS extends EvalFunc<Double> {
    public Double exec(Tuple input) throws IOException {
        if (input == null || input.size() == 0)
            return null;
        Double d;
        try{
            d = DataType.toDouble(input.get(0));
        } catch (NumberFormatException nfe){
            System.err.println("Failed to process input; error - " + nfe.getMessage());
            return null;
        } catch (Exception e){
            throw new IOException("Caught exception processing input row ", e);
        }
        return Math.abs(d);
    }
    public List<FuncSpec> getArgToFuncMapping() throws FrontendException {
        List<FuncSpec> funcList = new ArrayList<FuncSpec>();
        funcList.add(new FuncSpec(this.getClass().getName(), new Schema(new Schema.FieldSchema(null, DataType.BYTEARRAY))));
        funcList.add(new FuncSpec(DoubleAbs.class.getName(),  new Schema(new Schema.FieldSchema(null, DataType.DOUBLE))));
        funcList.add(new FuncSpec(FloatAbs.class.getName(),   new Schema(new Schema.FieldSchema(null, DataType.FLOAT))));
        funcList.add(new FuncSpec(IntAbs.class.getName(),  new Schema(new Schema.FieldSchema(null, DataType.INTEGER))));
        funcList.add(new FuncSpec(LongAbs.class.getName(),  new Schema(new Schema.FieldSchema(null, DataType.LONG))));
        return funcList;
    }
}
The main thing to notice in this example is the getArgToFuncMapping() method. This method returns a list that contains a mapping from the input schema to the class that should be used to handle it. In this example the main class handles the bytearray input and outsources the rest of the work to other classes implemented in separate files in the same package. The example of one such class is below. This class handles integer input values.
import java.io.IOException;
import org.apache.pig.EvalFunc;
import org.apache.pig.data.Tuple;

public class IntAbs extends EvalFunc<Integer> {
    public Integer exec(Tuple input) throws IOException {
        if (input == null || input.size() == 0)
            return null;
        Integer d;
        try{
            d = (Integer)input.get(0);
        } catch (Exception e){
            throw new IOException("Caught exception processing input row ", e);
        }
        return Math.abs(d);
    }
}
A note on error handling. The ABS class covers the case of the bytearray which means the data has not been converted yet to its actual type. This is why a null value is returned when NumberFormatException is encountered. However, the IntAbs function is only called if the data is already of type Integer which means it has already been converted to the real type and bad format has been dealt with. This is why an exception is thrown if the input can't be cast to Integer.
The example above covers a reasonably simple case where the UDF only takes one parameter and there is a separate function for each parameter type. However, this will not always be the case. If Pig can't find an exact match it tries to do a best match. The rule for the best match is to find the most efficient function that can be used safely. This means that Pig must find the function that, for each input parameter, provides the smallest type that is equal to or greater than the input type. The type progression rules are: int>long>float>double.
For instance, let's consider function MAX which is part of the piggybank described later in this document. Given two values, the function returns the larger value. The function table for MAX looks like this:
public List<FuncSpec> getArgToFuncMapping() throws FrontendException {
    List<FuncSpec> funcList = new ArrayList<FuncSpec>();
    Util.addToFunctionList(funcList, IntMax.class.getName(), DataType.INTEGER);
    Util.addToFunctionList(funcList, DoubleMax.class.getName(), DataType.DOUBLE);
    Util.addToFunctionList(funcList, FloatMax.class.getName(), DataType.FLOAT);
    Util.addToFunctionList(funcList, LongMax.class.getName(), DataType.LONG);

    return funcList;
}
The Util.addToFunctionList function is a helper function that adds an entry to the list as the first argument, with the key of the class name passed as the second argument, and the schema containing two fields of the same type as the third argument.
Let's now see how this function can be used in a Pig script:
REGISTER piggybank.jar
A = LOAD 'student_data' AS (name: chararray, gpa1: float, gpa2: double);
B = FOREACH A GENERATE name, org.apache.pig.piggybank.evaluation.math.MAX(gpa1, gpa2);
DUMP B;
In this example, the function gets one parameter of type float and another of type double. The best fit will be the function that takes two double values. Pig makes this choice on the user's behalf by inserting implicit casts for the parameters. Running the script above is equivalent to running the script below:
A = LOAD 'student_data' AS (name: chararray, gpa1: float, gpa2: double);
B = FOREACH A GENERATE name, org.apache.pig.piggybank.evaluation.math.MAX((double)gpa1, gpa2);
DUMP B;
A special case of the best fit approach is handling data without a schema specified. The type for this data is interpreted as bytearray. Since the type of the data is not known, there is no way to choose a best fit version. The only time a cast is performed is when the function table contains only a single entry. This works well to maintain backward compatibility.
Let's revisit the UPPER function from our first example. As it is written now, it would only work if the data passed to it is of type chararray. To make it work with data whose type is not explicitly set, a function table with a single entry needs to be added:
package myudfs;
import java.io.IOException;
import org.apache.pig.EvalFunc;
import org.apache.pig.data.Tuple;

public class UPPER extends EvalFunc<String>
{
    public String exec(Tuple input) throws IOException {
        if (input == null || input.size() == 0)
            return null;
        try{
            String str = (String)input.get(0);
            return str.toUpperCase();
        }catch(Exception e){
            System.err.println("WARN: UPPER: failed to process input; error - " + e.getMessage());
            return null;
        }
    }
    public List<FuncSpec> getArgToFuncMapping() throws FrontendException {
        List<FuncSpec> funcList = new ArrayList<FuncSpec>();
        funcList.add(new FuncSpec(this.getClass().getName(), new Schema(new Schema.FieldSchema(null, DataType.CHARARRAY))));
        return funcList;
    }
}
Now the following script will ran:
-- this is myscript.pig
REGISTER myudfs.jar;
A = LOAD 'student_data' AS (name, age, gpa);
B = FOREACH A GENERATE myudfs.UPPER(name);
DUMP B;
Variable-length arguments:
The last input schema field in getArgToFuncMapping() can be marked as vararg, which enables UDF writers to create UDFs that take variable length arguments. This is done by overriding the getSchemaType() method:
@Override
public SchemaType getSchemaType() {
    return SchemaType.VARARG;
}
For an example see CONCAT.

Using Counters

Hadoop counters are easily accessible within EvalFunc by using PigStatusReporter object. Here is one example:
public class UPPER extends EvalFunc<String>
{
        public String exec(Tuple input) throws IOException {
                if (input == null || input.size() == 0) {
                    PigStatusReporter reporter = PigStatusReporter.getInstance();
                    if (reporter != null) {
                       reporter.incrCounter(PigWarning.UDF_WARNING_1, 1);
                    }
                    return null;
                }
                try{
                        String str = (String)input.get(0);
                        return str.toUpperCase();
                }catch(Exception e){
                    throw new IOException("Caught exception processing input row ", e);
                }
        }
}

Access input schema inside EvalFunc

Not only inside outputSchema at compile time, input schema is also accessible in exec at runtime. For example:
public class AddSchema extends EvalFunc<String>
{
        public String exec(Tuple input) throws IOException {
                if (input == null || input.size() == 0)
                    return null;
                String result = "";
                for (int i=0;i<input.size();i++) {
                    result += getInputSchema().getFields().get(i).alias;
                    result += ":";
                    result += input.get(i);
                }
                return result;
        }
}

Reporting Progress

A challenge of running a large shared system is to make sure system resources are used efficiently. One aspect of this challenge is detecting runaway processes that are no longer making progress. Pig uses a heartbeat mechanism for this purpose. If any of the tasks stops sending a heartbeat, the system assumes that it is dead and kills it.
Most of the time, single-tuple processing within a UDF is very short and does not require a UDF to heartbeat. The same is true for aggregate functions that operate on large bags because bag iteration code takes care of it. However, if you have a function that performs a complex computation that can take an order of minutes to execute, you should add a progress indicator to your code. This is very easy to accomplish. The EvalFunc class provides a progress function that you need to call in your exec method.
For instance, the UPPER function would now look as follows:
public class UPPER extends EvalFunc<String>
{
        public String exec(Tuple input) throws IOException {
                if (input == null || input.size() == 0)
                return null;
                try{
                        progress();
                        String str = (String)input.get(0);
                        return str.toUpperCase();
                }catch(Exception e){
                    throw new IOException("Caught exception processing input row ", e);
                }
        }
}

Using Distributed Cache

Use getCacheFiles or getShipFiles to return a list of HDFS files or local files that need to be shipped to distributed cache. Inside exec method, you can assume that these files already exist in distributed cache. For example:
public class Udfcachetest extends EvalFunc<String> { 

    public String exec(Tuple input) throws IOException { 
        String concatResult = "";
        FileReader fr = new FileReader("./smallfile1"); 
        BufferedReader d = new BufferedReader(fr);
        concatResult +=d.readLine();
        fr = new FileReader("./smallfile2");
        d = new BufferedReader(fr);
        concatResult +=d.readLine();
        return concatResult;
    } 

    public List<String> getCacheFiles() { 
        List<String> list = new ArrayList<String>(1); 
        list.add("/user/pig/tests/data/small#smallfile1");  // This is hdfs file
        return list; 
    } 

    public List<String> getShipFiles() {
        List<String> list = new ArrayList<String>(1);
        list.add("/home/hadoop/pig/smallfile2");  // This local file
        return list;
    }
} 

a = load '1.txt'; 
b = foreach a generate Udfcachetest(*); 
dump b;

Compile time evaluation

If the parameters of the EvalFunc are all constants, Pig could evaluate the result at compile time. The benefit of evaluating at compile time is performance optimization, and enable certain other optimizations at front end (such as partition pruning, which only allow constant not UDF in filter condition). By default, compile time evaluation is disabled in EvalFunc to prevent potential side effect. To enable it, override allowCompileTimeCalculation. For example:
public class CurrentTime extends EvalFunc<DateTime> {
    public String exec(Tuple input) throws IOException {
        return new DateTime();
    }
    @Override
    public boolean allowCompileTimeCalculation() {
        return true;
    }
}

Typecasting from bytearrays

Just like Load Function and Streaming, Java UDF has a getLoadCaster() method that returns LoadCaster to convert byte arrays to specific types. A UDf implementation should implement this if casts (implicit or explicit) from DataByteArray fields to other types need to be supported. Default implementation returns null and Pig will determine if all parameters passed to the UDF have identical loadcaster and use it when true.

Clean up static variable in Tez

In Tez, jvm could reuse for other tasks. It is important to cleanup static variable to make sure there is no side effect. Here is one example:
public class UPPER extends EvalFunc<String>
{
        static boolean initialized = false;
        static {
            JVMReuseManager.getInstance().registerForStaticDataCleanup(UPPER.class);
        }
        public String exec(Tuple input) throws IOException {
            if (!initialized) {
                init();
                initialized = true;
            }
            ......
        }
        @StaticDataCleanup
        public static void staticDataCleanup() {
            initialized = false;
        }
}

Load/Store Functions

The load/store UDFs control how data goes into Pig and comes out of Pig. Often, the same function handles both input and output but that does not have to be the case.
The Pig load/store API is aligned with Hadoop's InputFormat and OutputFormat classes. This enables you to create new LoadFunc and StoreFunc implementations based on existing Hadoop InputFormat and OutputFormat classes with minimal code. The complexity of reading the data and creating a record lies in the InputFormat while the complexity of writing the data lies in the OutputFormat. This enables Pig to easily read/write data in new storage formats as and when an Hadoop InputFormat and OutputFormat is available for them.
Note: Both the LoadFunc and StoreFunc implementations should use the Hadoop 20 API based classes (InputFormat/OutputFormat and related classes) under the new org.apache.hadoop.mapreduce package instead of the old org.apache.hadoop.mapred package.

Load Functions


LoadFunc abstract class has three main methods for loading data and for most use cases it would suffice to extend it. There are three other optional interfaces which can be implemented to achieve extended functionality:
  • LoadMetadata has methods to deal with metadata - most implementation of loaders don't need to implement this unless they interact with some metadata system. The getSchema() method in this interface provides a way for loader implementations to communicate the schema of the data back to pig. If a loader implementation returns data comprised of fields of real types (rather than DataByteArray fields), it should provide the schema describing the data returned through the getSchema() method. The other methods are concerned with other types of metadata like partition keys and statistics. Implementations can return null return values for these methods if they are not applicable for that implementation.
  • LoadPushDown has methods to push operations from Pig runtime into loader implementations. Currently only the pushProjection() method is called by Pig to communicate to the loader the exact fields that are required in the Pig script. The loader implementation can choose to honor the request (return only those fields required by Pig script) or not honor the request (return all fields in the data). If the loader implementation can efficiently honor the request, it should implement LoadPushDown to improve query performance. (Irrespective of whether the implementation can or cannot honor the request, if the implementation also implements getSchema(), the schema returned in getSchema() should describe the entire tuple of data.)
    • pushProjection(): This method tells LoadFunc which fields are required in the Pig script, thus enabling LoadFunc to optimize performance by loading only those fields that are needed. pushProjection() takes a requiredFieldList. requiredFieldList is read only and cannot be changed by LoadFunc. requiredFieldList includes a list of requiredField: each requiredField indicates a field required by the Pig script; each requiredField includes index, alias, type (which is reserved for future use), and subFields. Pig will use the column index requiredField.index to communicate with the LoadFunc about the fields required by the Pig script. If the required field is a map, Pig will optionally pass requiredField.subFields which contains a list of keys that the Pig script needs for the map. For example, if the Pig script needs two keys for the map, "key1" and "key2", the subFields for that map will contain two requiredField; the alias field for the first RequiredField will be "key1" and the alias for the second requiredField will be "key2". LoadFunc will use requiredFieldResponse.requiredFieldRequestHonored to indicate whether the pushProjection() request is honored.
  • LoadCaster has methods to convert byte arrays to specific types. A loader implementation should implement this if casts (implicit or explicit) from DataByteArray fields to other types need to be supported.
  • LoadPredicatePushdown has the methods to push predicates to the loader. It is different than LoadMetadata.setPartitionFilter in that loader may load records which does not satisfy the predicates. In other words, predicates is only a hint. Note this interface is still in development and might change in next version. Currently only OrcStorage implements this interface.
  • NonFSLoadFunc is a marker interface to indicate that a LoadFunc implementation is not a filesystem loader. This is useful for LoadFunc classes that for example supply queries instead of filesystem pathes to the LOAD operator.
The LoadFunc abstract class is the main class to extend for implementing a loader. The methods which need to be overridden are explained below:
  • getInputFormat(): This method is called by Pig to get the InputFormat used by the loader. The methods in the InputFormat (and underlying RecordReader) are called by Pig in the same manner (and in the same context) as by Hadoop in a MapReduce java program. If the InputFormat is a Hadoop packaged one, the implementation should use the new API based one under org.apache.hadoop.mapreduce. If it is a custom InputFormat, it should be implemented using the new API in org.apache.hadoop.mapreduce. If a custom loader using a text-based InputFormat or a file-based InputFormat would like to read files in all subdirectories under a given input directory recursively, then it should use the PigTextInputFormat and PigFileInputFormat classes provided in org.apache.pig.backend.hadoop.executionengine.mapReduceLayer. The Pig InputFormat classes work around a current limitation in the Hadoop TextInputFormat and FileInputFormat classes which only read one level down from the provided input directory. For example, if the input in the load statement is 'dir1' and there are subdirs 'dir2' and 'dir2/dir3' beneath dir1, the Hadoop TextInputFormat and FileInputFormat classes read the files under 'dir1' only. Using PigTextInputFormat or PigFileInputFormat (or by extending them), the files in all the directories can be read.
  • setLocation(): This method is called by Pig to communicate the load location to the loader. The loader should use this method to communicate the same information to the underlying InputFormat. This method is called multiple times by pig - implementations should bear this in mind and should ensure there are no inconsistent side effects due to the multiple calls.
  • prepareToRead(): Through this method the RecordReader associated with the InputFormat provided by the LoadFunc is passed to the LoadFunc. The RecordReader can then be used by the implementation in getNext() to return a tuple representing a record of data back to pig.
  • getNext(): The meaning of getNext() has not changed and is called by Pig runtime to get the next tuple in the data - in this method the implementation should use the underlying RecordReader and construct the tuple to return.

The following methods have default implementations in LoadFunc and should be overridden only if needed:
  • setUdfContextSignature(): This method will be called by Pig both in the front end and back end to pass a unique signature to the Loader. The signature can be used to store into the UDFContext any information which the Loader needs to store between various method invocations in the front end and back end. A use case is to store RequiredFieldList passed to it in LoadPushDown.pushProjection(RequiredFieldList) for use in the back end before returning tuples in getNext(). The default implementation in LoadFunc has an empty body. This method will be called before other methods.
  • relativeToAbsolutePath(): Pig runtime will call this method to allow the Loader to convert a relative load location to an absolute location. The default implementation provided in LoadFunc handles this for FileSystem locations. If the load source is something else, loader implementation may choose to override this.
  • getCacheFiles(): Return a list of hdfs files to ship to distributed cache.
  • getShipFiles(): Return a list of local files to ship to distributed cache.
Example Implementation
The loader implementation in the example is a loader for text data with line delimiter as '\n' and '\t' as default field delimiter (which can be overridden by passing a different field delimiter in the constructor) - this is similar to current PigStorage loader in Pig. The implementation uses an existing Hadoop supported Inputformat - TextInputFormat - as the underlying InputFormat.
public class SimpleTextLoader extends LoadFunc {
    protected RecordReader in = null;
    private byte fieldDel = '\t';
    private ArrayList<Object> mProtoTuple = null;
    private TupleFactory mTupleFactory = TupleFactory.getInstance();
    private static final int BUFFER_SIZE = 1024;

    public SimpleTextLoader() {
    }

    /**
     * Constructs a Pig loader that uses specified character as a field delimiter.
     *
     * @param delimiter
     *            the single byte character that is used to separate fields.
     *            ("\t" is the default.)
     */
    public SimpleTextLoader(String delimiter) {
        this();
        if (delimiter.length() == 1) {
            this.fieldDel = (byte)delimiter.charAt(0);
        } else if (delimiter.length() >  1 & & delimiter.charAt(0) == '\\') {
            switch (delimiter.charAt(1)) {
            case 't':
                this.fieldDel = (byte)'\t';
                break;

            case 'x':
               fieldDel =
                    Integer.valueOf(delimiter.substring(2), 16).byteValue();
               break;

            case 'u':
                this.fieldDel =
                    Integer.valueOf(delimiter.substring(2)).byteValue();
                break;

            default:
                throw new RuntimeException("Unknown delimiter " + delimiter);
            }
        } else {
            throw new RuntimeException("PigStorage delimeter must be a single character");
        }
    }

    @Override
    public Tuple getNext() throws IOException {
        try {
            boolean notDone = in.nextKeyValue();
            if (notDone) {
                return null;
            }
            Text value = (Text) in.getCurrentValue();
            byte[] buf = value.getBytes();
            int len = value.getLength();
            int start = 0;

            for (int i = 0; i < len; i++) {
                if (buf[i] == fieldDel) {
                    readField(buf, start, i);
                    start = i + 1;
                }
            }
            // pick up the last field
            readField(buf, start, len);

            Tuple t =  mTupleFactory.newTupleNoCopy(mProtoTuple);
            mProtoTuple = null;
            return t;
        } catch (InterruptedException e) {
            int errCode = 6018;
            String errMsg = "Error while reading input";
            throw new ExecException(errMsg, errCode,
                    PigException.REMOTE_ENVIRONMENT, e);
        }

    }

    private void readField(byte[] buf, int start, int end) {
        if (mProtoTuple == null) {
            mProtoTuple = new ArrayList<Object>();
        }

        if (start == end) {
            // NULL value
            mProtoTuple.add(null);
        } else {
            mProtoTuple.add(new DataByteArray(buf, start, end));
        }
    }

    @Override
    public InputFormat getInputFormat() {
        return new TextInputFormat();
    }

    @Override
    public void prepareToRead(RecordReader reader, PigSplit split) {
        in = reader;
    }

    @Override
    public void setLocation(String location, Job job)
            throws IOException {
        FileInputFormat.setInputPaths(job, location);
    }
}

Store Functions


StoreFunc abstract class has the main methods for storing data and for most use cases it should suffice to extend it. There is an optional interface which can be implemented to achieve extended functionality:
  • StoreMetadata: This interface has methods to interact with metadata systems to store schema and store statistics. This interface is optional and should only be implemented if metadata needs to stored.
  • StoreResources: This interface has methods to put hdfs files or local files to distributed cache.
  • ErrorHandling: This interface allow you to skip bad records in the storer so the storer will not throw exception and terminate the job. You can implement your own error handler by overriding ErrorHandler interface, or use predefined error handler: CounterBasedErrorHandler. ErrorHandling can be turned on by setting the property pig.error-handling.enabled to true in pig.properties. Default is false. CounterBasedErrorHandler uses two settings - pig.error-handling.min.error.records (the minimum number of errors to trigger error handling) and pig.error-handling.error.threshold (percentage of the number of records as a fraction exceeding which error is thrown).

The methods which need to be overridden in StoreFunc are explained below:
  • getOutputFormat(): This method will be called by Pig to get the OutputFormat used by the storer. The methods in the OutputFormat (and underlying RecordWriter and OutputCommitter) will be called by pig in the same manner (and in the same context) as by Hadoop in a map-reduce java program. If the OutputFormat is a hadoop packaged one, the implementation should use the new API based one under org.apache.hadoop.mapreduce. If it is a custom OutputFormat, it should be implemented using the new API under org.apache.hadoop.mapreduce. The checkOutputSpecs() method of the OutputFormat will be called by pig to check the output location up-front. This method will also be called as part of the Hadoop call sequence when the job is launched. So implementations should ensure that this method can be called multiple times without inconsistent side effects.
  • setStoreLocation(): This method is called by Pig to communicate the store location to the storer. The storer should use this method to communicate the same information to the underlying OutputFormat. This method is called multiple times by pig - implementations should bear in mind that this method is called multiple times and should ensure there are no inconsistent side effects due to the multiple calls.
  • prepareToWrite(): Writing of the data is through the OutputFormat provided by the StoreFunc. In prepareToWrite() the RecordWriter associated with the OutputFormat provided by the StoreFunc is passed to the StoreFunc. The RecordWriter can then be used by the implementation in putNext() to write a tuple representing a record of data in a manner expected by the RecordWriter.
  • putNext(): This method is called by Pig runtime to write the next tuple of data - this is the method wherein the implementation will use the underlying RecordWriter to write the Tuple out.

The following methods have default implementations in StoreFunc and should be overridden only if necessary:
  • setStoreFuncUDFContextSignature(): This method will be called by Pig both in the front end and back end to pass a unique signature to the Storer. The signature can be used to store into the UDFContext any information which the Storer needs to store between various method invocations in the front end and back end. The default implementation in StoreFunc has an empty body. This method will be called before other methods.
  • relToAbsPathForStoreLocation(): Pig runtime will call this method to allow the Storer to convert a relative store location to an absolute location. An implementation is provided in StoreFunc which handles this for FileSystem based locations.
  • checkSchema(): A Store function should implement this function to check that a given schema describing the data to be written is acceptable to it. The default implementation in StoreFunc has an empty body. This method will be called before any calls to setStoreLocation().
Example Implementation
The storer implementation in the example is a storer for text data with line delimiter as '\n' and '\t' as default field delimiter (which can be overridden by passing a different field delimiter in the constructor) - this is similar to current PigStorage storer in Pig. The implementation uses an existing Hadoop supported OutputFormat - TextOutputFormat as the underlying OutputFormat.
public class SimpleTextStorer extends StoreFunc {
    protected RecordWriter writer = null;

    private byte fieldDel = '\t';
    private static final int BUFFER_SIZE = 1024;
    private static final String UTF8 = "UTF-8";
    public PigStorage() {
    }

    public PigStorage(String delimiter) {
        this();
        if (delimiter.length() == 1) {
            this.fieldDel = (byte)delimiter.charAt(0);
        } else if (delimiter.length() > 1delimiter.charAt(0) == '\\') {
            switch (delimiter.charAt(1)) {
            case 't':
                this.fieldDel = (byte)'\t';
                break;

            case 'x':
               fieldDel =
                    Integer.valueOf(delimiter.substring(2), 16).byteValue();
               break;
            case 'u':
                this.fieldDel =
                    Integer.valueOf(delimiter.substring(2)).byteValue();
                break;

            default:
                throw new RuntimeException("Unknown delimiter " + delimiter);
            }
        } else {
            throw new RuntimeException("PigStorage delimeter must be a single character");
        }
    }

    ByteArrayOutputStream mOut = new ByteArrayOutputStream(BUFFER_SIZE);

    @Override
    public void putNext(Tuple f) throws IOException {
        int sz = f.size();
        for (int i = 0; i < sz; i++) {
            Object field;
            try {
                field = f.get(i);
            } catch (ExecException ee) {
                throw ee;
            }

            putField(field);

            if (i != sz - 1) {
                mOut.write(fieldDel);
            }
        }
        Text text = new Text(mOut.toByteArray());
        try {
            writer.write(null, text);
            mOut.reset();
        } catch (InterruptedException e) {
            throw new IOException(e);
        }
    }

    @SuppressWarnings("unchecked")
    private void putField(Object field) throws IOException {
        //string constants for each delimiter
        String tupleBeginDelim = "(";
        String tupleEndDelim = ")";
        String bagBeginDelim = "{";
        String bagEndDelim = "}";
        String mapBeginDelim = "[";
        String mapEndDelim = "]";
        String fieldDelim = ",";
        String mapKeyValueDelim = "#";

        switch (DataType.findType(field)) {
        case DataType.NULL:
            break; // just leave it empty

        case DataType.BOOLEAN:
            mOut.write(((Boolean)field).toString().getBytes());
            break;

        case DataType.INTEGER:
            mOut.write(((Integer)field).toString().getBytes());
            break;

        case DataType.LONG:
            mOut.write(((Long)field).toString().getBytes());
            break;

        case DataType.FLOAT:
            mOut.write(((Float)field).toString().getBytes());
            break;

        case DataType.DOUBLE:
            mOut.write(((Double)field).toString().getBytes());
            break;

        case DataType.BYTEARRAY: {
            byte[] b = ((DataByteArray)field).get();
            mOut.write(b, 0, b.length);
            break;
                                 }

        case DataType.CHARARRAY:
            // oddly enough, writeBytes writes a string
            mOut.write(((String)field).getBytes(UTF8));
            break;

        case DataType.MAP:
            boolean mapHasNext = false;
            Map<String, Object> m = (Map<String, Object>)field;
            mOut.write(mapBeginDelim.getBytes(UTF8));
            for(Map.Entry<String, Object> e: m.entrySet()) {
                if(mapHasNext) {
                    mOut.write(fieldDelim.getBytes(UTF8));
                } else {
                    mapHasNext = true;
                }
                putField(e.getKey());
                mOut.write(mapKeyValueDelim.getBytes(UTF8));
                putField(e.getValue());
            }
            mOut.write(mapEndDelim.getBytes(UTF8));
            break;

        case DataType.TUPLE:
            boolean tupleHasNext = false;
            Tuple t = (Tuple)field;
            mOut.write(tupleBeginDelim.getBytes(UTF8));
            for(int i = 0; i < t.size(); ++i) {
                if(tupleHasNext) {
                    mOut.write(fieldDelim.getBytes(UTF8));
                } else {
                    tupleHasNext = true;
                }
                try {
                    putField(t.get(i));
                } catch (ExecException ee) {
                    throw ee;
                }
            }
            mOut.write(tupleEndDelim.getBytes(UTF8));
            break;

        case DataType.BAG:
            boolean bagHasNext = false;
            mOut.write(bagBeginDelim.getBytes(UTF8));
            Iterator<Tuple> tupleIter = ((DataBag)field).iterator();
            while(tupleIter.hasNext()) {
                if(bagHasNext) {
                    mOut.write(fieldDelim.getBytes(UTF8));
                } else {
                    bagHasNext = true;
                }
                putField((Object)tupleIter.next());
            }
            mOut.write(bagEndDelim.getBytes(UTF8));
            break;

        default: {
            int errCode = 2108;
            String msg = "Could not determine data type of field: " + field;
            throw new ExecException(msg, errCode, PigException.BUG);
        }

        }
    }

    @Override
    public OutputFormat getOutputFormat() {
        return new TextOutputFormat<WritableComparable, Text>();
    }

    @Override
    public void prepareToWrite(RecordWriter writer) {
        this.writer = writer;
    }

    @Override
    public void setStoreLocation(String location, Job job) throws IOException {
        job.getConfiguration().set("mapred.textoutputformat.separator", "");
        FileOutputFormat.setOutputPath(job, new Path(location));
        if (location.endsWith(".bz2")) {
            FileOutputFormat.setCompressOutput(job, true);
            FileOutputFormat.setOutputCompressorClass(job,  BZip2Codec.class);
        }  else if (location.endsWith(".gz")) {
            FileOutputFormat.setCompressOutput(job, true);
            FileOutputFormat.setOutputCompressorClass(job, GzipCodec.class);
        }
    }
}

Using Short Names

There are two ways to call a Java UDF using a short name. One way is specifying the package to an import list via Java property, and the other is defining an alias of the UDF by DEFINE statement.

Import Lists

An import list allows you to specify the package to which a UDF or a group of UDFs belong, eliminating the need to qualify the UDF on every call. An import list can be specified via the udf.import.list Java property on the Pig command line:
pig -Dudf.import.list=com.yahoo.yst.sds.ULT
You can supply multiple locations as well:
pig -Dudf.import.list=com.yahoo.yst.sds.ULT:org.apache.pig.piggybank.evaluation
To make use of import scripts, do the following:
myscript.pig:
A = load '/data/SDS/data/searcg_US/20090820' using ULTLoader as (s, m, l);
....

command:
pig -cp sds.jar -Dudf.import.list=com.yahoo.yst.sds.ULT myscript.pig 

Defining Aliases

You can define an alias for a function using DEFINE statement:
REGISTER piggybank.jar
DEFINE MAXNUM org.apache.pig.piggybank.evaluation.math.MAX;
A = LOAD 'student_data' AS (name: chararray, gpa1: float, gpa2: double);
B = FOREACH A GENERATE name, MAXNUM(gpa1, gpa2);
DUMP B;
The first parameter of DEFINE statement is an alias of the function. The second parameter is the fully-qualified name of the function. After the statement, you can call the function using the alias instead of the fually-qualified name.

Advanced Topics


UDF Interfaces

Java UDFs can be invoked multiple ways. The simplest UDF can just extend EvalFunc, which requires only the exec function to be implemented (see How to Write a Simple Eval Function). Every eval UDF must implement this. Additionally, if a function is algebraic, it can implement Algebraic interface to significantly improve query performance in the cases when combiner can be used (see Algebraic Interface). Finally, a function that can process tuples in an incremental fashion can also implement the Accumulator interface to improve query memory consumption (see Accumulator Interface).
The optimizer selects the exact method by which a UDF is invoked based on the UDF type and the query. Note that only a single interface is used at any given time. The optimizer tries to find the most efficient way to execute the function. If a combiner is used and the function implements the Algebraic interface then this interface will be used to invoke the function. If the combiner is not invoked but the accumulator can be used and the function implements Accumulator interface then that interface is used. If neither of the conditions is satisfied then the exec function is used to invoke the UDF.

Function Instantiation

One problem that users run into is when they make assumption about how many times a constructor for their UDF is called. For instance, they might be creating side files in the store function and doing it in the constructor seems like a good idea. The problem with this approach is that in most cases Pig instantiates functions on the client side to, for instance, examine the schema of the data.
Users should not make assumptions about how many times a function is instantiated; instead, they should make their code resilient to multiple instantiations. For instance, they could check if the files exist before creating them.

Passing Configurations to UDFs

The singleton UDFContext class provides two features to UDF writers. First, on the backend, it allows UDFs to get access to the JobConf object, by calling getJobConf. This is only available on the backend (at run time) as the JobConf has not yet been constructed on the front end (during planning time).
Second, it allows UDFs to pass configuration information between instantiations of the UDF on the front and backends. UDFs can store information in a configuration object when they are constructed on the front end, or during other front end calls such as checkSchema. They can then read that information on the backend when exec (for EvalFunc) or getNext (for LoadFunc) is called. Note that information will not be passed between instantiations of the function on the backend. The communication channel only works from front end to back end.
To store information, the UDF calls getUDFProperties. This returns a Properties object which the UDF can record the information in or read the information from. To avoid name space conflicts UDFs are required to provide a signature when obtaining a Properties object. This can be done in two ways. The UDF can provide its Class object (via this.getClass()). In this case, every instantiation of the UDF will be given the same Properties object. The UDF can also provide its Class plus an array of Strings. The UDF can pass its constructor arguments, or some other identifying strings. This allows each instantiation of the UDF to have a different properties object thus avoiding name space collisions between instantiations of the UDF.

Monitoring Long-Running UDFs

Sometimes one may discover that a UDF that executes very quickly in the vast majority of cases turns out to run exceedingly slowly on occasion. This can happen, for example, if a UDF uses complex regular expressions to parse free-form strings, or if a UDF uses some external service to communicate with. As of version 0.8, Pig provides a facility for monitoring the length of time a UDF is executing for every invocation, and terminating its execution if it runs too long. This facility can be turned on using a simple Java annotation:
 import org.apache.pig.builtin.MonitoredUDF;
 
 @MonitoredUDF
 public class MyUDF extends EvalFunc<Integer> {
   /* implementation goes here */
 }
Simply annotating your UDF in this way will cause Pig to terminate the UDF's exec() method if it runs for more than 10 seconds, and return the default value of null. The duration of the timeout and the default value can be specified in the annotation, if desired:
 import org.apache.pig.builtin.MonitoredUDF;
 
 @MonitoredUDF(timeUnit = TimeUnit.MILLISECONDS, duration = 100, intDefault = 10)
 public class MyUDF extends EvalFunc<Integer> {
   /* implementation goes here */
 }
intDefault, longDefault, doubleDefault, floatDefault, and stringDefault can be specified in the annotation; the correct default will be chosen based on the return type of the UDF. Custom defaults for tuples and bags are not supported at this time.
If desired, custom logic can also be implemented for error handling by creating a subclass of MonitoredUDFExecutor.ErrorCallback, and overriding its handleError and/or handleTimeout methods. Both of those methods are static, and are passed in the instance of the EvalFunc that produced an exception, as well as an exception, so you may use any state you have in the UDF to process the errors as desired. The default behavior is to increment Hadoop counters every time an error is encountered. Once you have an implementation of the ErrorCallback that performs your custom logic, you can provide it in the annotation:
 import org.apache.pig.builtin.MonitoredUDF;

 @MonitoredUDF(errorCallback=MySpecialErrorCallback.class)
 public class MyUDF extends EvalFunc<Integer> {
   /* implementation goes here */
 }
Currently the MonitoredUDF annotation works with regular and Algebraic UDFs, but has no effect on UDFs that run in the Accumulator mode.

Writing Jython UDFs


Registering the UDF

You can register a Jython script as shown here. This example uses org.apache.pig.scripting.jython.JythonScriptEngine to interpret the Jython script. You can develop and use custom script engines to support multiple programming languages and ways to interpret them. Currently, Pig identifies jython as a keyword and ships the required scriptengine (jython) to interpret it.
Register 'test.py' using jython as myfuncs;
The following syntax is also supported, where myfuncs is the namespace created for all the functions inside test.py.
register 'test.py' using org.apache.pig.scripting.jython.JythonScriptEngine as myfuncs;
A typical test.py looks like this:
@outputSchema("word:chararray")
def helloworld():  
  return 'Hello, World'

@outputSchema("word:chararray,num:long")
def complex(word):
  return str(word),len(word)

@outputSchemaFunction("squareSchema")
def square(num):
  return ((num)*(num))

@schemaFunction("squareSchema")
def squareSchema(input):
  return input

# No decorator - bytearray
def concat(str):
  return str+str
The register statement above registers the Jython functions defined in test.py in Pig’s runtime within the defined namespace (myfuncs here). They can then be referred later on in the pig script as myfuncs.helloworld(), myfuncs.complex(), and myfuncs.square(). An example usage is:
b = foreach a generate myfuncs.helloworld(), myfuncs.square(3);

Decorators and Schemas

To annotate a Jython script so that Pig can identify return types, use Jython decorators to define output schema for the script UDF.
  • outputSchema - Defines schema for a script UDF in a format that Pig understands and is able to parse.
  • outputFunctionSchema - Defines a script delegate function that defines schema for this function depending upon the input type. This is needed for functions that can accept generic types and perform generic operations on these types. A simple example is square which can accept multiple types. SchemaFunction for this type is a simple identity function (same schema as input).
  • schemaFunction - Defines delegate function and is not registered to Pig.
When no decorator is specified, Pig assumes the output datatype as bytearray and converts the output generated by script function to bytearray. This is consistent with Pig's behavior in case of Java UDFs.
Sample Schema String - y:{t:(word:chararray,num:long)}, variable names inside a schema string are not used anywhere, they just make the syntax identifiable to the parser.

Example Scripts

Simple tasks like string manipulation, mathematical computations, and reorganizing data types can be easily done using Jython scripts without having to develop long and complex UDFs in Java. The overall overhead of using scripting language is much less and development cost is almost negligible. The following UDFs, developed in Jython, can be used with Pig.
 mySampleLib.py
 ---------------------
 #/usr/bin/python
 
 ##################
 # Math functions #
 ##################
 #square - Square of a number of any data type
 @outputSchemaFunction("squareSchema")
 def square(num):
   return ((num)*(num))
 @schemaFunction("squareSchema")
 def squareSchema(input):
   return input
 
 #Percent- Percentage
 @outputSchema("percent:double")
 def percent(num, total):
   return num * 100 / total
 
 ####################
 # String Functions #
 ####################
 #commaFormat- format a number with commas, 12345-> 12,345
 @outputSchema("numformat:chararray")
 def commaFormat(num):
   return '{:,}'.format(num)
 
 #concatMultiple- concat multiple words
 @outputSchema("onestring:chararray")
 def concatMult4(word1, word2, word3, word4):
   return word1 word2 word3 word4
 
 #######################
 # Data Type Functions #
 #######################
 #collectBag- collect elements of a bag into other bag
 #This is useful UDF after group operation
 @outputSchema("y:bag{t:tuple(len:int,word:chararray)}") 
 def collectBag(bag):
   outBag = []
   for word in bag:
     tup=(len(bag), word[1])
     outBag.append(tup)
   return outBag
 
 # Few comments- 
 # Pig mandates that a bag should be a bag of tuples, Jython UDFs should follow this pattern.
 # Tuples in Jython are immutable, appending to a tuple is not possible.
 

Advanced Topics


Importing Modules

You can import Jython modules in your Jython script. Pig resolves Jython dependencies recursively, which means Pig will automatically ship all dependent Jython modules to the backend. Jython modules should be found in the jython search path: JYTHON_HOME, JYTHONPATH, or current directory.

Combined Scripts

UDFs and Pig scripts are generally stored in separate files. For testing purposes you can combine the code in a single file - a "combined" script. Note, however, if you then decide to embed this "combined" script in a host language, the language of the UDF must match the host language.
This example combines Jython and Pig. This "combined" script can only be embedded in Jython.
With Jython you MUST use the if __name__ == '__main__': construct to separate UDFs and control flow. Otherwise the script will result in an error.
 #!/usr/bin/jython
from org.apache.pig.scripting import *

@outputSchema("word:chararray")
def helloworld():  
   return 'Hello, World'
  
if __name__ == '__main__':
       P = Pig.compile("""a = load '1.txt' as (a0, a1);
                          b = foreach a generate helloworld();
                          store b into 'myoutput'; """)

result = P.bind().runSingle();
 

Writing JavaScript UDFs

Note: JavaScript UDFs are an experimental feature.

Registering the UDF

You can register JavaScript as shown here. This example uses org.apache.pig.scripting.js.JsScriptEngine to interpret JavaScript. You can develop and use custom script engines to support multiple programming languages and ways to interpret them. Currently, Pig identifies js as a keyword and ships the required scriptengine (Rhino) to interpret it.
 register 'test.js' using javascript as myfuncs;
 
The following syntax is also supported, where myfuncs is the namespace created for all the functions inside test.js.
 register 'test.js' using org.apache.pig.scripting.js.JsScriptEngine as myfuncs;
 
The register statement above registers the js functions defined in test.js in Pig’s runtime within the defined namespace (myfuncs here). They can then be referred later on in the pig script as myfuncs.helloworld(), myfuncs.complex(), and myfuncs.square(). An example usage is:
 b = foreach a generate myfuncs.helloworld(), myfuncs.complex($0);
 

Return Types and Schemas

Since JavaScript functions are first class objects, you can annotate them by adding attributes. Add an outputSchema attribute to your function so that Pig can identify return types for the script UDF.
  • outputSchema - Defines schema for a script udf in a format that Pig understands and is able to parse.
  • Sample Schema String - y:{t:(word:chararray,num:long)} Variable names inside a schema string are used for type conversion between Pig and JavaScript. Tuples are converted to Objects using the names and vice versa

Example Scripts

A simple JavaScript UDF (udf.js) is shown here.
helloworld.outputSchema = "word:chararray";
function helloworld() {
    return 'Hello, World';
}
    
complex.outputSchema = "(word:chararray,num:long)";
function complex(word){
    return {word:word, num:word.length};
}
This Pig script registers the JavaScript UDF (udf.js).
register 'udf.js' using javascript as myfuncs; 
A = load 'data' as (a0:chararray, a1:int);
B = foreach A generate myfuncs.helloworld(), myfuncs.complex(a0);
... ... 

Advanced Topics

UDFs and Pig scripts are generally stored in separate files. For testing purposes you can combine the code in a single file - a "combined" script. Note, however, if you then decide to embed this "combined" script in a host language, the language of the UDF and the host language must match.
This example combines JavaScript and Pig. This "combined" script can only be embedded in JavaScript.
With JavaScript, the control flow MUST be defined in the main function. Otherwise the script will result in an error.
importPackage(Packages.org.apache.pig.scripting.js)
pig = org.apache.pig.scripting.js.JSPig;

helloworld.outputSchema = "word:chararray" 
function helloworld() { 
    return 'Hello, World'; 
}

function main() {
  var P = pig.compile(" a = load '1.txt' as (a0, a1);”+
       “b = foreach a generate helloworld();”+
           “store b into 'myoutput';");

  var result = P.bind().runSingle();
}
 

Writing Ruby UDFs

Note: Ruby UDFs are an experimental feature.

Writing a Ruby UDF

You must extend PigUdf and define your Ruby UDFs in the class.
 require 'pigudf'
 class Myudfs < PigUdf
     def square num
         return nil if num.nil?
         num**2
     end
 end
 

Return Types and Schemas

You have two ways to define the return schema:
outputSchema - Defines the schema for a UDF in a format that Pig understands.
 outputSchema "word:chararray"
 
 outputSchema "t:(m:[], t:(name:chararray, age:int, gpa:double), b:{t:(name:chararray, age:int, gpa:double)})"
 
Schema function
 outputSchemaFunction :squareSchema
 def squareSchema input
     input
 end
 
You need to put outputSchema/outputSchemaFunction statement right before your UDF. The schema function itself can be defined anywhere inside the class.

Registering the UDF

You can register a Ruby UDF as shown here.
 register 'test.rb' using jruby as myfuncs;
 
This is a shortcut to the complete syntax:
 register 'test.rb' using org.apache.pig.scripting.jruby.JrubyScriptEngine as myfuncs;
 
The register statement above registers the Ruby functions defined in test.rb in Pig’s runtime within the defined namespace (myfuncs in this example). They can then be referred later on in the Pig Latin script as myfuncs.square(). An example usage is:
 b = foreach a generate myfuncs.concat($0, $1);
 

Example Scripts

Here are two complete Ruby UDF samples.
 require 'pigudf'
 class Myudfs < PigUdf
 outputSchema "word:chararray"
     def concat *input
         input.inject(:+)
     end
 end
 
 require 'pigudf'
 class Myudfs < PigUdf
 outputSchemaFunction :squareSchema
     def square num
         return nil if num.nil?
         num**2
     end
     def squareSchema input
         input
     end
 end
 

Advanced Topics

You can also write Algebraic and Accumulator UDFs using Ruby. You need to extend your class from AlgebraicPigUdf and AccumulatorPigUdf respectively. For an Algebraic UDF, define initial, intermed, and final methods in the class. For an Accumulator UDF, define exec and get methods in the class. Below are example for each type of UDF:
 class Count < AlgebraicPigUdf
     output_schema Schema.long
     def initial t
          t.nil? ? 0 : 1
     end
     def intermed t
          return 0 if t.nil?
          t.flatten.inject(:+)
     end
     def final t
         intermed(t)
     end
 end
 
 class Sum < AccumulatorPigUdf
     output_schema { |i| i.in.in[0] }
     def exec items
         @sum ||= 0
         @sum += items.flatten.inject(:+)
     end
     def get
         @sum
     end
 end
 

Writing Groovy UDFs

Note: Groovy UDFs are an experimental feature.

Registering the UDF

You can register a Groovy script as shown here. This example uses org.apache.pig.scripting.groovy.GroovyScriptEngine to interpret the Groovy script. You can develop and use custom script engines to support multiple programming languages and ways to interpret them. Currently, Pig identifies groovy as a keyword and ships the required scriptengine (groovy-all) to interpret it.
Register 'test.groovy' using groovy as myfuncs;
The following syntax is also supported, where myfuncs is the namespace created for all the functions inside test.groovy.
register 'test.groovy' using org.apache.pig.scripting.groovy.GroovyScriptEngine as myfuncs;
A registered script can contain multiple UDFs. UDFs can be static or instance methods, an instance of the enclosing class will be created as needed. Only methods for which a return schema is defined will be available in Pig.

Return Types and Schemas

You have two ways to define the return schema, both use annotations:
@OutputSchema annotation - Defines the schema for a UDF in a format that Pig understands.
import org.apache.pig.builtin.OutputSchema;

class GroovyUDFs {
  @OutputSchema('x:long')
  long square(long x) {
    return x*x;
  }
}
outputSchema "t:(m:[], t:(name:chararray, age:int, gpa:double), b:{t:(name:chararray, age:int, gpa:double)})"
@OutputSchemaFunction annotation - Defines the name of a function which will return the schema at runtime according to the input schema.
import org.apache.pig.scripting.groovy.OutputSchemaFunction;

class GroovyUDFs {
  @OutputSchemaFunction('squareSchema')
  public static square(x) {
    return x * x;
  }

  public static squareSchema(input) {          
    return input;
  }
}        

Only methods annotated with either @OutputSchema or @OutputSchemaFunction will be exposed to Pig as UDFs. In the example above, squareSchema will not be available in Pig as a UDF.

Type Conversions

The data passed back and forth between Pig and Groovy goes through a conversion process. The following conversions rules are applied:
Pig to Groovy
  • Tuple: groovy.lang.Tuple
  • DataBag: groovy.lang.Tuple containing the bag's size and an iterator on its content
  • org.joda.time.DateTime: org.joda.time.DateTime
  • Map: java.util.Map
  • int/long/float/double: as is
  • chararray: String
  • bytearray: byte[] (content is copied)
  • boolean: boolean
  • biginteger: BigInteger
  • bigdecimal: BigDecimal
  • null: null
Anything else raises an exception
Groovy to Pig
  • Object[]: Tuple
  • groovy.lang.Tuple: Tuple
  • org.apache.pig.data.Tuple: Tuple
  • org.apache.pig.data.DataBag: DataBag
  • org.joda.time.DateTime: org.joda.time.DateTime
  • java.util.Map: Map
  • java.util.List: DataBag
  • Byte/Short/Integer: int
  • Long: long
  • Float: float
  • Double: double
  • String: chararray
  • byte[]: DataByteArray (content is copied)
  • Boolean: boolean
  • BigInteger: biginteger
  • BigDecimal: bigdecimal
  • null: null
Anything else raises an exception

Advanced Topics

You can also write Algebraic and Accumulator UDFs using Groovy. Both types of UDFs are declared using annotations, a single Groovy file can therefore enclose several Algebraic/Accumulator UDFs, all mixed with regular UDFs.
Algebraic UDFs are declared using three annotations, @AlgebraicInitial, @AlgebraicIntermed and @AlgebraicFinal which are to annotate methods that correspond to the initial, intermed and final steps of an Algebraic UDF. Those annotations have a single parameter which is the name of the Algebraic UDF that will be available in Pig. The methods annotated with @AlgebraicInitial and @AlgebraicIntermed accept a Tuple as parameter and return a Tuple. The return type of the method annotated with @AlgebraicFinal will determine the return type of the Algebraic UDF. Here is an example of an Algebraic UDF named 'sum' defined in Groovy:
import org.apache.pig.scripting.groovy.AlgebraicInitial;
import org.apache.pig.scripting.groovy.AlgebraicIntermed;
import org.apache.pig.scripting.groovy.AlgebraicFinal;

class GroovyUDFs {
  @AlgebraicFinal('sum')
  public static long algFinal(Tuple t) {
    long x = 0;
    for (Object o: t[1]) {
      x = x + o;
    }
    return x;
  }
  @AlgebraicInitial('sum')
  public static Tuple algInitial(Tuple t) {
    long x = 0;
    for (Object o: t[1]) {
      x = x + o[0];
    }
    return [x];
  }
  @AlgebraicIntermed('sum')
  public static Tuple algIntermed(Tuple t) {
    long x = 0;
    for (Object o: t[1]) {
      x = x + o;
    }
    return [x];
  }
}
 
Similarly, Accumulator UDFs are declared using the three annotations @AccumulatorAccumulate, @AccumulatorGetValue and @AccumulatorCleanup which are to annotate methods that correspond to the methods accumulate, getValue and cleanup of a Java Accumulator UDF. Those annotations have a single parameter which is the name of the Accumulator UDF that will be available in Pig. The methods annotated with @AccumulatorAccumulate and @AccumulatorCleanup return void. The methods annotated with @AccumulatorGetValue and @AccumulatorCleanup take no parameters. The method annotated with @AccumulatorAccumulate takes a Tuple as parameter. The return schema of the Accumulator UDF is determined by the @OutputSchema or @OutputSchemaFunction annotation used on the method annotated with @AccumulatorGetValue. Note that even though a method annotated with @AccumulatorGetValue has an @OutputSchema or @OutputSchemaFunction annotation, it will not be available in Pig, only the Accumulator UDF to which it belongs will.
Since Accumulator UDFs maintain state, the methods annotated with the @AccumulatorXXX annotations cannot be static. A single instance of the enclosing class will be used when invoking them, thus enabling them to access a single state.
The following example defines an Accumulator UDF named 'sumacc':
import org.apache.pig.builtin.OutputSchema;
import org.apache.pig.scripting.groovy.AccumulatorAccumulate;
import org.apache.pig.scripting.groovy.AccumulatorGetValue;
import org.apache.pig.scripting.groovy.AccumulatorCleanup;

class GroovyUDFs {
  private int sum = 0;
  @AccumulatorAccumulate('sumacc')
  public void accuAccumulate(Tuple t) {
    for (Object o: t[1]) {
      sum += o[0]
    }
  }
  @AccumulatorGetValue('sumacc')
  @OutputSchema('sum: long')
  public long accuGetValue() {
    return this.sum;
  }
  @AccumulatorCleanup('sumacc')
  public void accuCleanup() {
    this.sum = 0L;
  }
}  
 

Writing Python UDFs

Here Python UDFs means C Python UDFs. It uses python command line to run the Python UDFs. It is different than Jython, which relies on Jython library. Instead, it streams the data in and out of the python process. The implementation mechanism is completely different than Jython.

Registering the UDF

You can register a Python script as shown here.
Register 'test.py' using streaming_python as myfuncs;
The following syntax is also supported, where myfuncs is the namespace created for all the functions inside test.py.
register 'test.py' using org.apache.pig.scripting.streaming.python.PythonScriptEngine as myfuncs;
A typical test.py looks like this:
from pig_util import outputSchema

@outputSchema("as:int")
def square(num):
    if num == None:
        return None
    return ((num) * (num))

@outputSchema("word:chararray")
def concat(word):
    return word + word
The register statement above registers the Python functions defined in test.py in Pig’s runtime within the defined namespace (myfuncs here). They can then be referred later on in the pig script as myfuncs.square(), myfuncs.concat(). An example usage is:
b = foreach a generate myfuncs.concat('hello', 'world'), myfuncs.square(3);

Decorators and Schemas

To annotate a Python script so that Pig can identify return types, use Python decorators to define output schema for the script UDF.
  • outputSchema - Defines schema for a script UDF in a format that Pig understands and is able to parse.
When no decorator is specified, Pig assumes the output datatype as bytearray and converts the output generated by script function to bytearray. This is consistent with Pig's behavior in case of Java UDFs.
Sample Schema String - words:{(word:chararray)}, variable names inside a schema string are not used anywhere, they just make the syntax identifiable to the parser.

Piggy Bank

Piggy Bank is a place for Pig users to share the Java UDFs they have written for use with Pig. The functions are contributed "as-is." If you find a bug in a function, take the time to fix it and contribute the fix to Piggy Bank. If you don't find the UDF you need, take the time to write and contribute the function to Piggy Bank.
Note: Piggy Bank currently supports Java UDFs. Support for Jython and JavaScript UDFs will be added at a later date.

Accessing Functions

The Piggy Bank functions are currently distributed in source form. Users are required to checkout the code and build the package themselves. No binary distributions or nightly builds are available at this time.
To build a jar file that contains all available UDFs, follow these steps:
  • Checkout UDF code: svn co http://svn.apache.org/repos/asf/pig/trunk/contrib/piggybank
  • Add pig.jar to your ClassPath: export CLASSPATH=$CLASSPATH:/path/to/pig.jar
  • Build the jar file: from directorytrunk/contrib/piggybank/java run ant. This will generate piggybank.jar in the same directory.
To obtain javadoc description of the functions run ant javadoc from directory trunk/contrib/piggybank/java. The documentation is generate in directory trunk/contrib/piggybank/java/build/javadoc.
To use a function, you need to determine which package it belongs to. The top level packages correspond to the function type and currently are:
  • org.apache.pig.piggybank.comparison - for custom comparator used by ORDER operator
  • org.apache.pig.piggybank.evaluation - for eval functions like aggregates and column transformations
  • org.apache.pig.piggybank.filtering - for functions used in FILTER operator
  • org.apache.pig.piggybank.grouping - for grouping functions
  • org.apache.pig.piggybank.storage - for load/store functions
(The exact package of the function can be seen in the javadocs or by navigating the source tree.)
For example, to use the UPPER function:
REGISTER /public/share/pig/contrib/piggybank/java/piggybank.jar ;
TweetsInaug = FILTER Tweets BY org.apache.pig.piggybank.evaluation.string.UPPER(text) 
    MATCHES '.*(INAUG|OBAMA|BIDEN|CHENEY|BUSH).*' ;
STORE TweetsInaug INTO 'meta/inaug/tweets_inaug' ;

Contributing Functions

To contribute a Java function that you have written, do the following:
  1. Check the existing javadoc to make sure that the function does not already exist as described in Accessing Functions.
  2. Checkout the UDF code as described in Accessing Functions.
  3. Place your java code in the directory that makes sense for your function. The directory structure currently has two levels: (1) function type, as described in Accessing Functions, and (2) function subtype, for some of the types (like math or string for eval functions). If you think your function requires a new subtype, feel free to add one.
  4. Make sure that your function is well documented and uses the javadoc style of documentation.
  5. Make sure that your code follows Pig coding conventions described in How to Contribute to Pig.
  6. Make sure that for each function, you add a corresponding test class in the test part of the tree.
  7. Submit your patch following the process described in How to Contribute to Pig.

Embedded Pig - Python, JavaScript and Groovy

To enable control flow, you can embed Pig Latin statements and Pig commands in the Python, JavaScript and Groovy scripting languages using a JDBC-like compile, bind, run model. For Python, make sure the Jython jar is included in your class path. For JavaScript, make sure the Rhino jar is included in your classpath. For Groovy, make sure the groovy-all jar is included in your classpath.
Note that host languages and the languages of UDFs (included as part of the embedded Pig) are completely orthogonal. For example, a Pig Latin statement that registers a Python UDF may be embedded in Python, JavaScript, or Java. The exception to this rule is "combined" scripts – here the languages must match (see the Advanced Topics for Python, Advanced Topics for JavaScript and Advanced Topics for Groovy).

Invocation Basics

Embedded Pig is supported in batch mode only, not interactive mode. You can request that embedded Pig be used by adding the --embedded option to the Pig command line. If this option is passed as an argument, that argument will refer to the language Pig is embedded in, either Python, JavaScript or Groovy. If no argument is specified, it is taken to refer to the reference implementation for Python.
Python
 $ pig myembedded.py
 
Pig will look for the #!/usr/bin/python line in the script.
#!/usr/bin/python 

# explicitly import Pig class 
from org.apache.pig.scripting import Pig 

# COMPILE: compile method returns a Pig object that represents the pipeline
P = Pig.compile("a = load '$in'; store a into '$out';")

input = 'original'
output = 'output'

# BIND and RUN 
result = P.bind({'in':input, 'out':output}).runSingle()

if result.isSuccessful() :
    print 'Pig job succeeded'
else :
    raise 'Pig job failed'
 
JavaScript
$ pig myembedded.js
Pig will look for the *.js extension in the script.
importPackage(Packages.org.apache.pig.scripting.js) 

Pig = org.apache.pig.scripting.js.JSPig

function main() {
    input = "original"
    output = "output"

    P = Pig.compile("A = load '$in'; store A into '$out';") 

    result = P.bind({'in':input, 'out':output}).runSingle() 

    if (result.isSuccessful()) {
        print("Pig job succeeded")
    } else {
        print("Pig job failed")
    }   
}
Groovy
$ pig myembedded.groovy
Pig will look for the *.groovy extension in the script.
import org.apache.pig.scripting.Pig;

public static void main(String[] args) {
  String input = "original"
  String output = "output"

  Pig P = Pig.compile("A = load '\$in'; store A into '\$out';")  

  result = P.bind(['in':input, 'out':output]).runSingle()  

  if (result.isSuccessful()) {
    print("Pig job succeeded")
  } else {
    print("Pig job failed")
  }  
}
Invocation Process
You invoke Pig in the host scripting language through an embedded Pig object.
Compile: Compile is a static function on the Pig class and in its simplest form takes a fragment of Pig Latin that defines the pipeline as its input:
# COMPILE: complie method returns a Pig object that represents the pipeline
P = Pig.compile("""A = load '$in'; store A into '$out';""")
Compile returns an instance of Pig object. This object can have certain values undefined. For example, you may want to define a pipeline without yet specifying the location of the input to the pipeline. The parameter will be indicated by a dollar sign followed by a sequence of alpha-numeric or underscore characters. Values for these parameters must be provided later at the time bind() is called on the Pig object. To call run() on a Pig object without all parameters being bound is an error.
Bind: Resolve the parameters during the bind call.
input = "original”
output = "output”

# BIND: bind method binds the variables with the parameters in the pipeline and returns a BoundScript object
Q = P.bind({'in':input, 'out':output}) 
Please note that all parameters must be resolved during bind. Having unbound parameters while running your script is an error. Also note that even if your script is fully defined during compile, bind without parameters still must be called.
Run: Bind call returns an instance of BoundScript object that can be used to execute the pipeline. The simplest way to execute the pipeline is to call runSingle function. (However, as mentioned later, this works only if a single set of variables is bound to the parameters. Otherwise, if multiple set of variables are bound, an exception will be thrown if runSingle is called.)
result = Q.runSingle()
The function returns a PigStats object that tells you whether the run succeeded or failed. In case of success, additional run statistics are provided.
Embedded Python Example
A complete embedded example is shown below.
#!/usr/bin/python

# explicitly import Pig class
from org.apache.pig.scripting import Pig

# COMPILE: compile method returns a Pig object that represents the pipeline
P = Pig.compile("""A = load '$in'; store A into '$out';""")

input = "original”
output = "output”

# BIND: bind method binds the variables with the parameters in the pipeline and returns a BoundScript object
Q = P.bind({'in':input, 'out':output}) 

# In this case, only one set of variables is bound to the pipeline, runSingle method returns a PigStats object. 
# If multiple sets of variables are bound to the pipeline, run method instead must be called and it returns 
# a list of PigStats objects.
result = Q.runSingle()

# check the result
if result.isSuccessful():
    print "Pig job succeeded"
else:
    raise "Pig job failed"    


OR, SIMPLY DO THIS:


#!/usr/bin/python

# explicitly import Pig class
from org.apache.pig.scripting import Pig

in = "original”
out = "output”

# implicitly bind the parameters to the local variables 
result= Pig.compile("""A = load '$in'; store A into '$out';""").bind().runSingle() 

if result.isSuccessful():
    print "Pig job succeeded"
else:
    raise "Pig job failed"

Invocation Details

All three APIs (compile, bind, run) discussed in the previous section have several versions depending on what you are trying to do.

Compile

In its basic form, compile just takes a Pig Latin fragment that defines the pipeline as described in the previous section. Additionally, the pipeline can be given a name. This name is only used when the embedded script is invoked via the PigRunner Java API (as discussed later in this document).
 P = Pig.compile("P1", """A = load '$in'; store A into '$out';""")
In addition to providing Pig script via a string, you can store it in a file and pass the file to the compile call:
P = Pig.compileFromFile("myscript.pig")
You can also name a pipeline stored in the script:
P = Pig.compileFromFile("P2", "myscript.pig")

Bind

In its simplest form, bind takes no parameters. In this case, an implicit bind is performed; Pig internally constructs a map of parameters from the local variables specified by the user in the script.
Q = P.bind() 
Finally, you might want to run the same pipeline in parallel with a different set of parameters, for instance for multiple dates. In this case, bind function, needs to be passed a list of maps with each element of the list containing parameters for a single invocation. In the example below, the pipeline is run for the US, the UK, and Brazil.
P = Pig.compile("""A = load '$in';
                   B = filter A by user is not null;
                   ...
                   store Z into '$out';
                """)

Q = P.bind([{'in':'us_raw','out':'us_processed'},
        {'in':'uk_raw','out':'uk_processed'},
        {'in':'brazil_raw','out':'brazil_processed'}])

results = Q.run() # it blocks until all pipelines are completed

for i in [0, 1, 2]:
    result = results[i]
    ... # check result for each pipeline


Run

We have already seen that the simplest way to run a script is to call runSingle without any parameters. Additionally, a Java Properties object or a file containing a list of properties can be passed to this call. The properties are passed to Pig and a treated as any other properties passed from command line.
# In a jython script 

from java.util import Properties
... ...

props = Properties()
props.put(key1, val1)  
props.put(key2, val2) 
... ... 

Pig.compile(...).bind(...).runSingle(props)
A more general version of run allows to run one or more pipelines concurrently. In this case, a list of PigStats results is returned – one for each pipeline run. The example in the previous section shows how to make use of this call.
As the case with runSingle, a set of Java Properties or a property file can be passed to the call.

Passing Parameters to a Script

Inside your script, you can define parameters and then pass parameters from command line to your script. There are two ways to pass parameters to your script:

1. -param
Similar to regular Pig parameter substitution, you can define parameters using -param/–param_file on Pig's command line. This variable will be treated as one of the binding variables when binding the Pig Latin script. For example, you can invoke the below Python script using: pig –param loadfile=student.txt script.py.
#!/usr/bin/python
from org.apache.pig.scripting import Pig

P = Pig.compile("""A = load '$loadfile' as (name, age, gpa);
store A into 'output';""")

Q = P.bind()

result = Q.runSingle()

2. Command line arguments
Currently this feature is only available in Python and Groovy. You can pass command line arguments (the arguments after the script file name) to Python. These will become sys.argv in Python and will be passed as main's arguments in Groovy. For example: pig script.py student.txt. The corresponding script is:
#!/usr/bin/python
import sys
from org.apache.pig.scripting import Pig

P = Pig.compile("A = load '" + sys.argv[1] + "' as (name, age, gpa);" +
"store A into 'output';");

Q = P.bind()

result = Q.runSingle()
and in Groovy, pig script.groovy student.txt:
import org.apache.pig.scripting.Pig;

public static void main(String[] args) {

  P = Pig.compile("A = load '" + args[1] + "' as (name, age, gpa);" +
                      "store A into 'output';");

  Q = P.bind()

  result = Q.runSingle()
}

PigRunner API

Starting with Pig 0.8, some applications such as Oozie workflow invoke Pig using the PigRunner Java class rather than through the command line. For these applications, the PigRunner interface has been expanded to accommodate embedded Pig. PigRunner accepts Python and JavaScript scripts as input. These scripts can potentially contain multiple Pig pipelines; therefore, we need a way to return results for all of them.
To do this and to preserve backward compatibility PigStats and related objects were expanded as shown below:
  • PigStats is now an abstract class. (PigStats as it was before has become SimplePigStats.)
  • SimplePigStats is a new class that extends PigStats. SimplePigStats.getAllStats() will return null.
  • EmbeddedPigStats is a new class that extends PigStats. EmbeddedPigStats will return null for methods not listed in the proposal below.
  • isEmbedded() is a new abstract method that accommodates embedded Pig.
  • getAllStats() and List< > getAllErrorMessages() methods were added to the PigStats class. The map returned from getAllStats is keyed on the name of the pipeline provided in the compile call. If the name was not compiled an internally generated id would be used.
  • The PigProgressNotificationListener interface was modified to add script id to all its methods.
For more details, see Java Objects.

Usage Examples


Passing a Pig Script

This example shows you how to pass an entire Pig script to the compile call.
#!/usr/bin/python

from org.apache.pig.scripting import Pig

P = Pig.compileFromFile("""myscript.pig""")

input = "original"
output = "output"

result = p.bind({'in':input, 'out':output}).runSingle()
if result.isSuccessful():
    print "Pig job succeeded"
else:
    raise "Pig job failed" 

Convergence

There is a class of problems that involve iterating over a data pipeline an indeterminate number of times until a certain value is reached. Examples arise in machine learning, graph traversal, and a host of numerical analysis problems which involve finding interpolations, extrapolations or regressions. The Python example below shows one way to achieve convergence using Pig scripts.
#!/usr/bin/python

# explicitly import Pig class
from org.apache.pig.scripting import Pig

P = Pig.compile("""A = load '$input' as (user, age, gpa);
                   B = group A all;
                   C = foreach B generate AVG(A.gpa);
                   store C into '$output';
                """)
# initial output
input = "studenttab5"
output = "output-5"
final = "final-output"

for i in range(1, 4):
    Q = P.bind({'input':input, 'output':output}) # attaches $input, $output in Pig Latin to input, output Python variable
    results = Q.runSingle()

    if results.isSuccessful() == "FAILED":
        raise "Pig job failed"
    iter = results.result("C").iterator()
    if iter.hasNext():
        tuple = iter.next()
        value = tuple.get(0)
        if float(str(value)) < 3:
            print "value: " + str(value)
            input = "studenttab" + str(i+5)
            output = "output-" + str(i+5)
            print "output: " + output
        else:
           Pig.fs("mv " + output + " " + final)
           break

Automated Pig Latin Generation

A number of user frameworks do automated generation of Pig Latin.

Conditional Compilation
A sub-use case of automated generation is conditional code generation. Different processing might be required based on whether this is weekday or a weekend.
str = "A = load 'input';" 
if today.isWeekday():
    str = str + "B = filter A by weekday_filter(*);" 
else:
    str = str + "B = filter A by weekend_filter(*);" 
str = str + "C = group B by user;" 
results = Pig.compile(str).bind().runSingle()

Parallel Execution
Another sub-use case of automated generation is parallel execution of identical pipelines. You may have a single pipeline that you would like to run multiple data sets through in parallel. In the example below, the pipeline is run for the US, the UK, and Brazil.
P = Pig.compile("""A = load '$in';
                   B = filter A by user is not null;
                   ...
                   store Z into '$out';
                """)

Q = P.bind([{'in':'us_raw','out':'us_processed'},
        {'in':'uk_raw','out':'uk_processed'},
        {'in':'brazil_raw','out':'brazil_processed'}])

results = Q.run() # it blocks until all pipelines are completed

for i in [0, 1, 2]:
    result = results[i]
    ... # check result for each pipeline

Java Objects


Pig Object

public class Pig {    
    /**
     * Run a filesystem command.  Any output from this command is written to
     * stdout or stderr as appropriate.
     * @param cmd Filesystem command to run along with its arguments as one
     * string.
     * @throws IOException
     */
    public static void fs(String cmd) throws IOException {...}
    
    /**
     * Register a jar for use in Pig.  Once this is done this jar will be
     * registered for ALL SUBSEQUENT Pig pipelines in this script.  
     * If you wish to register it for only a single Pig pipeline, use 
     * register within that definition.
     * @param jarfile Path of jar to include.
     * @throws IOException if the indicated jarfile cannot be found.
     */
    public static void registerJar(String jarfile) throws IOException {...}
    
    /**
     * Register script UDFs for use in Pig. Once this is done all UDFs
     * defined in the file will be available for ALL SUBSEQUENT 
     * Pig pipelines in this script. If you wish to register UDFS for 
     * only a single Pig pipeline, use register within that definition.
     * @param udffile Path of the script UDF file
     * @param namespace namespace of the UDFs
     * @throws IOException
     */
    public static void registerUDF(String udffile, String namespace) throws IOException {...}
    
    /**
     * Define an alias for a UDF or a streaming command.  This definition
     * will then be present for ALL SUBSEQUENT Pig pipelines defined in this 
     * script.  If you wish to define it for only a single Pig pipeline, use
     * define within that definition.
     * @param alias name of the defined alias
     * @param definition string this alias is defined as
     */
    public static void define(String alias, String definition) throws IOException {...}

    /**
     * Set a variable for use in Pig Latin.  This set
     * will then be present for ALL SUBSEQUENT Pig pipelines defined in this 
     * script.  If you wish to set it for only a single Pig pipeline, use
     * set within that definition.
     * @param var variable to set
     * @param value to set it to
     */
    public static void set(String var, String value) throws IOException {...}
            
    /**
     * Define a Pig pipeline.  
     * @param pl Pig Latin definition of the pipeline.
     * @return Pig object representing this pipeline.
     * @throws IOException if the Pig Latin does not compile.
     */
    public static Pig compile(String pl) throws IOException {...}

    /**
     * Define a named portion of a Pig pipeline.  This allows it
     * to be imported into another pipeline.
     * @param name Name that will be used to define this pipeline.
     * The namespace is global.
     * @param pl Pig Latin definition of the pipeline.
     * @return Pig object representing this pipeline.
     * @throws IOException if the Pig Latin does not compile.
     */
    public static Pig compile(String name, String pl) throws IOException {...}

    /**
     * Define a Pig pipeline based on Pig Latin in a separate file.
     * @param filename File to read Pig Latin from.  This must be a purely 
     * Pig Latin file.  It cannot contain host language constructs in it.
     * @return Pig object representing this pipeline.
     * @throws IOException if the Pig Latin does not compile or the file
     * cannot be found.
     */
    public static Pig compileFromFile(String filename) throws IOException {...}

    /**
     * Define a named Pig pipeline based on Pig Latin in a separate file.
     * This allows it to be imported into another pipeline.
     * @param name Name that will be used to define this pipeline.
     * The namespace is global.
     * @param filename File to read Pig Latin from.  This must be a purely 
     * Pig Latin file.  It cannot contain host language constructs in it.
     * @return Pig object representing this pipeline.
     * @throws IOException if the Pig Latin does not compile or the file
     * cannot be found.
     */
    public static Pig compileFromFile(String name, String filename) throws IOException {...}
    
    /**
     * Bind this to a set of variables. Values must be provided
     * for all Pig Latin parameters.
     * @param vars map of variables to bind.  Keys should be parameters defined 
     * in the Pig Latin.  Values should be strings that provide values for those
     * parameters.  They can be either constants or variables from the host
     * language.  Host language variables must contain strings.
     * @return a {@link BoundScript} object 
     * @throws IOException if there is not a key for each
     * Pig Latin parameter or if they contain unsupported types.
     */
    public BoundScript bind(Map<String, String> vars) throws IOException {...}
        
    /**
     * Bind this to multiple sets of variables.  This will 
     * cause the Pig Latin script to be executed in parallel over these sets of 
     * variables.
     * @param vars list of maps of variables to bind.  Keys should be parameters defined 
     * in the Pig Latin.  Values should be strings that provide values for those
     * variables.  They can be either constants or variables from the host
     * language.  Host language variables must be strings.
     * @return a {@link BoundScript} object 
     * @throws IOException  if there is not a key for each
     * Pig Latin parameter or if they contain unsupported types.
     */
    public BoundScript bind(List<Map<String, String>> vars) throws IOException {...}

    /**
     * Bind a Pig object to variables in the host language (optional
     * operation).  This does an implicit mapping of variables in the host
     * language to parameters in Pig Latin.  For example, if the user
     * provides a Pig Latin statement
     * p = Pig.compile("A = load '$input';");
     * and then calls this function it will look for a variable called
     * input in the host language.  Scoping rules of the host
     * language will be followed in selecting which variable to bind.  The 
     * variable bound must contain a string value.  This method is optional
     * because not all host languages may support searching for in scope
     * variables.
     * @throws IOException if host language variables are not found to resolve all
     * Pig Latin parameters or if they contain unsupported types.
     */
    public BoundScript bind() throws IOException {...}

}

BoundScript Object

public class BoundScript {
    
    /**
     * Run a pipeline on Hadoop.  
     * If there are no stores in this pipeline then nothing will be run. 
     * @return {@link PigStats}, null if there is no bound query to run.
     * @throws IOException
     */
    public PigStats runSingle() throws IOException {...}
     
    /**
     * Run a pipeline on Hadoop.  
     * If there are no stores in this pipeline then nothing will be run.  
     * @param prop Map of properties that Pig should set when running the script.
     * This is intended for use with scripting languages that do not support
     * the Properties object.
     * @return {@link PigStats}, null if there is no bound query to run.
     * @throws IOException
     */
    public PigStats runSingle(Properties prop) throws IOException {...}
    
    /**
     * Run a pipeline on Hadoop.  
     * If there are no stores in this pipeline then nothing will be run.  
     * @param propfile File with properties that Pig should set when running the script.
     * @return {@link PigStats}, null if there is no bound query to run.
     * @throws IOException
     */
    public PigStats runSingle(String propfile) throws IOException {...}

    /**
     * Run multiple instances of bound pipeline on Hadoop in parallel.  
     * If there are no stores in this pipeline then nothing will be run.  
     * Bind is called first with the list of maps of variables to bind. 
     * @return a list of {@link PigStats}, one for each map of variables passed
     * to bind.
     * @throws IOException
     */    
    public List<PigStats> run() throws IOException {...}
    
    /**
     * Run multiple instances of bound pipeline on Hadoop in parallel.
     * @param prop Map of properties that Pig should set when running the script.
     * This is intended for use with scripting languages that do not support
     * the Properties object.
     * @return a list of {@link PigStats}, one for each map of variables passed
     * to bind.
     * @throws IOException
     */
    public List<PigStats>  run(Properties prop) throws IOException {...}
    
    /**
     * Run multiple instances of bound pipeline on Hadoop in parallel.
     * @param propfile File with properties that Pig should set when running the script.
     * @return a list of PigResults, one for each map of variables passed
     * to bind.
     * @throws IOException
     */
    public List<PigStats>  run(String propfile) throws IOException {...}

    /**
     * Run illustrate for this pipeline.  Results will be printed to stdout.  
     * @throws IOException if illustrate fails.
     */
    public void illustrate() throws IOException {...}

    /**
     * Explain this pipeline.  Results will be printed to stdout.
     * @throws IOException if explain fails.
     */
    public void explain() throws IOException {...}

    /**
     * Describe the schema of an alias in this pipeline.
     * Results will be printed to stdout.
     * @param alias to be described
     * @throws IOException if describe fails.
     */
    public void describe(String alias) throws IOException {...}

}

PigStats Object

public abstract class PigStats {
    public abstract boolean isEmbedded();
    
    /**
     * An embedded script contains one or more pipelines. 
     * For a named pipeline in the script, the key in the returning map is the name of the pipeline. 
     * Otherwise, the key in the returning map is the script id of the pipeline.
     */
    public abstract Map<String, List<PigStats>> getAllStats();
    
    public abstract List<String> getAllErrorMessages();      
}

PigProgressNotificationListener Object

public interface PigProgressNotificationListener extends java.util.EventListener {

    /** 
     * Invoked just before launching MR jobs spawned by the script.
     * @param scriptId id of the script
     * @param numJobsToLaunch the total number of MR jobs spawned by the script
     */
    public void launchStartedNotification(String scriptId, int numJobsToLaunch);
    
    /**
     * Invoked just before submitting a batch of MR jobs.
     * @param scriptId id of the script
     * @param numJobsSubmitted the number of MR jobs in the batch
     */
    public void jobsSubmittedNotification(String scriptId, int numJobsSubmitted);
    
    /**
     * Invoked after a MR job is started.
     * @param scriptId id of the script 
     * @param assignedJobId the MR job id
     */
    public void jobStartedNotification(String scriptId, String assignedJobId);
    
    /**
     * Invoked just after a MR job is completed successfully. 
     * @param scriptId id of the script 
     * @param jobStats the {@link JobStats} object associated with the MR job
     */
    public void jobFinishedNotification(String scriptId, JobStats jobStats);
    
    /**
     * Invoked when a MR job fails.
     * @param scriptId id of the script 
     * @param jobStats the {@link JobStats} object associated with the MR job
     */
    public void jobFailedNotification(String scriptId, JobStats jobStats);
    
    /**
     * Invoked just after an output is successfully written.
     * @param scriptId id of the script
     * @param outputStats the {@link OutputStats} object associated with the output
     */
    public void outputCompletedNotification(String scriptId, OutputStats outputStats);
    
    /**
     * Invoked to update the execution progress. 
     * @param scriptId id of the script
     * @param progress the percentage of the execution progress
     */
    public void progressUpdatedNotification(String scriptId, int progress);
    
    /**
     * Invoked just after all MR jobs spawned by the script are completed.
     * @param scriptId id of the script
     * @param numJobsSucceeded the total number of MR jobs succeeded
     */
    public void launchCompletedNotification(String scriptId, int numJobsSucceeded);
}

Embedded Pig - Java

To enable control flow, you can embed Pig Latin statements and Pig commands in the Java programming language.
Note that host languages and the languages of UDFs (included as part of the embedded Pig) are completely orthogonal. For example, a Pig Latin statement that registers a Java UDF may be embedded in Python, JavaScript, Groovy, or Java. The exception to this rule is "combined" scripts – here the languages must match (see the Advanced Topics for Python, Advanced Topics for JavaScript and Advanced Topics for Groovy).

PigServer Interface

Currently, PigServer is the main interface for embedding Pig in Java. PigServer can now be instantiated from multiple threads. (In the past, PigServer contained references to static data that prevented multiple instances of the object to be created from different threads within your application.) Please note that PigServer is NOT thread safe; the same object can't be shared across multiple threads.

Usage Examples

Local Mode
From your current working directory, compile the program. (Note that idlocal.class is written to your current working directory. Include “.” in the class path when you run the program.)
$ javac -cp pig.jar idlocal.java
From your current working directory, run the program. To view the results, check the output file, id.out.
Unix:    $ java -cp pig.jar:. idlocal
Windows: $ java –cp .;pig.jar idlocal
idlocal.java - The sample code is based on Pig Latin statements that extract all user IDs from the /etc/passwd file. Copy the /etc/passwd file to your local working directory.
import java.io.IOException;
import org.apache.pig.PigServer;
public class idlocal{ 
    public static void main(String[] args) {
        try {
            PigServer pigServer = new PigServer("local");
            runIdQuery(pigServer, "passwd");
        }
        catch(Exception e) {
        }
    }
    public static void runIdQuery(PigServer pigServer, String inputFile) throws IOException {
        pigServer.registerQuery("A = load '" + inputFile + "' using PigStorage(':');");
        pigServer.registerQuery("B = foreach A generate $0 as id;");
        pigServer.store("B", "id.out");
    }
}
Mapreduce Mode
Point $HADOOPDIR to the directory that contains the hadoop-site.xml file. Example:
$ export HADOOPDIR=/yourHADOOPsite/conf 
From your current working directory, compile the program. (Note that idmapreduce.class is written to your current working directory. Include “.” in the class path when you run the program.)
$ javac -cp pig.jar idmapreduce.java
From your current working directory, run the program. To view the results, check the idout directory on your Hadoop system.
Unix:   $ java -cp pig.jar:.:$HADOOPDIR idmapreduce
Cygwin: $ java –cp '.;pig.jar;$HADOOPDIR' idmapreduce
idmapreduce.java - The sample code is based on Pig Latin statements that extract all user IDs from the /etc/passwd file. Copy the /etc/passwd file to your home directory on the HDFS.
import java.io.IOException;
import org.apache.pig.PigServer;
public class idmapreduce{
    public static void main(String[] args) {
        try {
            PigServer pigServer = new PigServer("mapreduce");
            runIdQuery(pigServer, "passwd");
        }
        catch(Exception e) {
        }
    }
    public static void runIdQuery(PigServer pigServer, String inputFile) throws IOException {
        pigServer.registerQuery("A = load '" + inputFile + "' using PigStorage(':');")
        pigServer.registerQuery("B = foreach A generate $0 as id;");
        pigServer.store("B", "idout");
    }
}

Pig Macros

Pig Latin supports the definition, expansion, and import of macros.

DEFINE (macros)

Defines a Pig macro.

Syntax

Define Macro
DEFINE macro_name (param [, param ...]) RETURNS {void | alias [, alias ...]} { pig_latin_fragment };

Expand Macro
alias [, alias ...] = macro_name (param [, param ...]) ;

Terms

macro_name The name of the macro. Macro names are global.
param (optional) A comma-separated list of one or more parameters, including IN aliases (Pig relations), enclosed in parentheses, that are referenced in the Pig Latin fragment.
Unlike user defined functions (UDFs), which only allow quoted strings as its parameters, Pig macros support four types of parameters:
  • alias (IDENTIFIER)
  • integer
  • float
  • string literal (quoted string)
Note that type is NOT part of parameter definition. It is your responsibility to document the types of the parameters in a macro.
void If the macro has no return alias, then void must be specified.
alias (optional) A comma-separated list of one or more return aliases (Pig relations) that are referenced in the Pig Latin fragment. The alias must exist in the macro in the form $<alias>.
If the macro has no return alias, then void must be specified.
pig_latin_fragment One or more Pig Latin statements, enclosed in curly brackets.

Usage

Macro Definition
A macro definition can appear anywhere in a Pig script as long as it appears prior to the first use. A macro definition can include references to other macros as long as the referenced macros are defined prior to the macro definition. Recursive references are not allowed.
Note the following restrictions:
  • Macros are not allowed inside a FOREACH nested block.
  • Macros cannot contain Grunt shell commands.
  • Macros cannot include a user-defined schema that has a name collision with an alias in the macro.
In this example the macro is named my_macro. Note that only aliases A and C are visible from the outside; alias B is not visible from the outside.
 DEFINE my_macro(A, sortkey) RETURNS C {
    B = FILTER $A BY my_filter(*);
    $C = ORDER B BY $sortkey;
}
Macro Expansion
A macro can be expanded inline using the macro expansion syntax. Note the following:
  • Any alias in the macro which isn't visible from the outside will be prefixed with a macro name and suffixed with an instance id to avoid namespace collision.
  • Macro expansion is not a complete replacement for function calls. Recursive expansions are not supported.
In this example my_macro (defined above) is expanded. Because alias B is not visible from the outside it is renamed macro_my_macro_B_0.
/* These statements ... */

X = LOAD 'users' AS (user, address, phone);
Y = my_macro(X, user);
STORE Y into 'bar';

/* Are expanded into these statements ... */

X = LOAD 'users' AS (user, address, phone);
macro_my_macro_B_0 = FILTER X BY my_filter(*);
Y = ORDER macro_my_macro_B_0  BY user;
STORE Y INTO 'output';
Macro Import
A macro can be imported from another Pig script (see IMPORT (macros)). Splitting your macros from your main Pig script is useful for making reusable code.

Examples

In this example no parameters are passed to the macro.
DEFINE my_macro() returns B {
   D = LOAD 'data' AS (a0:int, a1:int, a2:int);   
   $B = FILTER D BY ($1 == 8) OR (NOT ($0+$2 > $1));
};

X = my_macro();
STORE X INTO 'output';
In this example parameters are passed and returned.
DEFINE group_and_count (A, group_key, reducers) RETURNS B {
   D = GROUP $A BY $group_key PARALLEL $reducers;
   $B = FOREACH D GENERATE group, COUNT($A);
};

X = LOAD 'users' AS (user, age, zip);
Y = group_and_count (X, user, 20);
Z = group_and_count (X, age, 30);
STORE Y into 'byuser';
STORE Z into 'byage';
In this example the macro does not have a return alias; thus, void must be specified.
DEFINE my_macro(A, sortkey) RETURNS void {     
      B = FILTER $A BY my_filter(*);     
      C = ORDER B BY $sortkey;
      STORE C INTO 'my_output';  
};

/* To expand this macro, use the following */

my_macro(alpha, 'user');
In this example a name collision will occur. Here letter B is used as alias name and as name in user-defined schema. Pig will throw an exception when name collision is detected.
DEFINE my_macro(A, sortkey) RETURNS E {     
      B = FILTER $A BY my_filter(*);     
      C = ORDER B BY $sortkey;
      D = LOAD 'in' as (B:bag{});
   $E = FOREACH D GENERATE COUNT(B); 
   };
This example demonstrates the importance of knowing parameter types before using them in a macro script. Notice that when pass parameter $outfile to my_macro1 inside my_macro2, it must be quoted.
-- A: an alias
-- outfile: output file path (quoted string)
DEFINE my_macro1(A, outfile) RETURNS void {     
       STORE $A INTO '$outfile'; 
   };

-- A: an alias
-- sortkey: column name (quoted string)
-- outfile: output file path (quoted string)
DEFINE my_macro2(A, sortkey, outfile) RETURNS void {     
      B = FILTER $A BY my_filter(*);     
      C = ORDER B BY $sortkey;
      my_macro1(C, '$outfile');
   };

   alpha = Load 'input' as (user, age, gpa);
   my_macro2(alpha, 'age', 'order_by_age.txt');
In this example a macro (group_with_parallel) refers to another macro (foreach_count).
DEFINE foreach_count(A, C) RETURNS B {
   $B = FOREACH $A GENERATE group, COUNT($C);
};

DEFINE group_with_parallel (A, group_key, reducers) RETURNS B {
   C = GROUP $A BY $group_key PARALLEL $reducers;
   $B = foreach_count(C, $A);
};
       
/* These statements ... */
 
X = LOAD 'users' AS (user, age, zip);
Y = group_with_parallel (X, user, 23);
STORE Y INTO 'byuser';

/* Are expanded into these statements ... */

X = LOAD 'users' AS (user, age, zip);
macro_group_with_parallel_C_0 = GROUP X by (user) PARALLEL 23;
Y = FOREACH macro_group_with_parallel_C_0 GENERATE group, COUNT(X);
STORE Y INTO 'byuser';

IMPORT (macros)

Import macros defined in a separate file.

Syntax

IMPORT 'file-with-macro';

Terms

file-with-macro The name of a file (enclosed in single quotes) that contains one or more macro definitions; for example, 'my_macro.pig' or 'mypath/my_macro.pig'.
Macro names are global and all macros share the same name space. While the file can contain more than one macro definition, having two macros with the same name in your execution context will result in an error.
Files are imported based on either (1) the given file path or (2) the import path specified via the Pig property pig.import.search.path. If a file path is given, whether absolute or relative to the current directory (starting with . or ..), the import path will be ignored.

Usage

Use the IMPORT command to import a macro defined in a separate file into your Pig script.
IMPORT adds the macro definitions to the Pig Latin namespace; these macros can then be invoked as if they were defined in the same file.
Macros can only contain Pig Latin statements; Grunt shell commands are not supported. REGISTER statements and parameter definitions with %default or %declare are both valid however. Your macro file also IMPORT other macro files, so long as these imports are not recursive.
See also: DEFINE (macros)

Example

In this example, because a path is not given, Pig will use the import path specified in pig.import.search.path.
/* myscript.pig */
...
...
IMPORT 'my_macro.pig';
...
...

Parameter Substitution


Description

Substitute values for parameters at run time.

Syntax: Specifying Parameters Using the Pig Command Line

pig {-param param_name = param_value | -param_file file_name} [-debug | -dryrun] script

Syntax: Specifying Parameters Using Preprocessor Statements in a Pig Script

{%declare | %default} param_name param_value

Terms

pig Keyword
Note: exec, run, and explain also support parameter substitution.
-param Flag. Use this option when the parameter is included in the command line.
Multiple parameters can be specified. If the same parameter is specified multiple times, the last value will be used and a warning will be generated.
Command line parameters and parameter files can be combined with command line parameters taking precedence.
param_name The name of the parameter.
The parameter name has the structure of a standard language identifier: it must start with a letter or underscore followed by any number of letters, digits, and underscores.
Parameter names are case insensitive.
If you pass a parameter to a script that the script does not use, this parameter is silently ignored. If the script has a parameter and no value is supplied or substituted, an error will result.
param_value The value of the parameter.
A parameter value can take two forms:
  • A sequence of characters enclosed in single or double quotes. In this case the unquoted version of the value is used during substitution. Quotes within the value can be escaped with the backslash character ( \ ). Single word values that don't use special characters such as % or = don't have to be quoted.
  • A command enclosed in back ticks.
The value of a parameter, in either form, can be expressed in terms of other parameters as long as the values of the dependent parameters are already defined.
There are no hard limits on the size except that parameters need to fit into memory.
-param_file Flag. Use this option when the parameter is included in a file.
Multiple files can be specified. If the same parameter is present multiple times in the file, the last value will be used and a warning will be generated. If a parameter present in multiple files, the value from the last file will be used and a warning will be generated.
Command line parameters and parameter files can be combined with command line parameters taking precedence.
file_name The name of a file containing one or more parameters.
A parameter file will contain one line per parameter. Empty lines are allowed. Perl-style (#) comment lines are also allowed. Comments must take a full line and # must be the first character on the line. Each parameter line will be of the form: param_name = param_value. White spaces around = are allowed but are optional.
-debug Flag. With this option, the script is run and a fully substituted Pig script is produced in the current working directory named original_script_name.substituted
-dryrun Flag. With this option, the script is not run and a fully substituted Pig script is produced in the current working directory named original_script_name.substituted
script A pig script. The pig script must be the last element in the Pig command line.
  • If parameters are specified in the Pig command line or in a parameter file, the script should include a $param_name for each para_name included in the command line or parameter file.
  • If parameters are specified using the preprocessor statements, the script should include either %declare or %default.
  • In the script, parameter names can be escaped with the backslash character ( \ ) in which case substitution does not take place.
%declare Preprocessor statement included in a Pig script.
Use to describe one parameter in terms of other parameters.
The declare statement is processed prior to running the Pig script.
The scope of a parameter value defined using declare is all the lines following the declare statement until the next declare statement that defines the same parameter is encountered. When used with run/exec command, see Scope section.
%default Preprocessor statement included in a Pig script.
Use to provide a default value for a parameter. The default value has the lowest priority and is used if a parameter value has not been defined by other means.
The default statement is processed prior to running the Pig script.
The scope is the same as for %declare.

Usage

Parameter substitution enables you to write Pig scripts that include parameters and to supply values for these parameters at run time. For instance, suppose you have a job that needs to run every day using the current day's data. You can create a Pig script that includes a parameter for the date. Then, when you run this script you can specify or supply a value for the date parameter using one of the supported methods.

Specifying Parameters

You can specify parameter names and parameter values as follows:
  • As part of a command line.
  • In parameter file, as part of a command line.
  • With the declare statement, as part of Pig script.
  • With default statement, as part of a Pig script.
Parameter substitution may be used inside of macros. When there are conflicts between names of parameters defined at the top level and names of arguments or return values for a given macro, then ones inside the macro are used. See DEFINE (macros).

Precedence

Precedence for parameters is as follows, from highest to lowest:
  1. Parameters defined using the declare statement
  2. Parameters defined in the command line using -param
  3. Parameters defined in parameter files specified by -param_file
  4. Parameters defined using the default statement

Processing Order and Precedence

Parameters are processed as follows:
  • Command line parameters are scanned in the order they are specified on the command line.
  • Parameter files are scanned in the order they are specified on the command line. Within each file, the parameters are processed in the order they are listed.
  • Declare and default preprocessors statements are processed in the order they appear in the Pig script.

Scope

Scope of the parameters is global except when used with run/exec command. Caller would not see the parameters declared within the callee's scripts. See example for more details.

Examples


Specifying Parameters in the Command Line

Suppose we have a data file called 'mydata' and a pig script called 'myscript.pig'.
mydata
1       2       3
4       2       1
8       3       4
myscript.pig
A = LOAD '$data' USING PigStorage() AS (f1:int, f2:int, f3:int);
DUMP A;
In this example the parameter (data) and the parameter value (mydata) are specified in the command line. If the parameter name in the command line (data) and the parameter name in the script ($data) do not match, the script will not run. If the value for the parameter (mydata) is not found, an error is generated.
$ pig -param data=mydata myscript.pig

(1,2,3)
(4,2,1)
(8,3,4)

Specifying parameters Using a Parameter File

Suppose we have a parameter file called 'myparams.'
# my parameters
data1 = mydata1
cmd = `generate_name`
In this example the parameters and values are passed to the script using the parameter file.
$ pig -param_file myparams script2.pig

Specifying Parameters Using the Declare Statement

In this example the command is executed and its stdout is used as the parameter value.
%declare CMD `generate_date`;
A = LOAD '/data/mydata/$CMD';
B = FILTER A BY $0>'5';

etc ... 


Specifying Parameters Using the Default Statement

In this example the parameter (DATE) and value ('20090101') are specified in the Pig script using the default statement. If a value for DATE is not specified elsewhere, the default value 20090101 is used.
%default DATE '20090101';
A = load '/data/mydata/$DATE';

etc ... 


Specifying Parameter Values as a sequence of Characters

In this example the characters (in this case, Joe's URL) can be enclosed in single or double quotes, and quotes within the sequence of characters can be escaped.
%declare DES 'Joe\'s URL';
A = LOAD 'data' AS (name, description, url);
B = FILTER A BY description == '$DES';
 
etc ... 

In this example single word values that don't use special characters (in this case, mydata) don't have to be enclosed in quotes.
$ pig -param data=mydata myscript.pig

Specifying Parameter Values as a Command

In this example the command is enclosed in back ticks. First, the parameters mycmd and date are substituted when the declare statement is encountered. Then the resulting command is executed and its stdout is placed in the path before the load statement is run.
%declare CMD `$mycmd $date`;
A = LOAD '/data/mydata/$CMD';
B = FILTER A BY $0>'5';
 
etc ... 


Scoping with run/exec commands

In this example, parameters passed to run/exec command or declared within the called scripts are not visible to the caller.
/* main.pig */
run -param var1=10 script1.pig
exec script2.pig

A = ...
B = FOREACH A generate $var1, $var2, ...  --ERROR. unknown parameters var1, var2

/* script1.pig */
...
/* script2.pig */
%declare var2 20
...

Shell Commands


fs

Invokes any FsShell command from within a Pig script or the Grunt shell.

Syntax

fs subcommand subcommand_parameters

Terms

subcommand The FsShell command.
subcommand_parameters The FsShell command parameters.

Usage

Use the fs command to invoke any FsShell command from within a Pig script or Grunt shell. The fs command greatly extends the set of supported file system commands and the capabilities supported for existing commands such as ls that will now support globing. For a complete list of FsShell commands, see File System Shell Guide

Examples

In these examples a directory is created, a file is copied, a file is listed.
fs -mkdir /tmp
fs -copyFromLocal file-x file-y
fs -ls file-y

sh

Invokes any sh shell command from within a Pig script or the Grunt shell.

Syntax

sh subcommand subcommand_parameters

Terms

subcommand The sh shell command.
subcommand_parameters The sh shell command parameters.

Usage

Use the sh command to invoke any sh shell command from within a Pig script or Grunt shell.
Note that only real programs can be run form the sh command. Commands such as cd are not programs but part of the shell environment and as such cannot be executed unless the user invokes the shell explicitly, like "bash cd".

Example

In this example the ls command is invoked.
grunt> sh ls 
bigdata.conf 
nightly.conf 
..... 
grunt> 

Utility Commands


clear

Clear the screen of Pig grunt shell and position the cursor at top of the screen.

Syntax

clear

Terms

key Description.
none no parameters

Example

In this example the clear command clean up Pig grunt shell.
grunt> clear

exec

Run a Pig script.

Syntax

exec [–param param_name = param_value] [–param_file file_name] [script] 

Terms

–param param_name = param_value See Parameter Substitution.
–param_file file_name See Parameter Substitution.
script The name of a Pig script.

Usage

Use the exec command to run a Pig script with no interaction between the script and the Grunt shell (batch mode). Aliases defined in the script are not available to the shell; however, the files produced as the output of the script and stored on the system are visible after the script is run. Aliases defined via the shell are not available to the script.
With the exec command, store statements will not trigger execution; rather, the entire script is parsed before execution starts. Unlike the run command, exec does not change the command history or remembers the handles used inside the script. Exec without any parameters can be used in scripts to force execution up to the point in the script where the exec occurs.

For comparison, see the run command. Both the exec and run commands are useful for debugging because you can modify a Pig script in an editor and then rerun the script in the Grunt shell without leaving the shell. Also, both commands promote Pig script modularity as they allow you to reuse existing components.

Examples

In this example the script is displayed and run.
grunt> cat myscript.pig
a = LOAD 'student' AS (name, age, gpa);
b = LIMIT a 3;
DUMP b;

grunt> exec myscript.pig
(alice,20,2.47)
(luke,18,4.00)
(holly,24,3.27)
In this example parameter substitution is used with the exec command.
grunt> cat myscript.pig
a = LOAD 'student' AS (name, age, gpa);
b = ORDER a BY name;

STORE b into '$out';

grunt> exec –param out=myoutput myscript.pig
In this example multiple parameters are specified.
grunt> exec –param p1=myparam1 –param p2=myparam2 myscript.pig

help

Prints a list of Pig commands or properties.

Syntax

-help [properties]

Terms

properties List Pig properties.

Usage

The help command prints a list of Pig commands or properties.

Example

Use "-help" to get a list of commands.
$ pig -help

Apache Pig version 0.8.0-dev (r987348)
compiled Aug 19 2010, 16:38:44

USAGE: Pig [options] [-] : Run interactively in grunt shell.
       Pig [options] -e[xecute] cmd [cmd ...] : Run cmd(s).
       Pig [options] [-f[ile]] file : Run cmds found in file.
  options include:
    -4, -log4jconf - Log4j configuration file, overrides log conf
    -b, -brief - Brief logging (no timestamps)
    -c, -check - Syntax check
etc …
Use "-help properties" to get a list of properties.
$ pig -help properties

The following properties are supported:
    Logging:
        verbose=true|false; default is false. This property is the same as -v switch
        brief=true|false; default is false. This property is the same as -b switch
        debug=OFF|ERROR|WARN|INFO|DEBUG; default is INFO. This property is the same as -d switch
        aggregate.warning=true|false; default is true. If true, prints count of warnings
            of each type rather than logging each warning.
etc …

history

Display the list of statements used so far.

Syntax

history [-n]

Terms

key Description.
-n Omit line numbers in the list.

Usage

The history command shows the statements used so far.

Example

In this example the history command shows all the statements with line numbers and without them.
grunt> a = LOAD 'student' AS (name, age, gpa);
grunt> b = order a by name;
grunt> history
1 a = LOAD 'student' AS (name, age, gpa);
2 b = order a by name;

grunt> c = order a by name;
grunt> history -n
a = LOAD 'student' AS (name, age, gpa);
b = order a by name;
c = order a by name;

kill

Kills a job.

Syntax

kill jobid

Terms

jobid The job id.

Usage

Use the kill command to kill a Pig job based on the job id.
The kill command will attempt to kill any MapReduce jobs associated with the Pig job. Under certain conditions, however, this may fail; for example, when a Pig job is killed and does not have a chance to call its shutdown procedures.

Example

In this example the job with id job_0001 is killed.
grunt> kill job_0001

quit

Quits from the Pig grunt shell.

Syntax

exit

Terms

none no parameters

Usage

The quit command enables you to quit or exit the Pig grunt shell.

Example

In this example the quit command exits the Pig grunt shall.
grunt> quit

run

Run a Pig script.

Syntax

run [–param param_name = param_value] [–param_file file_name] script

Terms

–param param_name = param_value See Parameter Substitution.
–param_file file_name See Parameter Substitution.
script The name of a Pig script.

Usage

Use the run command to run a Pig script that can interact with the Grunt shell (interactive mode). The script has access to aliases defined externally via the Grunt shell. The Grunt shell has access to aliases defined within the script. All commands from the script are visible in the command history.
With the run command, every store triggers execution. The statements from the script are put into the command history and all the aliases defined in the script can be referenced in subsequent statements after the run command has completed. Issuing a run command on the grunt command line has basically the same effect as typing the statements manually.
For comparison, see the exec command. Both the run and exec commands are useful for debugging because you can modify a Pig script in an editor and then rerun the script in the Grunt shell without leaving the shell. Also, both commands promote Pig script modularity as they allow you to reuse existing components.

Example

In this example the script interacts with the results of commands issued via the Grunt shell.
grunt> cat myscript.pig
b = ORDER a BY name;
c = LIMIT b 10;

grunt> a = LOAD 'student' AS (name, age, gpa);

grunt> run myscript.pig

grunt> d = LIMIT c 3;

grunt> DUMP d;
(alice,20,2.47)
(alice,27,1.95)
(alice,36,2.27)
In this example parameter substitution is used with the run command.
grunt> a = LOAD 'student' AS (name, age, gpa);

grunt> cat myscript.pig
b = ORDER a BY name;
STORE b into '$out';

grunt> run –param out=myoutput myscript.pig

set

Shows/Assigns values to keys used in Pig.

Syntax

set [key 'value']

Terms

key Key (see table). Case sensitive.
value Value for key (see table). Case sensitive.

Usage

Use the set command to assign values to keys, as shown in the table. All keys and their corresponding values (for Pig and Hadoop) are case sensitive. If set command is used without key/value pair argument, Pig prints all the configurations and system properties.
Key Value Description
default_parallel a whole number Sets the number of reducers for all MapReduce jobs generated by Pig (see Use the Parallel Features).
debug on/off Turns debug-level logging on or off.
job.name Single-quoted string that contains the job name. Sets user-specified name for the job
job.priority Acceptable values (case insensitive): very_low, low, normal, high, very_high Sets the priority of a Pig job.
stream.skippath String that contains the path. For streaming, sets the path from which not to ship data (see DEFINE (UDFs, streaming) and About Auto-Ship).
All Pig and Hadoop properties can be set, either in the Pig script or via the Grunt command line.

Examples

In this example key value pairs are set at the command line.
grunt> SET debug 'on'
grunt> SET job.name 'my job'
grunt> SET default_parallel 100
In this example default_parallel is set in the Pig script; all MapReduce jobs that get launched will use 20 reducers.
SET default_parallel 20;
A = LOAD 'myfile.txt' USING PigStorage() AS (t, u, v);
B = GROUP A BY t;
C = FOREACH B GENERATE group, COUNT(A.t) as mycount;
D = ORDER C BY mycount;
STORE D INTO 'mysortedcount' USING PigStorage();
In this example multiple key value pairs are set in the Pig script. These key value pairs are put in job-conf by Pig (making the pairs available to Pig and Hadoop). This is a script-wide setting; if a key value is defined multiple times in the script the last value will take effect and will be set for all jobs generated by the script.
...
SET mapred.map.tasks.speculative.execution false; 
SET pig.logfile mylogfile.log; 
SET my.arbitrary.key my.arbitary.value; 

Tez mode

Apache Tez provides an alternative execution engine than MapReduce focusing on performance. By using optimized job flow, edge semantics and container reuse, we see consistent performance boost for both large job and small job.

How to enable Tez

To run Pig in tez mode, simply add "-x tez" in pig command line. Alternatively, you can add "exectype=tez" to conf/pig.properties to change the default exec type to Tez. Java system property "-Dexectype=tez" is also good to trigger the Tez mode.
Prerequisite: Tez requires the tez tarball to be available in hdfs while running a job on the cluster and a tez-site.xml with tez.lib.uris setting pointing to that hdfs location in classpath. Copy the tez tarball to hdfs and add the tez conf directory($TEZ_HOME/conf) containing tez-site.xml to environmental variable "PIG_CLASSPATH" if pig on tez fails with "tez.lib.uris is not defined". This is required by the Apache Pig distribution.
  <property>
    <name>tez.lib.uris</name>
    <value>${fs.defaultFS}/apps/tez/tez-0.5.2.tar.gz</value>
  </property>

Tez DAG generation

Every Pig script will be compiled into 1 or more Tez DAG (typically 1). Every Tez DAG consists of a number of vertices and and edges connecting vertices. For example, a simple join involves 1 DAG which consists of 3 vertices: load left input, load right input and join. Do an explain in Tez mode will show you the DAG Pig script compiled into.

Tez session/container reuse

One downside of MapReduce is the startup cost for a job is very high. That hurts the performance especially for small job. Tez alleviate the problem by using session and container reuse, so it is not necessary to start an application master for every job, and start a JVM for every task. By default, session/container reuse is on and we usually shall not turn it off. JVM reuse might cause some side effect if static variable is used since static variable might live across different jobs. So if static variable is used in EvalFunc/LoadFunc/StoreFunc, be sure to implement a cleanup function and register with JVMReuseManager.

Automatic parallelism

Just like MapReduce, if user specify "parallel" in their Pig statement, or user define default_parallel in Tez mode, Pig will honor it (the only exception is if user specify a parallel which is apparently too low, Pig will override it)
If user specify neither "parallel" or "default_parallel", Pig will use automatic parallelism. In MapReduce, Pig submit one MapReduce job a time and before submiting a job, Pig has chance to automatically set reduce parallelism based on the size of input file. On the contrary, Tez submit a DAG as a unit and automatic parallelism is managed in three parts
  • Before submiting a DAG, Pig estimate parallelism of each vertex statically based on the input file size of the DAG and the complexity of the pipeline of each vertex
  • When DAG progress, Pig adjust the parallelism of vertexes with the best knowledge available at that moment (Pig grace paralellism)
  • At runtime, Tez adjust vertex parallelism dynamically based on the input data volume of the vertex. Note currently Tez can only decrease the parallelism dynamically not increase. So in step 1 and 2, Pig overestimate the parallelism
The following parameter control the behavior of automatic parallelism in Tez (share with MapReduce):
pig.exec.reducers.bytes.per.reducer
pig.exec.reducers.max

API change

If invoking Pig in Java, there is change in PigStats and PigProgressNotificationListener if using PigRunner.run(), check Pig Statistics and Pig Progress Notification Listener

Known issues

Currently known issue in Tez mode includes:
  • Tez local mode is not stable, we see job hang in some cases
  • Tez specific GUI is not available yet, there is no GUI to track task progress. However, log message is available in GUI

Timing your UDFs

The first step to improving performance and efficiency is measuring where the time is going. Pig provides a light-weight method for approximately measuring how much time is spent in different user-defined functions (UDFs) and Loaders. Simply set the pig.udf.profile property to true. This will cause new counters to be tracked for all Map-Reduce jobs generated by your script: approx_microsecs measures the approximate amount of time spent in a UDF, and approx_invocations measures the approximate number of times the UDF was invoked. In addition, the frequency of profiling can be configured via the pig.udf.profile.frequency (by default, every 100th invocation). Note that this may produce a large number of counters (two per UDF). Excessive amounts of counters can lead to poor JobTracker performance, so use this feature carefully, and preferably on a test cluster.

Combiner

The Pig combiner is an optimizer that is invoked when the statements in your scripts are arranged in certain ways. The examples below demonstrate when the combiner is used and not used. Whenever possible, make sure the combiner is used as it frequently yields an order of magnitude improvement in performance.

When the Combiner is Used

The combiner is generally used in the case of non-nested foreach where all projections are either expressions on the group column or expressions on algebraic UDFs (see Make Your UDFs Algebraic).
Example:
A = load 'studenttab10k' as (name, age, gpa);
B = group A by age;
C = foreach B generate ABS(SUM(A.gpa)), COUNT(org.apache.pig.builtin.Distinct(A.name)), (MIN(A.gpa) + MAX(A.gpa))/2, group.age;
explain C;
In the above example:
  • The GROUP statement can be referred to as a whole or by accessing individual fields (as in the example).
  • The GROUP statement and its elements can appear anywhere in the projection.
In the above example, a variety of expressions can be applied to algebraic functions including:
  • A column transformation function such as ABS can be applied to an algebraic function SUM.
  • An algebraic function (COUNT) can be applied to another algebraic function (Distinct), but only the inner function is computed using the combiner.
  • A mathematical expression can be applied to one or more algebraic functions.
You can check if the combiner is used for your query by running EXPLAIN on the FOREACH alias as shown above. You should see the combine section in the MapReduce part of the plan:
.....
Combine Plan
B: Local Rearrange[tuple]{bytearray}(false) - scope-42
| |
| Project[bytearray][0] - scope-43
|
|---C: New For Each(false,false,false)[bag] - scope-28
| |
| Project[bytearray][0] - scope-29
| |
| POUserFunc(org.apache.pig.builtin.SUM$Intermediate)[tuple] - scope-30
| |
| |---Project[bag][1] - scope-31
| |
| POUserFunc(org.apache.pig.builtin.Distinct$Intermediate)[tuple] - scope-32
| |
| |---Project[bag][2] - scope-33
|
|---POCombinerPackage[tuple]{bytearray} - scope-36--------
.....
The combiner is also used with a nested foreach as long as the only nested operation used is DISTINCT (see FOREACH and Example: Nested Block).
A = load 'studenttab10k' as (name, age, gpa);
B = group A by age;
C = foreach B { D = distinct (A.name); generate group, COUNT(D);}
Finally, use of the combiner is influenced by the surrounding environment of the GROUP and FOREACH statements.

When the Combiner is Not Used

The combiner is generally not used if there is any operator that comes between the GROUP and FOREACH statements in the execution plan. Even if the statements are next to each other in your script, the optimizer might rearrange them. In this example, the optimizer will push FILTER above FOREACH which will prevent the use of the combiner:
A = load 'studenttab10k' as (name, age, gpa);
B = group A by age;
C = foreach B generate group, COUNT (A);
D = filter C by group.age <30;
Please note that the script above can be made more efficient by performing filtering before the GROUP statement:
A = load 'studenttab10k' as (name, age, gpa);
B = filter A by age <30;
C = group B by age;
D = foreach C generate group, COUNT (B);
Note: One exception to the above rule is LIMIT. Starting with Pig 0.9, even if LIMIT comes between GROUP and FOREACH, the combiner will still be used. In this example, the optimizer will push LIMIT above FOREACH but this will not prevent the use of the combiner.
A = load 'studenttab10k' as (name, age, gpa);
B = group A by age;
C = foreach B generate group, COUNT (A);
D = limit C 20;
The combiner is also not used in the case where multiple FOREACH statements are associated with the same GROUP:
A = load 'studenttab10k' as (name, age, gpa);
B = group A by age;
C = foreach B generate group, COUNT (A);
D = foreach B generate group, MIN (A.gpa). MAX(A.gpa);
.....
Depending on your use case, it might be more efficient (improve performance) to split your script into multiple scripts.

Hash-based Aggregation in Map Task

To improve performance, hash-based aggregation will aggregate records in the map task before sending them to the combiner. This optimization reduces the serializing/deserializing costs of the combiner by sending it fewer records.
Turning On Off
Hash-based aggregation has been shown to improve the speed of group-by operations by up to 50%. However, since this is a very new feature, it is currently turned OFF by default. To turn it ON, set the property pig.exec.mapPartAgg to true.
Configuring
If the group-by keys used for grouping don't result in a sufficient reduction in the number of records, the performance might be worse with this feature turned ON. To prevent this from happening, the feature turns itself off if the reduction in records sent to combiner is not more than a configurable threshold. This threshold can be set using the property pig.exec.mapPartAgg.minReduction. It is set to a default value of 10, which means that the number of records that get sent to the combiner should be reduced by a factor of 10 or more.

Memory Management

Pig allocates a fix amount of memory to store bags and spills to disk as soon as the memory limit is reached. This is very similar to how Hadoop decides when to spill data accumulated by the combiner.
The amount of memory allocated to bags is determined by pig.cachedbag.memusage; the default is set to 20% (0.2) of available memory. Note that this memory is shared across all large bags used by the application.

Reducer Estimation

By default Pig determines the number of reducers to use for a given job based on the size of the input to the map phase. The input data size is divided by the pig.exec.reducers.bytes.per.reducer parameter value (default 1GB) to determine the number of reducers. The maximum number of reducers for a job is limited by the pig.exec.reducers.max parameter (default 999).
The default reducer estimation algorithm described above can be overridden by setting the pig.exec.reducer.estimator parameter to the fully qualified class name of an implementation of org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigReducerEstimator(MapReduce) or org.apache.pig.backend.hadoop.executionengine.tez.plan.optimizer.TezOperDependencyParallelismEstimator(Tez). The class must exist on the classpath of the process submitting the Pig job. If the pig.exec.reducer.estimator.arg parameter is set, the value will be passed to a constructor of the implementing class that takes a single String.

Multi-Query Execution

With multi-query execution Pig processes an entire script or a batch of statements at once.

Turning it On or Off

Multi-query execution is turned on by default. To turn it off and revert to Pig's "execute-on-dump/store" behavior, use the "-M" or "-no_multiquery" options.
To run script "myscript.pig" without the optimization, execute Pig as follows:
$ pig -M myscript.pig
or
$ pig -no_multiquery myscript.pig

How it Works

Multi-query execution introduces some changes:
  • For batch mode execution, the entire script is first parsed to determine if intermediate tasks can be combined to reduce the overall amount of work that needs to be done; execution starts only after the parsing is completed (see the EXPLAIN operator and the run and exec commands).
  • Two run scenarios are optimized, as explained below: explicit and implicit splits, and storing intermediate results.

Explicit and Implicit Splits

There might be cases in which you want different processing on separate parts of the same data stream.
Example 1:
A = LOAD ...
...
SPLIT A' INTO B IF ..., C IF ...
...
STORE B' ...
STORE C' ...
Example 2:
A = LOAD ...
...
B = FILTER A' ...
C = FILTER A' ...
...
STORE B' ...
STORE C' ...
In prior Pig releases, Example 1 will dump A' to disk and then start jobs for B' and C'. Example 2 will execute all the dependencies of B' and store it and then execute all the dependencies of C' and store it. Both are equivalent, but the performance will be different.
Here's what the multi-query execution does to increase the performance:
  • For Example 2, adds an implicit split to transform the query to Example 1. This eliminates the processing of A' multiple times.
  • Makes the split non-blocking and allows processing to continue. This helps reduce the amount of data that has to be stored right at the split.
  • Allows multiple outputs from a job. This way some results can be stored as a side-effect of the main job. This is also necessary to make the previous item work.
  • Allows multiple split branches to be carried on to the combiner/reducer. This reduces the amount of IO again in the case where multiple branches in the split can benefit from a combiner run.

Storing Intermediate Results

Sometimes it is necessary to store intermediate results.
A = LOAD ...
...
STORE A'
...
STORE A''
If the script doesn't re-load A' for the processing of A the steps above A' will be duplicated. This is a special case of Example 2 above, so the same steps are recommended. With multi-query execution, the script will process A and dump A' as a side-effect.

Store vs. Dump

With multi-query exection, you want to use STORE to save (persist) your results. You do not want to use DUMP as it will disable multi-query execution and is likely to slow down execution. (If you have included DUMP statements in your scripts for debugging purposes, you should remove them.)
DUMP Example: In this script, because the DUMP command is interactive, the multi-query execution will be disabled and two separate jobs will be created to execute this script. The first job will execute A > B > DUMP while the second job will execute A > B > C > STORE.
A = LOAD 'input' AS (x, y, z);
B = FILTER A BY x > 5;
DUMP B;
C = FOREACH B GENERATE y, z;
STORE C INTO 'output';
STORE Example: In this script, multi-query optimization will kick in allowing the entire script to be executed as a single job. Two outputs are produced: output1 and output2.
A = LOAD 'input' AS (x, y, z);
B = FILTER A BY x > 5;
STORE B INTO 'output1';
C = FOREACH B GENERATE y, z;
STORE C INTO 'output2'; 

Error Handling

With multi-query execution Pig processes an entire script or a batch of statements at once. By default Pig tries to run all the jobs that result from that, regardless of whether some jobs fail during execution. To check which jobs have succeeded or failed use one of these options.
First, Pig logs all successful and failed store commands. Store commands are identified by output path. At the end of execution a summary line indicates success, partial failure or failure of all store commands.
Second, Pig returns different code upon completion for these scenarios:
  • Return code 0: All jobs succeeded
  • Return code 1: Used for retrievable errors
  • Return code 2: All jobs have failed
  • Return code 3: Some jobs have failed
In some cases it might be desirable to fail the entire script upon detecting the first failed job. This can be achieved with the "-F" or "-stop_on_failure" command line flag. If used, Pig will stop execution when the first failed job is detected and discontinue further processing. This also means that file commands that come after a failed store in the script will not be executed (this can be used to create "done" files).
This is how the flag is used:
$ pig -F myscript.pig
or
$ pig -stop_on_failure myscript.pig

Backward Compatibility

Most existing Pig scripts will produce the same result with or without the multi-query execution. There are cases though where this is not true. Path names and schemes are discussed here.
Any script is parsed in it's entirety before it is sent to execution. Since the current directory can change throughout the script any path used in LOAD or STORE statement is translated to a fully qualified and absolute path.
In map-reduce mode, the following script will load from "hdfs://<host>:<port>/data1" and store into "hdfs://<host>:<port>/tmp/out1".
cd /;
A = LOAD 'data1';
cd tmp;
STORE A INTO 'out1';
These expanded paths will be passed to any LoadFunc or Slicer implementation. In some cases this can cause problems, especially when a LoadFunc/Slicer is not used to read from a dfs file or path (for example, loading from an SQL database).
Solutions are to either:
  • Specify "-M" or "-no_multiquery" to revert to the old names
  • Specify a custom scheme for the LoadFunc/Slicer
Arguments used in a LOAD statement that have a scheme other than "hdfs" or "file" will not be expanded and passed to the LoadFunc/Slicer unchanged.
In the SQL case, the SQLLoader function is invoked with 'sql://mytable'.
A = LOAD 'sql://mytable' USING SQLLoader();

Implicit Dependencies

If a script has dependencies on the execution order outside of what Pig knows about, execution may fail.

Example

In this script, MYUDF might try to read from out1, a file that A was just stored into. However, Pig does not know that MYUDF depends on the out1 file and might submit the jobs producing the out2 and out1 files at the same time.
...
STORE A INTO 'out1';
B = LOAD 'data2';
C = FOREACH B GENERATE MYUDF($0,'out1');
STORE C INTO 'out2';
To make the script work (to ensure that the right execution order is enforced) add the exec statement. The exec statement will trigger the execution of the statements that produce the out1 file.
...
STORE A INTO 'out1';
EXEC;
B = LOAD 'data2';
C = FOREACH B GENERATE MYUDF($0,'out1');
STORE C INTO 'out2';

Example

In this script, the STORE/LOAD operators have different file paths; however, the LOAD operator depends on the STORE operator.
A = LOAD '/user/xxx/firstinput' USING PigStorage();
B = group ....
C = .... agrregation function
STORE C INTO '/user/vxj/firstinputtempresult/days1';
..
Atab = LOAD '/user/xxx/secondinput' USING  PigStorage();
Btab = group ....
Ctab = .... agrregation function
STORE Ctab INTO '/user/vxj/secondinputtempresult/days1';
..
E = LOAD '/user/vxj/firstinputtempresult/' USING  PigStorage();
F = group ....
G = .... aggregation function
STORE G INTO '/user/vxj/finalresult1';

Etab =LOAD '/user/vxj/secondinputtempresult/' USING  PigStorage();
Ftab = group ....
Gtab = .... aggregation function
STORE Gtab INTO '/user/vxj/finalresult2';
To make the script work, add the exec statement.
A = LOAD '/user/xxx/firstinput' USING PigStorage();
B = group ....
C = .... agrregation function
STORE C INTO '/user/vxj/firstinputtempresult/days1';
..
Atab = LOAD '/user/xxx/secondinput' USING  PigStorage();
Btab = group ....
Ctab = .... agrregation function
STORE Ctab INTO '/user/vxj/secondinputtempresult/days1';

EXEC;

E = LOAD '/user/vxj/firstinputtempresult/' USING  PigStorage();
F = group ....
G = .... aggregation function
STORE G INTO '/user/vxj/finalresult1';
..
Etab =LOAD '/user/vxj/secondinputtempresult/' USING  PigStorage();
Ftab = group ....
Gtab = .... aggregation function
STORE Gtab INTO '/user/vxj/finalresult2';
If the STORE and LOAD both had exact matching file paths, Pig will recognize the implicit dependency and launch two different mapreduce jobs/Tez DAGs with the second job depending on the output of the first one. exec is not required to be specified in that case.

Optimization Rules

Pig supports various optimization rules, all of which are enabled by default. To disable all or specific optimizations, use one or more of the following methods. Note some optimization rules are mandatory and cannot be disabled.
  • The pig.optimizer.rules.disabled pig property, which accepts a comma-separated list of optimization rules to disable; the all keyword disables all non-mandatory optimizations. (e.g.: set pig.optimizer.rules.disabled 'ColumnMapKeyPrune';)
  • The -t, -optimizer_off command-line options. (e.g.: pig -optimizer_off [opt_rule | all])
FilterLogicExpressionSimplifier is an exception to the above. The rule is disabled by default, and enabled by setting the pig.exec.filterLogicExpressionSimplifier pig property to true.

PartitionFilterOptimizer

Push the filter condition to loader.
A = LOAD 'input' as (dt, state, event) using HCatLoader();
B = FILTER A BY dt=='201310' AND state=='CA';
Filter condition will be pushed to loader if loader supports (Usually the loader is partition aware, such as HCatLoader)
A = LOAD 'input' as (dt, state, event) using HCatLoader();
--Filter is removed
Loader will be instructed to loader the partition with dt=='201310' and state=='CA'

PredicatePushdownOptimizer

Push the filter condition to loader. Different than PartitionFilterOptimizer, the filter condition will be evaluated in Pig. In other words, the filter condition pushed to the loader is a hint. Loader might still load records which does not satisfy filter condition.
A = LOAD 'input' using OrcStorage();
B = FILTER A BY dt=='201310' AND state=='CA';
Filter condition will be pushed to loader if loader supports
A = LOAD 'input' using OrcStorage();  -- Filter condition push to loader
B = FILTER A BY dt=='201310' AND state=='CA';  -- Filter evaluated in Pig again

ConstantCalculator

This rule evaluates constant expression.
1) Constant pre-calculation 

B = FILTER A BY a0 > 5+7; 
is simplified to 
B = FILTER A BY a0 > 12; 

2) Evaluate UDF

B = FOREACH A generate UPPER(CONCAT('a', 'b'));
is simplified to 
B = FOREACH A generate 'AB';

SplitFilter

Split filter conditions so that we can push filter more aggressively.
A = LOAD 'input1' as (a0, a1);
B = LOAD 'input2' as (b0, b1);
C = JOIN A by a0, B by b0;
D = FILTER C BY a1>0 and b1>0;
Here D will be splitted into:
X = FILTER C BY a1>0;
D = FILTER X BY b1>0;
So "a1>0" and "b1>0" can be pushed up individually.

PushUpFilter

The objective of this rule is to push the FILTER operators up the data flow graph. As a result, the number of records that flow through the pipeline is reduced.
A = LOAD 'input';
B = GROUP A BY $0;
C = FILTER B BY $0 < 10;

MergeFilter

Merge filter conditions after PushUpFilter rule to decrease the number of filter statements.

PushDownForEachFlatten

The objective of this rule is to reduce the number of records that flow through the pipeline by moving FOREACH operators with a FLATTEN down the data flow graph. In the example shown below, it would be more efficient to move the foreach after the join to reduce the cost of the join operation.
A = LOAD 'input' AS (a, b, c);
B = LOAD 'input2' AS (x, y, z);
C = FOREACH A GENERATE FLATTEN($0), B, C;
D = JOIN C BY $1, B BY $1;

LimitOptimizer

The objective of this rule is to push the LIMIT operator up the data flow graph (or down the tree for database folks). In addition, for top-k (ORDER BY followed by a LIMIT) the LIMIT is pushed into the ORDER BY.
A = LOAD 'input';
B = ORDER A BY $0;
C = LIMIT B 10;

ColumnMapKeyPrune

Prune the loader to only load necessary columns. The performance gain is more significant if the corresponding loader support column pruning and only load necessary columns (See LoadPushDown.pushProjection). Otherwise, ColumnMapKeyPrune will insert a ForEach statement right after loader.
A = load 'input' as (a0, a1, a2);
B = ORDER A by a0;
C = FOREACH B GENERATE a0, a1;
a2 is irrelevant in this query, so we can prune it earlier. The loader in this query is PigStorage and it supports column pruning. So we only load a0 and a1 from the input file.
ColumnMapKeyPrune also prunes unused map keys:
A = load 'input' as (a0:map[]);
B = FOREACH A generate a0#'key1';

AddForEach

Prune unused column as soon as possible. In addition to prune the loader in ColumnMapKeyPrune, we can prune a column as soon as it is not used in the rest of the script
-- Original code: 

A = LOAD 'input' AS (a0, a1, a2); 
B = ORDER A BY a0;
C = FILTER B BY a1>0;
We can only prune a2 from the loader. However, a0 is never used after "ORDER BY". So we can drop a0 right after "ORDER BY" statement.
-- Optimized code: 

A = LOAD 'input' AS (a0, a1, a2); 
B = ORDER A BY a0;
B1 = FOREACH B GENERATE a1;  -- drop a0
C = FILTER B1 BY a1>0;

MergeForEach

The objective of this rule is to merge together two feach statements, if these preconditions are met:
  • The foreach statements are consecutive.
  • The first foreach statement does not contain flatten.
  • The second foreach is not nested.
-- Original code: 

A = LOAD 'file.txt' AS (a, b, c); 
B = FOREACH A GENERATE a+b AS u, c-b AS v; 
C = FOREACH B GENERATE $0+5, v; 

-- Optimized code: 

A = LOAD 'file.txt' AS (a, b, c); 
C = FOREACH A GENERATE a+b+5, c-b;

GroupByConstParallelSetter

Force parallel "1" for "group all" statement. That's because even if we set parallel to N, only 1 reducer will be used in this case and all other reducer produce empty result.
A = LOAD 'input';
B = GROUP A all PARALLEL 10;

Performance Enhancers

Use Optimization

Pig supports various optimization rules which are turned on by default. Become familiar with these rules.

Use Types

If types are not specified in the load statement, Pig assumes the type of =double= for numeric computations. A lot of the time, your data would be much smaller, maybe, integer or long. Specifying the real type will help with speed of arithmetic computation. It has an additional advantage of early error detection.
--Query 1
A = load 'myfile' as (t, u, v);
B = foreach A generate t + u;

--Query 2
A = load 'myfile' as (t: int, u: int, v);
B = foreach A generate t + u;
The second query will run more efficiently than the first. In some of our queries with see 2x speedup.

Project Early and Often

Pig does not (yet) determine when a field is no longer needed and drop the field from the row. For example, say you have a query like:
A = load 'myfile' as (t, u, v);
B = load 'myotherfile' as (x, y, z);
C = join A by t, B by x;
D = group C by u;
E = foreach D generate group, COUNT($1);
There is no need for v, y, or z to participate in this query. And there is no need to carry both t and x past the join, just one will suffice. Changing the query above to the query below will greatly reduce the amount of data being carried through the map and reduce phases by pig.
A = load 'myfile' as (t, u, v);
A1 = foreach A generate t, u;
B = load 'myotherfile' as (x, y, z);
B1 = foreach B generate x;
C = join A1 by t, B1 by x;
C1 = foreach C generate t, u;
D = group C1 by u;
E = foreach D generate group, COUNT($1);
Depending on your data, this can produce significant time savings. In queries similar to the example shown here we have seen total time drop by 50%.

Filter Early and Often

As with early projection, in most cases it is beneficial to apply filters as early as possible to reduce the amount of data flowing through the pipeline.
-- Query 1
A = load 'myfile' as (t, u, v);
B = load 'myotherfile' as (x, y, z);
C = filter A by t == 1;
D = join C by t, B by x;
E = group D by u;
F = foreach E generate group, COUNT($1);

-- Query 2
A = load 'myfile' as (t, u, v);
B = load 'myotherfile' as (x, y, z);
C = join A by t, B by x;
D = group C by u;
E = foreach D generate group, COUNT($1);
F = filter E by C.t == 1;
The first query is clearly more efficient than the second one because it reduces the amount of data going into the join.
One case where pushing filters up might not be a good idea is if the cost of applying filter is very high and only a small amount of data is filtered out.

Reduce Your Operator Pipeline

For clarity of your script, you might choose to split your projects into several steps for instance:
A = load 'data' as (in: map[]);
-- get key out of the map
B = foreach A generate in#'k1' as k1, in#'k2' as k2;
-- concatenate the keys
C = foreach B generate CONCAT(k1, k2);
.......
While the example above is easier to read, you might want to consider combining the two foreach statements to improve your query performance:
A = load 'data' as (in: map[]);
-- concatenate the keys from the map
B = foreach A generate CONCAT(in#'k1', in#'k2');
....
The same goes for filters.

Make Your UDFs Algebraic

Queries that can take advantage of the combiner generally ran much faster (sometimes several times faster) than the versions that don't. The latest code significantly improves combiner usage; however, you need to make sure you do your part. If you have a UDF that works on grouped data and is, by nature, algebraic (meaning their computation can be decomposed into multiple steps) make sure you implement it as such. For details on how to write algebraic UDFs, see Algebraic Interface.
A = load 'data' as (x, y, z)
B = group A by x;
C = foreach B generate group, MyUDF(A);
....
If MyUDF is algebraic, the query will use combiner and run much faster. You can run explain command on your query to make sure that combiner is used.

Use the Accumulator Interface

If your UDF can't be made Algebraic but is able to deal with getting input in chunks rather than all at once, consider implementing the Accumulator interface to reduce the amount of memory used by your script. If your function is Algebraic and can be used on conjunction with Accumulator functions, you will need to implement the Accumulator interface as well as the Algebraic interface. For more information, see Accumulator Interface.
Note: Pig automatically chooses the interface that it expects to provide the best performance: Algebraic > Accumulator > Default.

Drop Nulls Before a Join

With the introduction of nulls, join and cogroup semantics were altered to work with nulls. The semantic for cogrouping with nulls is that nulls from a given input are grouped together, but nulls across inputs are not grouped together. This preserves the semantics of grouping (nulls are collected together from a single input to be passed to aggregate functions like COUNT) and the semantics of join (nulls are not joined across inputs). Since flattening an empty bag results in an empty row (and no output), in a standard join the rows with a null key will always be dropped.
This join
A = load 'myfile' as (t, u, v);
B = load 'myotherfile' as (x, y, z);
C = join A by t, B by x;
is rewritten by Pig to
A = load 'myfile' as (t, u, v);
B = load 'myotherfile' as (x, y, z);
C1 = cogroup A by t INNER, B by x INNER;
C = foreach C1 generate flatten(A), flatten(B);
Since the nulls from A and B won't be collected together, when the nulls are flattened we're guaranteed to have an empty bag, which will result in no output. So the null keys will be dropped. But they will not be dropped until the last possible moment.
If the query is rewritten to
A = load 'myfile' as (t, u, v);
B = load 'myotherfile' as (x, y, z);
A1 = filter A by t is not null;
B1 = filter B by x is not null;
C = join A1 by t, B1 by x;
then the nulls will be dropped before the join. Since all null keys go to a single reducer, if your key is null even a small percentage of the time the gain can be significant. In one test where the key was null 7% of the time and the data was spread across 200 reducers, we saw a about a 10x speed up in the query by adding the early filters.

Take Advantage of Join Optimizations

Regular Join Optimizations
Optimization for regular joins ensures that the last table in the join is not brought into memory but streamed through instead. Optimization reduces the amount of memory used which means you can avoid spilling the data and also should be able to scale your query to larger data volumes.
To take advantage of this optimization, make sure that the table with the largest number of tuples per key is the last table in your query. In some of our tests we saw 10x performance improvement as the result of this optimization.
small = load 'small_file' as (t, u, v);
large = load 'large_file' as (x, y, z);
C = join small by t, large by x;
Specialized Join Optimizations
Optimization can also be achieved using fragment replicate joins, skewed joins, and merge joins. For more information see Specialized Joins.

Use the Parallel Features

You can set the number of reduce tasks for the MapReduce jobs generated by Pig using two parallel features. (The parallel features only affect the number of reduce tasks. Map parallelism is determined by the input file, one map for each HDFS block.)
You Set the Number of Reducers
Use the set default parallel command to set the number of reducers at the script level.
Alternatively, use the PARALLEL clause to set the number of reducers at the operator level. (In a script, the value set via the PARALLEL clause will override any value set via "set default parallel.") You can include the PARALLEL clause with any operator that starts a reduce phase: COGROUP, CROSS, DISTINCT, GROUP, JOIN (inner), JOIN (outer), and ORDER BY. PARALLEL clause can also be used with UNION if Tez is the execution mode. It will turn off the union optimization and introduce an extra reduce step. Though it will have slightly degraded performance due to the extra step, it is very useful for controlling the number of output files.
The number of reducers you need for a particular construct in Pig that forms a MapReduce boundary depends entirely on (1) your data and the number of intermediate keys you are generating in your mappers and (2) the partitioner and distribution of map (combiner) output keys. In the best cases we have seen that a reducer processing about 1 GB of data behaves efficiently.
Let Pig Set the Number of Reducers
If neither "set default parallel" nor the PARALLEL clause are used, Pig sets the number of reducers using a heuristic based on the size of the input data. You can set the values for these properties:
  • pig.exec.reducers.bytes.per.reducer - Defines the number of input bytes per reduce; default value is 1000*1000*1000 (1GB).
  • pig.exec.reducers.max - Defines the upper bound on the number of reducers; default is 999.
The formula, shown below, is very simple and will improve over time. The computed value takes all inputs within the script into account and applies the computed value to all the jobs within Pig script.
#reducers = MIN (pig.exec.reducers.max, total input size (in bytes) / bytes per reducer)
Examples
In this example PARALLEL is used with the GROUP operator.
A = LOAD 'myfile' AS (t, u, v);
B = GROUP A BY t PARALLEL 18;
...
In this example all the MapReduce jobs that get launched use 20 reducers.
SET default_parallel 20;
A = LOAD 'myfile.txt' USING PigStorage() AS (t, u, v);
B = GROUP A BY t;
C = FOREACH B GENERATE group, COUNT(A.t) as mycount;
D = ORDER C BY mycount;
STORE D INTO 'mysortedcount' USING PigStorage();

Use the LIMIT Operator

Often you are not interested in the entire output but rather a sample or top results. In such cases, using LIMIT can yield a much better performance as we push the limit as high as possible to minimize the amount of data travelling through the pipeline.
Sample:
A = load 'myfile' as (t, u, v);
B = limit A 500;
Top results:
A = load 'myfile' as (t, u, v);
B = order A by t;
C = limit B 500;

Prefer DISTINCT over GROUP BY/GENERATE

To extract unique values from a column in a relation you can use DISTINCT or GROUP BY/GENERATE. DISTINCT is the preferred method; it is faster and more efficient.
Example using GROUP BY - GENERATE:
A = load 'myfile' as (t, u, v);
B = foreach A generate u;
C = group B by u;
D = foreach C generate group as uniquekey;
dump D; 
Example using DISTINCT:
A = load 'myfile' as (t, u, v);
B = foreach A generate u;
C = distinct B;
dump C; 

Compress the Results of Intermediate Jobs

If your Pig script generates a sequence of MapReduce jobs, you can compress the output of the intermediate jobs using LZO compression. (Use the EXPLAIN operator to determine if your script produces multiple MapReduce Jobs.)
By doing this, you will save HDFS space used to store the intermediate data used by PIG and potentially improve query execution speed. In general, the more intermediate data that is generated, the more benefits in storage and speed that result.
You can set the value for these properties:
  • pig.tmpfilecompression - Determines if the temporary files should be compressed or not (set to false by default).
  • pig.tmpfilecompression.codec - Specifies which compression codec to use. Currently, Pig accepts "gz" and "lzo" as possible values. However, because LZO is under GPL license (and disabled by default) you will need to configure your cluster to use the LZO codec to take advantage of this feature. For details, see http://code.google.com/p/hadoop-gpl-compression/wiki/FAQ.
On the non-trivial queries (one ran longer than a couple of minutes) we saw significant improvements both in terms of query latency and space usage. For some queries we saw up to 96% disk saving and up to 4x query speed up. Of course, the performance characteristics are very much query and data dependent and testing needs to be done to determine gains. We did not see any slowdown in the tests we peformed which means that you are at least saving on space while using compression.
With gzip we saw a better compression (96-99%) but at a cost of 4% slowdown. Thus, we don't recommend using gzip.
Example
-- launch Pig script using lzo compression 

java -cp $PIG_HOME/pig.jar 
-Djava.library.path=<path to the lzo library> 
-Dpig.tmpfilecompression=true 
-Dpig.tmpfilecompression.codec=lzo org.apache.pig.Main  myscript.pig 

Combine Small Input Files

Processing input (either user input or intermediate input) from multiple small files can be inefficient because a separate map has to be created for each file. Pig can now combined small files so that they are processed as a single map.
You can set the values for these properties:
  • pig.maxCombinedSplitSize – Specifies the size, in bytes, of data to be processed by a single map. Smaller files are combined untill this size is reached.
  • pig.splitCombination – Turns combine split files on or off (set to “true” by default).
This feature works with PigStorage. However, if you are using a custom loader, please note the following:
  • If your loader implementation makes use of the PigSplit object passed through the prepareToRead method, then you may need to rebuild the loader since the definition of PigSplit has been modified.
  • The loader must be stateless across the invocations to the prepareToRead method. That is, the method should reset any internal states that are not affected by the RecordReader argument.
  • If a loader implements IndexableLoadFunc, or implements OrderedLoadFunc and CollectableLoadFunc, its input splits won't be subject to possible combinations.

Direct Fetch

When the DUMP operator is used to execute Pig Latin statements, Pig can take the advantage to minimize latency by directly reading data from HDFS rather than launching MapReduce jobs.
The result is fetched if the query contains any of the following operators: FILTER, FOREACH, LIMIT, STREAM, UNION. Fetching will be disabled in case of:
  • the presence of other operators, sample loaders and scalar expressions
  • no LIMIT operator
  • implicit splits
Also note that direct-fetch doesn't support UDFs that interact with the distributed cache. You can check if the query can be fetched by running EXPLAIN. You should see "No MR jobs. Fetch only." in the MapReduce part of the plan.
Direct fetch is turned on by default. To turn it off set the property opt.fetch to false or start Pig with the "-N" or "-no_fetch" option.

Auto Local Mode

Processing small mapreduce jobs on hadoop cluster could be slow as it has overhead of job startup and job scheduling. For jobs with small input data, pig can convert them to run them as in-process mapreduce with hadoop's local mode. If pig.auto.local.enabled flag is set to true, pig will convert mapreduce jobs with input data less than pig.auto.local.input.maxbytes (100MB by default) to run in local mode, provided the number of reducers required by the job are less than or equal to 1. Note, jobs converted to run in local mode load and store data from HDFS, so any job in the pig workflow(dag) could be converted to run in local mode without affecting its downstream jobs.
You can set the values for these properties in order to configure the behavior:
  • pig.auto.local.enabled - Turns on/off auto local mode feature (false by default).
  • pig.auto.local.input.maxbytes - Controls the max threshold size (in bytes) to convert jobs to run in local mode (100MB by default).
Sometimes, you may want change job configuration for jobs that are converted to run in local mode (eg- change io.sort.mb for small jobs). To do so, you can use pig.local. prefix to any configuration and configuration will be set on converted jobs. For example, set pig.local.io.sort.mb 100 will change io.sort.mb value to 100 for jobs converted to run in local mode.

User Jar Cache

Jars required for user defined functions (UDFs) are copied to distributed cache by pig to make them available on task nodes. To put these jars on distributed cache, pig clients copy these jars to HDFS under a temporary location. For scheduled jobs, these jars do not change frequently. Also, creating a lot of small jar files on HDFS is not HDFS friendly. To avoid copying these small jar files to HDFS again and again, pig allows users to configure a user level jar cache (readable only to the user for security reasons). If pig.user.cache.enabled flag is set to true, UDF jars are copied to jar cache location (configurable) under a directory named with the hash (SHA) of the jar. Hash of the jar is used to identify the existence of the jar in subsequent uses of the jar by the user. If a jar with same hash and filename is found in the cache, it is used avoiding copy of the jar to hdfs.
You can set the values for these properties in order to configure the jar cache:
  • pig.user.cache.enabled - Turn on/off user jar cache feature (false by default).
  • pig.user.cache.location - Path on HDFS that will be used a staging directory for the user jar cache (defaults to pig.temp.dir or /tmp).
User jar cache feature is fail safe. If jars cannot be copied to jar cache due to any permission/configuration problems, pig will default old behavior.

Specialized Joins

Replicated Joins

Fragment replicate join is a special type of join that works well if one or more relations are small enough to fit into main memory. In such cases, Pig can perform a very efficient join because all of the hadoop work is done on the map side. In this type of join the large relation is followed by one or more small relations. The small relations must be small enough to fit into main memory; if they don't, the process fails and an error is generated.

Usage

Perform a replicated join with the USING clause (see JOIN (inner) and JOIN (outer)). In this example, a large relation is joined with two smaller relations. Note that the large relation comes first followed by the smaller relations; and, all small relations together must fit into main memory, otherwise an error is generated.
big = LOAD 'big_data' AS (b1,b2,b3);

tiny = LOAD 'tiny_data' AS (t1,t2,t3);

mini = LOAD 'mini_data' AS (m1,m2,m3);

C = JOIN big BY b1, tiny BY t1, mini BY m1 USING 'replicated';

Conditions

Fragment replicate joins are experimental; we don't have a strong sense of how small the small relation must be to fit into memory. In our tests with a simple query that involves just a JOIN, a relation of up to 100 M can be used if the process overall gets 1 GB of memory. Please share your observations and experience with us.
In order to avoid replicated joins on large relations, we fail if size of relation(s) to be replicated (in bytes) is greater than pig.join.replicated.max.bytes (default = 1GB).

Bloom Joins

Bloom join is a special type of join where a bloom filter is constructed using join keys of one relation and used to filter records of the other relations before doing a regular hash join. The amount of data sent to the reducers will be a lot less depending up on the numbers of records that are filtered on the map side. Bloom join is very useful in cases where the number of matching records between relations in a join are comparatively less compared to the total records allowing many to be filtered before the join. Before bloom join was added as a type of join, same functionality was achieved by users by using the builtin bloom udfs which is not as efficient and required more lines of code as well. Currently bloom join is only implemented in Tez execution mode. Builtin bloom udfs have to be used for other execution modes.

Usage

Perform a bloom join with the USING clause (see JOIN (inner) and JOIN (outer)). In this example, a large relation is joined with two smaller relations. Note that the large relation comes first followed by the smaller relations. Bloom filter is built from join keys of the right most relation which is small and the filter is applied on the big and medium relations. None of the relations are required to fit into main memory.
big = LOAD 'big_data' AS (b1,b2,b3);

medium = LOAD 'medium_data' AS (m1,m2,m3);

small = LOAD 'small_data' AS (s1,s2,s3);

C = JOIN big BY b1, medium BY m1, small BY s1 USING 'bloom';
In the case of inner join and right outer join, the right most relation is used for building the bloom filter and the users are expected to specify the smaller dataset as the right most relation. But in the case of left outer join, the left most relation is used for building the bloom filter and is expected to be the smaller dataset. This is because all records of the outer relation should be in the result and no records can be filtered. If the left relation turns out to be the bigger dataset, it would not be as efficient to build the bloom filter on the bigger dataset. But it might still perform better than a regular join if it is able to filter lot of records from the right relation.
big = LOAD 'big_data' AS (b1,b2,b3);

small = LOAD 'small_data' AS (m1,m2,m3);

C = JOIN small BY s1 LEFT, big BY b1 USING 'bloom';

Conditions

  • Bloom join cannot be used with a FULL OUTER join.
  • If the the underlying data is sufficiently skewed, bloom join might not help. Skewed join can be considered for those cases.

Tuning options

There are multiple pig properties than can be configured to construct a more efficient bloom filter. See Bloom Filter for a discussion of how to select the number of bits and the number of hash functions. Easier option would be to search for "bloom filter calculator" in a search engine and use one of the online bloom filter calculators available to arrive at the desired values.
  • pig.bloomjoin.strategy - The valid values for this are 'map' and 'reduce'. Default value is map. Bloom join has two different kind of implementations to be more efficient in different cases. In general, there is an extra reduce step in the DAG for construction of the bloom filter(s).
    • map - In each map, bloom filters are computed on the join keys partitioned by the hashcode of the key with pig.bloomjoin.num.filters number of partitions. Bloom filters for each partition from different maps are then combined in the reducers producing one bloom filter per partition. The default value of pig.bloomjoin.num.filters is 1 for this strategy and so usually only one bloom filter is created. This is efficient and fast if there are smaller number of maps (<10) and the number of distinct keys are not too high. It can be faster with larger number of maps and even with bigger bloom vector sizes, but the amount of data shuffled to the reducer for aggregation becomes huge making it inefficient.
    • reduce - Join keys are sent from the map to the reducer partitioned by hashcode of the key with pig.bloomjoin.num.filters number of partitions. In the reducers, one bloom filter is then computed per partition. Number of reducers are set equal to the number of partitions allowing for each bloom filter to be computed in parallel. The default value of pig.bloomjoin.num.filters is 11 for this strategy. This is efficient for larger datasets with lot of maps or very large bloom vector size. In this case size of keys sent to the reducer is smaller than sending bloom filters to reducer for aggregation making it efficient.
  • pig.bloomjoin.num.filters - The number of bloom filters that will be created. Default is 1 for map strategy and 11 for reduce strategy.
  • pig.bloomjoin.vectorsize.bytes - The size in bytes of the bit vector to be used for the bloom filter. A bigger vector size will be needed when the number of distinct keys is higher. Default value is 1048576 (1MB).
  • pig.bloomjoin.hash.functions - The type of hash function to use. Valid values are 'jenkins' and 'murmur'. Default is murmur.
  • pig.bloomjoin.hash.types - The number of hash functions to be used in bloom computation. It determines the probability of false positives. Higher the number lower the false positives. Too high a value can increase the cpu time. Default value is 3.

Skewed Joins

Parallel joins are vulnerable to the presence of skew in the underlying data. If the underlying data is sufficiently skewed, load imbalances will swamp any of the parallelism gains. In order to counteract this problem, skewed join computes a histogram of the key space and uses this data to allocate reducers for a given key. Skewed join does not place a restriction on the size of the input keys. It accomplishes this by splitting the left input on the join predicate and streaming the right input. The left input is sampled to create the histogram.
Skewed join can be used when the underlying data is sufficiently skewed and you need a finer control over the allocation of reducers to counteract the skew. It should also be used when the data associated with a given key is too large to fit in memory.

Usage

Perform a skewed join with the USING clause (see JOIN (inner) and JOIN (outer)).
A = LOAD 'skewed_data' AS (a1,a2,a3);
B = LOAD 'data' AS (b1,b2,b3);
C = JOIN A BY a1, B BY b1 USING 'skewed';

Conditions

Skewed join will only work under these conditions:
  • Skewed join works with two-table inner and outer join. Currently we do not support more than two tables for skewed join. Specifying three-way (or more) joins will fail validation. For such joins, we rely on you to break them up into two-way joins.
  • The skewed table must be specified as the left table. Pig samples on that table and determines the number of reducers per key.
  • The pig.skewedjoin.reduce.memusage Java parameter specifies the fraction of heap available for the reducer to perform the join. A low fraction forces Pig to use more reducers but increases copying cost. We have seen good performance when we set this value in the range 0.1 - 0.4. However, note that this is hardly an accurate range. Its value depends on the amount of heap available for the operation, the number of columns in the input and the skew. An appropriate value is best obtained by conducting experiments to achieve a good performance. The default value is 0.5.
  • Skewed join does not address (balance) uneven data distribution across reducers. However, in most cases, skewed join ensures that the join will finish (however slowly) rather than fail.

Merge Joins

Often user data is stored such that both inputs are already sorted on the join key. In this case, it is possible to join the data in the map phase of a MapReduce job. This provides a significant performance improvement compared to passing all of the data through unneeded sort and shuffle phases.
Pig has implemented a merge join algorithm, or sort-merge join. It works on pre-sorted data, and does not sort data for you. See Conditions, below, for restrictions that apply when using this join algorithm. Pig implements the merge join algorithm by selecting the left input of the join to be the input file for the map phase, and the right input of the join to be the side file. It then samples records from the right input to build an index that contains, for each sampled record, the key(s) the filename and the offset into the file the record begins at. This sampling is done in the first MapReduce job. A second MapReduce job is then initiated, with the left input as its input. Each map uses the index to seek to the appropriate record in the right input and begin doing the join.

Usage

Perform a merge join with the USING clause (see JOIN (inner) and JOIN (outer)).
C = JOIN A BY a1, B BY b1, C BY c1 USING 'merge';

Conditions

Condition A
Inner merge join (between two tables) will only work under these conditions:
  • Data must come directly from either a Load or an Order statement.
  • There may be filter statements and foreach statements between the sorted data source and the join statement. The foreach statement should meet the following conditions:
    • The foreach statement should not change the position of the join keys.
    • There should be no transformation on the join keys which will change the sort order.
    • UDFs also have to adhere to the previous condition and should not transform the JOIN keys in a way that would change the sort order.
  • Data must be sorted on join keys in ascending (ASC) order on both sides.
  • If sort is provided by the loader, rather than an explicit Order operation, the right-side loader must implement either the {OrderedLoadFunc} interface or {IndexableLoadFunc} interface.
  • Type information must be provided for the join key in the schema.
The PigStorage loader satisfies all of these conditions.
Condition B
Outer merge join (between two tables) and inner merge join (between three or more tables) will only work under these conditions:
  • No other operations can be done between the load and join statements.
  • Data must be sorted on join keys in ascending (ASC) order on both sides.
  • Left-most loader must implement {CollectableLoader} interface as well as {OrderedLoadFunc}.
  • All other loaders must implement {IndexableLoadFunc}.
  • Type information must be provided for the join key in the schema.
Pig does not provide a loader that supports outer merge joins. You will need to build your own loader to take advantage of this feature.

Merge-Sparse Joins

Merge-Sparse join is a specialization of merge join. Merge-sparse join is intended for use when one of the tables is very sparse, meaning you expect only a small number of records to be matched during the join. In tests this join performed well for cases where less than 1% of the data was matched in the join.

Usage

Perform a merge-sparse join with the USING clause (see JOIN (inner)).
a = load 'sorted_input1' using org.apache.pig.piggybank.storage.IndexedStorage('\t', '0');
b = load 'sorted_input2' using org.apache.pig.piggybank.storage.IndexedStorage('\t', '0');
c = join a by $0, b by $0 using 'merge-sparse';
store c into 'results';

Conditions

Merge-sparse join only works for inner joins and is not currently implemented for outer joins.
For inner joins, the preconditions are the same as for merge join with the exception of constrains on the right-side loader. For sparse-merge joins the loader must implement IndexedLoadFunc or the join will fail.
Piggybank now contains a load function called org.apache.pig.piggybank.storage.IndexedStorage that is a derivation of PigStorage and implements IndexedLoadFunc. This is the only loader included in the standard Pig distribution that can be used for merge-sparse join.

Performance Considerations

Note the following:
  • If one of the data sets is small enough to fit into memory, a Replicated Join is very likely to provide better performance.
  • You will also see better performance if the data in the left table is partitioned evenly across part files (no significant skew and each part file contains at least one full block of data).

Diagnostic Operators

DESCRIBE

Returns the schema of a relation.

Syntax

DESCRIBE alias;       

Terms

alias
The name of a relation.

Usage

Use the DESCRIBE operator to view the schema of a relation. You can view outer relations as well as relations defined in a nested FOREACH statement.

Example

In this example a schema is specified using the AS clause. If all data conforms to the schema, Pig will use the assigned types.
A = LOAD 'student' AS (name:chararray, age:int, gpa:float);

B = FILTER A BY name matches 'J.+';

C = GROUP B BY name;

D = FOREACH C GENERATE COUNT(B.age);

DESCRIBE A;
A: {name: chararray,age: int,gpa: float}

DESCRIBE B;
B: {name: chararray,age: int,gpa: float}

DESCRIBE C;
C: {group: chararray,B: {(name: chararray,age: int,gpa: float)}}

DESCRIBE D;
D: {long}
In this example no schema is specified. All fields default to type bytearray or long (see Data Types).
a = LOAD 'student';

b = FILTER a BY $0 matches 'J.+';

c = GROUP b BY $0;

d = FOREACH c GENERATE COUNT(b.$1);

DESCRIBE a;
Schema for a unknown.

DESCRIBE b;
2008-12-05 01:17:15,316 [main] WARN  org.apache.pig.PigServer - bytearray is implicitly cast to chararray under LORegexp Operator
Schema for b unknown.

DESCRIBE c;
2008-12-05 01:17:23,343 [main] WARN  org.apache.pig.PigServer - bytearray is implicitly caste to chararray under LORegexp Operator
c: {group: bytearray,b: {null}}

DESCRIBE d;
2008-12-05 03:04:30,076 [main] WARN  org.apache.pig.PigServer - bytearray is implicitly caste to chararray under LORegexp Operator
d: {long}
This example shows how to view the schema of a nested relation using the :: operator.
A = LOAD 'studentab10k' AS (name, age, gpa); 
B = GROUP A BY name; 
C = FOREACH B { 
     D = DISTINCT A.age; 
     GENERATE COUNT(D), group;} 

DESCRIBE C::D; 
D: {age: bytearray} 

DUMP

Dumps or displays results to screen.

Syntax

DUMP alias;       

Terms

alias
The name of a relation.

Usage

Use the DUMP operator to run (execute) Pig Latin statements and display the results to your screen. DUMP is meant for interactive mode; statements are executed immediately and the results are not saved (persisted). You can use DUMP as a debugging device to make sure that the results you are expecting are actually generated.
Note that production scripts SHOULD NOT use DUMP as it will disable multi-query optimizations and is likely to slow down execution (see Store vs. Dump).

Example

In this example a dump is performed after each statement.
A = LOAD 'student' AS (name:chararray, age:int, gpa:float);

DUMP A;
(John,18,4.0F)
(Mary,19,3.7F)
(Bill,20,3.9F)
(Joe,22,3.8F)
(Jill,20,4.0F)

B = FILTER A BY name matches 'J.+';

DUMP B;
(John,18,4.0F)
(Joe,22,3.8F)
(Jill,20,4.0F)

EXPLAIN

Displays execution plans.

Syntax

EXPLAIN [–script pigscript] [–out path] [–brief] [–dot] [-xml] [–param param_name = param_value] [–param_file file_name] alias;

Terms

–script
Use to specify a Pig script.
–out
Use to specify the output path (directory).
Will generate a logical_plan[.txt|.dot], physical_plan[.text|.dot], exec_plan[.text|.dot] file in the specified path.
Default (no path specified): Stdout
–brief
Does not expand nested plans (presenting a smaller graph for overview).
–dot, -xml
Text mode (default): multiple output (split) will be broken out in sections.
Dot mode: outputs a format that can be passed to the dot utility for graphical display – will generate a directed-acyclic-graph (DAG) of the plans in any supported format (.gif, .jpg ...).
Xml mode: outputs a xml which represent the plan (only logical plan is shown currently).
–param param_name = param_value
See Parameter Substitution.
–param_file file_name
See Parameter Substitution.
alias
The name of a relation.

Usage

Use the EXPLAIN operator to review the logical, physical, and map reduce execution plans that are used to compute the specified relationship.
If no script is given:
  • The logical plan shows a pipeline of operators to be executed to build the relation. Type checking and backend-independent optimizations (such as applying filters early on) also apply.
  • The physical plan shows how the logical operators are translated to backend-specific physical operators. Some backend optimizations also apply.
  • The mapreduce plan shows how the physical operators are grouped into map reduce jobs.

If a script without an alias is specified, it will output the entire execution graph (logical, physical, or map reduce).
If a script with a alias is specified, it will output the plan for the given alias.

Example

In this example the EXPLAIN operator produces all three plans. (Note that only a portion of the output is shown in this example.)
A = LOAD 'student' AS (name:chararray, age:int, gpa:float);

B = GROUP A BY name;

C = FOREACH B GENERATE COUNT(A.age);

EXPLAIN C;
-----------------------------------------------
Logical Plan:
-----------------------------------------------
Store xxx-Fri Dec 05 19:42:29 UTC 2008-23 Schema: {long} Type: Unknown
|
|---ForEach xxx-Fri Dec 05 19:42:29 UTC 2008-15 Schema: {long} Type: bag
 etc ...  

-----------------------------------------------
Physical Plan:
-----------------------------------------------
Store(fakefile:org.apache.pig.builtin.PigStorage) - xxx-Fri Dec 05 19:42:29 UTC 2008-40
|
|---New For Each(false)[bag] - xxx-Fri Dec 05 19:42:29 UTC 2008-39
    |   |
    |   POUserFunc(org.apache.pig.builtin.COUNT)[long] - xxx-Fri Dec 05 
 etc ...  

--------------------------------------------------
| Map Reduce Plan                               
-------------------------------------------------
MapReduce node xxx-Fri Dec 05 19:42:29 UTC 2008-41
Map Plan
Local Rearrange[tuple]{chararray}(false) - xxx-Fri Dec 05 19:42:29 UTC 2008-34
|   |
|   Project[chararray][0] - xxx-Fri Dec 05 19:42:29 UTC 2008-35
 etc ...  

If you are running in Tez mode, Map Reduce Plan will be replaced with Tez Plan:

#--------------------------------------------------
# There are 1 DAGs in the session
#--------------------------------------------------
#--------------------------------------------------
# TEZ DAG plan: PigLatin:185.pig-0_scope-0
#--------------------------------------------------
Tez vertex scope-21 -> Tez vertex scope-22,
Tez vertex scope-22

Tez vertex scope-21
# Plan on vertex
B: Local Rearrange[tuple]{chararray}(false) - scope-35 ->  scope-22
 etc ...  

ILLUSTRATE

Displays a step-by-step execution of a sequence of statements.

Syntax

ILLUSTRATE {alias | -script scriptfile};

Terms

alias
The name of a relation.
-script scriptfile
The script keyword followed by the name of a Pig script (for example, myscript.pig).
The script file should not contain an ILLUSTRATE statement.

Usage

Use the ILLUSTRATE operator to review how data is transformed through a sequence of Pig Latin statements. ILLUSTRATE allows you to test your programs on small datasets and get faster turnaround times.
ILLUSTRATE is based on an example generator (see Generating Example Data for Dataflow Programs). The algorithm works by retrieving a small sample of the input data and then propagating this data through the pipeline. However, some operators, such as JOIN and FILTER, can eliminate tuples from the data - and this could result in no data following through the pipeline. To address this issue, the algorithm will automatically generate example data, in near real-time. Thus, you might see data propagating through the pipeline that was not found in the original input data, but this data changes nothing and ensures that you will be able to examine the semantics of your Pig Latin statements.
As shown in the examples below, you can use ILLUSTRATE to review a relation or an entire Pig script.

Example - Relation

This example demonstrates how to use ILLUSTRATE with a relation. Note that the LOAD statement must include a schema (the AS clause).
grunt> visits = LOAD 'visits.txt' AS (user:chararray, url:chararray, timestamp:chararray);
grunt> DUMP visits;

(Amy,yahoo.com,19990421)
(Fred,harvard.edu,19991104)
(Amy,cnn.com,20070218)
(Frank,nba.com,20070305)
(Fred,berkeley.edu,20071204)
(Fred,stanford.edu,20071206)

grunt> recent_visits = FILTER visits BY timestamp >= '20071201';
grunt> user_visits = GROUP recent_visits BY user;
grunt> num_user_visits = FOREACH user_visits GENERATE group, COUNT(recent_visits);
grunt> DUMP num_user_visits;

(Fred,2)

grunt> ILLUSTRATE num_user_visits;
------------------------------------------------------------------------
| visits     | user: chararray | url: chararray | timestamp: chararray |
------------------------------------------------------------------------
|            | Fred            | berkeley.edu   | 20071204             |
|            | Fred            | stanford.edu   | 20071206             |
|            | Frank           | nba.com        | 20070305             |
------------------------------------------------------------------------
-------------------------------------------------------------------------------
| recent_visits     | user: chararray | url: chararray | timestamp: chararray |
-------------------------------------------------------------------------------
|                   | Fred            | berkeley.edu   | 20071204             |
|                   | Fred            | stanford.edu   | 20071206             |
-------------------------------------------------------------------------------
------------------------------------------------------------------------------------------------------------------
| user_visits     | group: chararray | recent_visits: bag({user: chararray,url: chararray,timestamp: chararray}) |
------------------------------------------------------------------------------------------------------------------
|                 | Fred             | {(Fred, berkeley.edu, 20071204), (Fred, stanford.edu, 20071206)}          |
------------------------------------------------------------------------------------------------------------------
--------------------------------------------------
| num_user_visits     | group: chararray | long  |
--------------------------------------------------
|                     | Fred             | 2     |
--------------------------------------------------

Example - Script

This example demonstrates how to use ILLUSTRATE with a Pig script. Note that the script itself should not contain an ILLUSTRATE statement.
grunt> cat visits.txt
Amy     yahoo.com       19990421
Fred    harvard.edu     19991104
Amy     cnn.com 20070218
Frank   nba.com 20070305
Fred    berkeley.edu    20071204
Fred    stanford.edu    20071206

grunt> cat visits.pig
visits = LOAD 'visits.txt' AS (user, url, timestamp);
recent_visits = FILTER visits BY timestamp >= '20071201';
historical_visits = FILTER visits BY timestamp <= '20000101';
DUMP recent_visits;
DUMP historical_visits;
STORE recent_visits INTO 'recent';
STORE historical_visits INTO 'historical';

grunt> exec visits.pig

(Fred,berkeley.edu,20071204)
(Fred,stanford.edu,20071206)

(Amy,yahoo.com,19990421)
(Fred,harvard.edu,19991104)


grunt> illustrate -script visits.pig

------------------------------------------------------------------------
| visits     | user: bytearray | url: bytearray | timestamp: bytearray |
------------------------------------------------------------------------
|            | Amy             | yahoo.com      | 19990421             |
|            | Fred            | stanford.edu   | 20071206             |
------------------------------------------------------------------------
-------------------------------------------------------------------------------
| recent_visits     | user: bytearray | url: bytearray | timestamp: bytearray |
-------------------------------------------------------------------------------
|                   | Fred            | stanford.edu   | 20071206             |
-------------------------------------------------------------------------------
---------------------------------------------------------------------------------------
| Store : recent_visits     | user: bytearray | url: bytearray | timestamp: bytearray |
---------------------------------------------------------------------------------------
|                           | Fred            | stanford.edu   | 20071206             |
---------------------------------------------------------------------------------------
-----------------------------------------------------------------------------------
| historical_visits     | user: bytearray | url: bytearray | timestamp: bytearray |
-----------------------------------------------------------------------------------
|                       | Amy             | yahoo.com      | 19990421             |
-----------------------------------------------------------------------------------
-------------------------------------------------------------------------------------------
| Store : historical_visits     | user: bytearray | url: bytearray | timestamp: bytearray |
-------------------------------------------------------------------------------------------
|                               | Amy             | yahoo.com      | 19990421             |
-------------------------------------------------------------------------------------------

Pig Scripts and MapReduce Job IDs (MapReduce mode only)

Complex Pig scripts often generate many MapReduce jobs. To help you debug a script, Pig prints a summary of the execution that shows which relations (aliases) are mapped to each MapReduce job.
JobId Maps Reduces MaxMapTime MinMapTIme AvgMapTime MaxReduceTime 
    MinReduceTime AvgReduceTime Alias Feature Outputs
job_201004271216_12712 1 1 3 3 3 12 12 12 B,C GROUP_BY,COMBINER
job_201004271216_12713 1 1 3 3 3 12 12 12 D SAMPLER
job_201004271216_12714 1 1 3 3 3 12 12 12 D ORDER_BY,COMBINER 
    hdfs://mymachine.com:9020/tmp/temp743703298/tmp-2019944040,

Pig Statistics

Pig Statistics is a framework for collecting and storing script-level statistics for Pig Latin. Characteristics of Pig Latin scripts and the resulting MapReduce jobs are collected while the script is executed. These statistics are then available for Pig users and tools using Pig (such as Oozie) to retrieve after the job is done.
The new Pig statistics and the existing Hadoop statistics can also be accessed via the Hadoop job history file (and job xml file).

Java API

Several new public classes make it easier for external tools such as Oozie to integrate with Pig statistics.
The Pig statistics are available here: http://pig.apache.org/docs/r0.17.0/api/
The stats classes are in the package: org.apache.pig.tools.pigstats
  • PigStats
  • SimplePigStats
  • EmbeddedPigStats
  • JobStats
  • TezPigScriptStats
  • TezDAGStats
  • TezVertexStats
  • OutputStats
  • InputStats

The PigRunner class mimics the behavior of the Main class but gives users a statistics object back. Optionally, you can call the API with an implementation of progress listener which will be invoked by Pig runtime during the execution.
package org.apache.pig;

public abstract class PigRunner {
    public static PigStats run(String[] args, PigProgressNotificationListener listener)
}

public interface PigProgressNotificationListener extends java.util.EventListener {
    // just before the launch of MR jobs for the script
    public void LaunchStartedNotification(int numJobsToLaunch);
    // number of jobs submitted in a batch
    public void jobsSubmittedNotification(int numJobsSubmitted);
    // a job is started
    public void jobStartedNotification(String assignedJobId);
    // a job is completed successfully
    public void jobFinishedNotification(JobStats jobStats);
    // a job is failed
    public void jobFailedNotification(JobStats jobStats);
    // a user output is completed successfully
    public void outputCompletedNotification(OutputStats outputStats);
    // updates the progress as percentage
    public void progressUpdatedNotification(int progress);
    // the script execution is done
    public void launchCompletedNotification(int numJobsSucceeded);
}
Depends on the type of the pig script, PigRunner.run() returns a particular subclass of PigStats: SimplePigStats(MapReduce/local mode), TezPigScriptStats(Tez/Tez local mode) or EmbeddedPigStats(embedded script). SimplePigStats contains a map of JobStats which capture the stats for each MapReduce job of the Pig script. TezPigScriptStats contains a map of TezDAGStats which capture the stats for each Tez DAG of the Pig script, and TezDAGStats contains a map of TezVertexStats which capture the stats for each vertex within the Tez DAG. Depending on the execution type, EmbeddedPigStats contains a map of SimplePigStats or TezPigScriptStats, which captures the Pig job launched in the embeded script.
If one is running Pig in Tez mode (or both Tez/MapReduce mode), should pass PigTezProgressNotificationListener which extends PigProgressNotificationListener to PigRunner.run() to make sure to get notification in both Tez mode or MapReduce mode.

Job XML

The following entries are included in job conf:
Pig Statistic
Description
pig.script.id
The UUID for the script. All jobs spawned by the script have the same script ID.
pig.script
The base64 encoded script text.
pig.command.line
The command line used to invoke the script.
pig.hadoop.version
The Hadoop version installed.
pig.version
The Pig version used.
pig.input.dirs
A comma-separated list of input directories for the job.
pig.map.output.dirs
A comma-separated list of output directories in the map phase of the job.
pig.reduce.output.dirs
A comma-separated list of output directories in the reduce phase of the job.
pig.parent.jobid
A comma-separated list of parent job ids.
pig.script.features
A list of Pig features used in the script.
pig.job.feature
A list of Pig features used in the job.
pig.alias
The alias associated with the job.

Pig Progress Notification Listener

Pig provides the ability to register a listener to receive event notifications during the execution of a script. Events include MapReduce plan creation, script launch, script progress, script completion, job submit, job start, job completion and job failure.
To register a listener, set the pig.notification.listener parameter to the fully qualified class name of an implementation of org.apache.pig.tools.pigstats.PigProgressNotificationListener. The class must exist on the classpath of the process submitting the Pig job. If the pig.notification.listener.arg parameter is set, the value will be passed to a constructor of the implementing class that takes a single String.

PigUnit

PigUnit is a simple xUnit framework that enables you to easily test your Pig scripts. With PigUnit you can perform unit testing, regression testing, and rapid prototyping. No cluster set up is required if you run Pig in local mode.

Build PigUnit

To compile PigUnit run the command shown below from the Pig trunk. The compile will create the pigunit.jar file.
$pig_trunk ant pigunit-jar   

Run PigUnit

You can run PigUnit using Pig's local mode or mapreduce mode.

Local Mode

PigUnit runs in Pig's local mode by default. Local mode is fast and enables you to use your local file system as the HDFS cluster. Local mode does not require a real cluster but a new local one is created each time.

Other Modes

PigUnit also runs in Pig's mapreduce/tez/tez_local mode. Mapreduce/Tez mode requires you to use a Hadoop cluster and HDFS installation. It is enabled when the Java system property pigunit.exectype is set to specific values (mr/tez/tez_local): e.g. -Dpigunit.exectype=mr or System.getProperties().setProperty("pigunit.exectype", "mr"), which means PigUnit will run in mr mode. The cluster you select to run mr/tez test must be specified in the CLASSPATH (similar to the HADOOP_CONF_DIR variable).

PigUnit Example

Many PigUnit examples are available in the PigUnit tests.
The example included here computes the top N of the most common queries. The Pig script, top_queries.pig, is similar to the Query Phrase Popularity in the Pig tutorial. It expects an input a file of queries and a parameter n (n is 2 in our case in order to do a top 2).
Setting up a test for this script is easy because the argument and the input data are specified by two text arrays. It is the same for the expected output of the script that will be compared to the actual result of the execution of the Pig script.

Java Test

  @Test
  public void testTop2Queries() {
    String[] args = {
        "n=2",
        };
 
    PigTest test = new PigTest("top_queries.pig", args);
 
    String[] input = {
        "yahoo",
        "yahoo",
        "yahoo",
        "twitter",
        "facebook",
        "facebook",
        "linkedin",
    };
 
    String[] output = {
        "(yahoo,3)",
        "(facebook,2)",
    };
 
    test.assertOutput("data", input, "queries_limit", output);
  }

top_queries.pig

data =
    LOAD 'input'
    AS (query:CHARARRAY);
     
queries_group =
    GROUP data
    BY query; 
    
queries_count = 
    FOREACH queries_group 
    GENERATE 
        group AS query, 
        COUNT(data) AS total;
        
queries_ordered =
    ORDER queries_count
    BY total DESC, query;
            
queries_limit =
    LIMIT queries_ordered $n;

STORE queries_limit INTO 'output';

Run

The test can be executed by JUnit (or any other Java testing framework). It requires:
  1. pig.jar
  2. pigunit.jar
The test takes about 25s to run and should pass. In case of error (for example change the parameter n to n=3), the diff of output is displayed:
junit.framework.ComparisonFailure: null expected:<...ahoo,3)
(facebook,2)[]> but was:<...ahoo,3)
(facebook,2)[
(linkedin,1)]>
        at junit.framework.Assert.assertEquals(Assert.java:81)
        at junit.framework.Assert.assertEquals(Assert.java:87)
        at org.apache.pig.pigunit.PigTest.assertEquals(PigTest.java:272)

Mocking

Sometimes you need to mock out the data in specific aliases. Using PigTest's mocking you can override an alias everywhere it is assigned. If you do not know the schema (or want to keep your test dynamic) you can use PigTest.getAliasToSchemaMap() to determine the schema. If you chose to go this route, you should cache the map for the specific script to ensure efficient execution.
  @Test
  public void testTop2Queries() {
    String[] args = {
        "n=2",
        };
 
    PigTest test = new PigTest("top_queries.pig", args);
 
    String[] mockData = {
        "yahoo",
        "yahoo",
        "yahoo",
        "twitter",
        "facebook",
        "facebook",
        "linkedin",
    };
    
    //You should cache the map if you can
    String schema = test.getAliasToSchemaMap().get("data");
    test.mockAlias("data", mockData, schema);
 
    String[] output = {
        "(yahoo,3)",
        "(facebook,2)",
    };
 
    test.assertOutputAnyOrder("queries_limit", output);
  }

Troubleshooting Tips

Common problems you may encounter are discussed below.

Classpath in Mapreduce Mode

When using PigUnit in mapreduce mode, be sure to include the $HADOOP_CONF_DIR of the cluster in your CLASSPATH.
MiniCluster generates one in build/classes.
org.apache.pig.backend.executionengine.ExecException: 
ERROR 4010: Cannot find hadoop configurations in classpath 
(neither hadoop-site.xml nor core-site.xml was found in the classpath).
If you plan to use local mode, please put -x local option in command line

UDF jars Not Found

This error means that you are missing some jars in your test environment.
WARN util.JarManager: Couldn't find the jar for 
org.apache.pig.piggybank.evaluation.string.LOWER, skip it

Storing Data

Pig currently drops all STORE and DUMP commands. You can tell PigUnit to keep the commands and execute the script:
test = new PigTest(PIG_SCRIPT, args);   
test.unoverride("STORE");
test.runScript();

Cache Archive

For cache archive to work, your test environment needs to have the cache archive options specified by Java properties or in an additional XML configuration in its CLASSPATH.
If you use a local cluster, you need to set the required environment variables before starting it:
export LD_LIBRARY_PATH=/home/path/to/lib

Future Enhancements

Improvements and other components based on PigUnit that could be built later.
For example, we could build a PigTestCase and PigTestSuite on top of PigTest to:
  1. Add the notion of workspaces for each test.
  2. Remove the boiler plate code appearing when there is more than one test methods.
  3. Add a standalone utility that reads test configurations and generates a test report.

0 comments:

Post a Comment