Dear Wiki user,

You have subscribed to a wiki page or wiki category on "Pig Wiki" for change 
notification.

The "Pig070LoadStoreHowTo" page has been changed by PradeepKamath.
http://wiki.apache.org/pig/Pig070LoadStoreHowTo?action=diff&rev1=6&rev2=7

--------------------------------------------------

  = Overview =
  This page describes how to go about writing Load functions and Store 
functions using the API available in Pig 0.7.0.
  
+ The main motivation for the changes in Pig 0.7.0 load/store api is to move 
closer to using Hadoop's InputFormat and OutputFormat classes. This way pig 
users/developers can create new LoadFunc and StoreFunc implementation based on 
existing Hadoop InputFormat and OutputFormat classes with minimal code. The 
complexity of reading the data and creating a record will now lie in the 
InputFormat and likewise on the writing end, the complexity of writing will lie 
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.  
+ 
+ '''A general note applicable to both LoadFunc and StoreFunc implementations 
is that the implementation should use the new Hadoop 20 API based classes 
(InputFormat/OutputFormat and related classes) under the 
org.apache.hadoop.mapreduce package instead of the old org.apache.hadoop.mapred 
package.'''
+ 
- == How to implement a Loader ==
+ = How to implement a Loader =
- [[LoadFunc || 
http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/LoadFunc.java?view=markup]]
  abstract class which has the main methods for loading data and for most use 
case it might suffice to extend it. There are 3 other optional interfaces which 
can be implemented to achieve extended functionality:
+ 
[[http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/LoadFunc.java?view=markup
 || LoadFunc]]  abstract class which has the main methods for loading data and 
for most use case it might suffice to extend it. There are 3 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 projections .i.e the pushProjection() method 
is called by Pig to communicate to the loader what exact fields are required in 
the pig script. The loader implementation can choose to honor the request or 
respond that it will not honor the request and return all fields in the data.If 
a loader implementation is able to efficiently return only required fields, it 
should implement !LoadPushDown to improve query performance.
   * !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.
@@ -25, +29 @@

   * 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.  
  
- === Example Implementation ===
+ == 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 new implementation uses an existing 
Hadoop supported !Inputformat - !TextInputFormat as the underlying !InputFormat.
  
  {{{
@@ -138, +142 @@

  }
  }}}
  
- 
- 
- 
- == How to implement a Storer ==
+ = How to implement a Storer =
+ 
[[http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/StoreFunc.java?view=markup
 || StoreFunc]]  abstract class has the main methods for storing data and for 
most use case it might 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 truely optional 
and should only be implemented if metadata needs to stored.
  
+ 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 in 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(): In the new API, 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(): The meaning of putNext() has not changed and is called by Pig 
runtime to write the next tuple of data - in the new API, this is the method 
wherein the implementation will use the the underlying RecordWriter to write 
the Tuple outThe meaning of putNext() has not changed and is called by Pig 
runtime to write the next tuple of data - in the new API, this is the method 
wherein the implementation will use the 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 new 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() > 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");
+         }
+     }
+ 
+     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);
+         }
+     }
+ 
+ }
+ }}}
+ 

Reply via email to