You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by Apache Wiki <wi...@apache.org> on 2010/02/17 22:58:58 UTC

[Pig Wiki] Update of "LoadStoreMigrationGuide" by AshutoshChauhan

Dear Wiki user,

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

The "LoadStoreMigrationGuide" page has been changed by AshutoshChauhan.
The comment on this change is: update about checkSpecs() of OutputFormat.
http://wiki.apache.org/pig/LoadStoreMigrationGuide?action=diff&rev1=21&rev2=22

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

  
  The main motivation for these changes 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.
  
- 
  = LoadFunc Migration =
- The methods in the old !LoadFunc have been split among a !LoadFunc abstract class which has the main methods for loading data and 3 new interfaces 
+ The methods in the old !LoadFunc have been split among a !LoadFunc abstract class which has the main methods for loading data and 3 new interfaces
+ 
   * !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. The other methods are concerned with other types of metadata like partition keys and statistics
   * !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 implementation can chose to honor the request or respond that it will not honor the request and return all fields in the data
   * !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.
  
  The main change is that the new !LoadFunc API is based on a !InputFormat to read the data. Implementations can choose to use existing !InputFormats like !TextInputFormat or implement a new one.
-  
+ 
  == Table mapping old API calls to new API calls in rough order of call sequence ==
- || '''Old Method in !LoadFunc''' || '''Equivalent New Method''' || '''New Class/Interface in which method is present''' || '''Explanation''' ||
+ ||'''Old Method in !LoadFunc''' ||'''Equivalent New Method''' ||'''New Class/Interface in which method is present''' ||'''Explanation''' ||
- || No equivalent method || setUDFContextSignature() || !LoadFunc || 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()||
+ ||No equivalent method ||setUDFContextSignature() ||!LoadFunc ||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() ||
- || No equivalent method || relativeToAbsolutePath() || !LoadFunc || 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 hdfs files and directories. If the load source is something else, loader implementation may choose to override this.||
+ ||No equivalent method ||relativeToAbsolutePath() ||!LoadFunc ||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 hdfs files and directories. If the load source is something else, loader implementation may choose to override this. ||
- || determineSchema() || getSchema() || !LoadMetadata || determineSchema() was used by old code to ask the loader to provide a schema for the data returned by it - the same semantics are now achieved through getSchema() of the !LoadMetadata interface. !LoadMetadata is an optional interface for loaders to implement - if a loader does not implement it, this will indicate to the pig runtime that the loader cannot return a schema for the data ||
+ ||determineSchema() ||getSchema() ||!LoadMetadata ||determineSchema() was used by old code to ask the loader to provide a schema for the data returned by it - the same semantics are now achieved through getSchema() of the !LoadMetadata interface. !LoadMetadata is an optional interface for loaders to implement - if a loader does not implement it, this will indicate to the pig runtime that the loader cannot return a schema for the data ||
- || fieldsToRead() || pushProject() || !LoadPushDown || fieldsToRead() was used by old code to convey to the loader the exact fields required by the pig script -the same semantics are now achieved through pushProject() of the !LoadPushDown interface. !LoadPushDown is an optional interface for loaders to implement - if a loader does not implement it, this will indicate to the pig runtime that the loader is not capable of returning just the required fields and will 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||
+ ||fieldsToRead() ||pushProject() ||!LoadPushDown ||fieldsToRead() was used by old code to convey to the loader the exact fields required by the pig script -the same semantics are now achieved through pushProject() of the !LoadPushDown interface. !LoadPushDown is an optional interface for loaders to implement - if a loader does not implement it, this will indicate to the pig runtime that the loader is not capable of returning just the required fields and will 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 ||
- || No equivalent method || getInputFormat() ||!LoadFunc ||  This method will be called by Pig to get the !InputFormat used by the loader. The methods in the !InputFormat (and underlying !RecordReader) will be called by pig in the same manner (and in the same context) as by Hadoop in a map-reduce java program.||
+ ||No equivalent method ||getInputFormat() ||!LoadFunc ||This method will be called by Pig to get the !InputFormat used by the loader. The methods in the !InputFormat (and underlying !RecordReader) will be called by pig in the same manner (and in the same context) as by Hadoop in a map-reduce java program. ||
- || No equivalent method || setLocation() || !LoadFunc || 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 in mind that this method is called multiple times and should ensure there are no inconsistent side effects due to the multiple calls.||
+ ||No equivalent method ||setLocation() ||!LoadFunc ||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 in mind that this method is called multiple times and should ensure there are no inconsistent side effects due to the multiple calls. ||
- || bindTo() || prepareToRead() || !LoadFunc || bindTo() was the old method which would provide an !InputStream among other things to the !LoadFunc. The !LoadFunc implementation would then read from the !InputStream in getNext(). In the new API, reading of the data is through the !InputFormat provided by the !LoadFunc. So the equivalent call is prepareToRead() wherein 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. ||
+ ||bindTo() ||prepareToRead() ||!LoadFunc ||bindTo() was the old method which would provide an !InputStream among other things to the !LoadFunc. The !LoadFunc implementation would then read from the !InputStream in getNext(). In the new API, reading of the data is through the !InputFormat provided by the !LoadFunc. So the equivalent call is prepareToRead() wherein 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() || getNext() || !LoadFunc || The meaning of getNext() has not changed and is called by Pig runtime to get the next tuple in the data - in the new API, this is the method wherein the implementation will use the the underlying !RecordReader and construct a tuple||
+ ||getNext() ||getNext() ||!LoadFunc ||The meaning of getNext() has not changed and is called by Pig runtime to get the next tuple in the data - in the new API, this is the method wherein the implementation will use the the underlying !RecordReader and construct a tuple ||
- || bytesToInteger(),...bytesToBag() ||  bytesToInteger(),...bytesToBag() || !LoadCaster || The meaning of these methods has not changed and is called by Pig runtime to cast a !DataByteArray fields to the right type when needed. In the new API, a !LoadFunc implementation should give a !LoadCaster object back to pig as the return value of getLoadCaster() method so that it can be used for casting. If a null is returned then casting from !DataByteArray to any other type (implicitly or explicitly) in the pig script will not be possible ||
+ ||bytesToInteger(),...bytesToBag() ||bytesToInteger(),...bytesToBag() ||!LoadCaster ||The meaning of these methods has not changed and is called by Pig runtime to cast a !DataByteArray fields to the right type when needed. In the new API, a !LoadFunc implementation should give a !LoadCaster object back to pig as the return value of getLoadCaster() method so that it can be used for casting. If a null is returned then casting from !DataByteArray to any other type (implicitly or explicitly) in the pig script will not be possible ||
  
+ 
- An example of how a simple !LoadFunc implementation based on old interface can be converted to the new interfaces is shown in the Examples section below. 
+ An example of how a simple !LoadFunc implementation based on old interface can be converted to the new interfaces is shown in the Examples section below.
  
  = StoreFunc Migration =
- 
  The main change is that the new !StoreFunc API is based on a !OutputFormat to read the data. Implementations can choose to use existing !OutputFormat like !TextOutputFormat or implement a new one.
  
  == Table mapping old API calls to new API calls in rough order of call sequence ==
- || '''Old Method in !StoreFunc''' || '''Equivalent New Method''' || '''New Class/Interface in which method is present''' || '''Explanation''' ||
+ ||'''Old Method in !StoreFunc''' ||'''Equivalent New Method''' ||'''New Class/Interface in which method is present''' ||'''Explanation''' ||
- || No equivalent method || setStoreFuncUDFContextSignature() || !StoreFunc || 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.||
+ ||No equivalent method ||setStoreFuncUDFContextSignature() ||!StoreFunc ||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. ||
- || No equivalent method || relToAbsPathForStoreLocation() || !StoreFunc || 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 !LoadFunc (as a static method) which handles this for hdfs files and directories.||
+ ||No equivalent method ||relToAbsPathForStoreLocation() ||!StoreFunc ||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 !LoadFunc (as a static method) which handles this for hdfs files and directories. ||
- || No equivalent method || checkSchema() || !StoreFunc || A Store function should implement this function to check that a given schema describing the data to be written is acceptable to it ||
+ ||No equivalent method ||checkSchema() ||!StoreFunc ||A Store function should implement this function to check that a given schema describing the data to be written is acceptable to it ||
- || No equivalent method || setStoreLocation() || !StoreFunc || 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.||
+ ||No equivalent method ||setStoreLocation() ||!StoreFunc ||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. ||
- || getStorePreparationClass() || getOutputFormat() || !StoreFunc ||In the old API, getStorePreparationClass() was the means by which the implementation could communicate to Pig the !OutputFormat to use for writing - this is now achieved through getOutputFormat(). getOutputFormat() is NOT an optional method and implementation SHOULD provide an !OutputFormat to use. 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. 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.||
+ ||getStorePreparationClass() ||getOutputFormat() ||!StoreFunc ||In the old API, getStorePreparationClass() was the means by which the implementation could communicate to Pig the !OutputFormat to use for writing - this is now achieved through getOutputFormat(). getOutputFormat() is NOT an optional method and implementation SHOULD provide an !OutputFormat to use. 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. 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. ||
- || bindTo() || prepareToWrite() || !StoreFunc || bindTo() was the old method which would provide an !OutputStream among other things to the !StoreFunc. The !StoreFunc implementation would then write to the !OutputStream in putNext(). In the new API, writing of the data is through the !OutputFormat provided by the !StoreFunc. So the equivalent call is prepareToWrite() wherein 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. ||
+ ||bindTo() ||prepareToWrite() ||!StoreFunc ||bindTo() was the old method which would provide an !OutputStream among other things to the !StoreFunc. The !StoreFunc implementation would then write to the !OutputStream in putNext(). In the new API, writing of the data is through the !OutputFormat provided by the !StoreFunc. So the equivalent call is prepareToWrite() wherein 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() || putNext() || !StoreFunc || 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 out ||
+ ||putNext() ||putNext() ||!StoreFunc ||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 out ||
- || finish() || no equivalent method in !StoreFunc - implementations can use close() in !RecordWriter or commitTask in !OutputCommitter || !RecordWriter or !OutputCommitter || finish() has been removed from !StoreFunc since the same semantics can be achieved by !RecordWriter.close() or !OutputCommitter.commitTask() - in the latter case !OutputCommitter.needsTaskCommit() should return true.||
+ ||finish() ||no equivalent method in !StoreFunc - implementations can use close() in !RecordWriter or commitTask in !OutputCommitter ||!RecordWriter or !OutputCommitter ||finish() has been removed from !StoreFunc since the same semantics can be achieved by !RecordWriter.close() or !OutputCommitter.commitTask() - in the latter case !OutputCommitter.needsTaskCommit() should return true. ||
  
  
- 
- An example of how a simple !StoreFunc implementation based on old interface can be converted to the new interfaces is shown in the Examples section below. 
+ An example of how a simple !StoreFunc implementation based on old interface can be converted to the new interfaces is shown in the Examples section below.
  
  = Examples =
- 
  == Loader ==
- 
  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.
  
  === Old Implementation ===
  {{{
  /**
-  * A load function that parses a line of input into fields using a delimiter to set the fields. 
+  * A load function that parses a line of input into fields using a delimiter to set the fields.
   */
  public class SimpleTextLoader extends Utf8StorageConverter {
      protected BufferedPositionedInputStream in = null;
-         
+ 
      long                end            = Long.MAX_VALUE;
      private byte recordDel = '\n';
      private byte fieldDel = '\t';
      private ByteArrayOutputStream mBuf = null;
      private ArrayList<Object> mProtoTuple = null;
      private static final String UTF8 = "UTF-8";
-     
+ 
      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.)
@@ -95, +92 @@

                      Integer.valueOf(delimiter.substring(2)).byteValue();
                  break;
  
-             default:                
+             default:
                  throw new RuntimeException("Unknown delimiter " + delimiter);
              }
-         } else {            
+         } else {
              throw new RuntimeException("PigStorage delimeter must be a single character");
          }
      }
@@ -115, +112 @@

              // to buffer.
              int b = in.read();
              prevByte = (byte)b;
-             
+ 
              if (b == fieldDel) {
                  readField();
              } else if (b == recordDel) {
@@ -135, +132 @@

      public void bindTo(String fileName, BufferedPositionedInputStream in, long offset, long end) throws IOException {
          this.in = in;
          this.end = end;
-         
+ 
          // Since we are not block aligned we throw away the first
          // record and cound on a different instance to read it
          if (offset != 0) {
              getNext();
          }
      }
-     
+ 
      private void readField() {
          if (mProtoTuple == null) mProtoTuple = new ArrayList<Object>();
          if (mBuf.size() == 0) {
              // NULL value
              mProtoTuple.add(null);
          } else {
-             
+ 
              byte[] array = mBuf.toByteArray();
              if (array.length==0)
                  mProtoTuple.add(null);
@@ -163, +160 @@

              DataStorage storage) throws IOException {
          return null;
      }
-         
+ 
      public RequiredFieldResponse fieldsToRead(RequiredFieldList requiredFieldList)
      throws FrontendException {
          // indicate to pig that this loader will return all fields and not just
@@ -175, +172 @@

  === New Implementation ===
  {{{
  public class SimpleTextLoader extends LoadFunc {
-     protected RecordReader in = null;               
+     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.)
@@ -211, +208 @@

                      Integer.valueOf(delimiter.substring(2)).byteValue();
                  break;
  
-             default:                
+             default:
                  throw new RuntimeException("Unknown delimiter " + delimiter);
              }
-         } else {            
+         } else {
              throw new RuntimeException("PigStorage delimeter must be a single character");
          }
      }
@@ -225, +222 @@

              boolean notDone = in.nextKeyValue();
              if (!notDone) {
                  return null;
-             }                                                                                           
+             }
              Text value = (Text) in.getCurrentValue();
              byte[] buf = value.getBytes();
              int len = value.getLength();
@@ -234, +231 @@

              for (int i = 0; i < len; i++) {
                  if (buf[i] == fieldDel) {
                      readField(buf, start, i);
-                     start = i + 1;                   
+                     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, 
+             throw new ExecException(errMsg, errCode,
                      PigException.REMOTE_ENVIRONMENT, e);
          }
-       
+ 
      }
  
      private void readField(byte[] buf, int start, int end) {
@@ -279, +276 @@

      public void setLocation(String location, Job job)
              throws IOException {
          FileInputFormat.setInputPaths(job, location);
-     }    
+     }
  }
- 
  }}}
- 
  == Storer ==
- 
  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.
  
  === Old Implementation ===
  {{{
  public class SimpleTextStorer implements StoreFunc {
-         
+ 
      protected byte recordDel = '\n';
      protected byte fieldDel = '\t';
-     
+ 
      protected static final String UTF8 = "UTF-8";
-     
+ 
      public SimpleTextStorer() {}
  
      public SimpleTextStorer(String delimiter) {
@@ -315, +309 @@

                      Integer.valueOf(delimiter.substring(2)).byteValue();
                  break;
  
-             default:                
+             default:
                  throw new RuntimeException("Unknown delimiter " + delimiter);
              }
-         } else {            
+         } else {
              throw new RuntimeException("PigStorage delimeter must be a single character");
          }
      }
@@ -425, +419 @@

              }
              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);
          }
-         
+ 
          }
      }
  
@@ -465, +459 @@

      /* (non-Javadoc)
       * @see org.apache.pig.StoreFunc#getStorePreparationClass()
       */
-    
+ 
      public Class getStorePreparationClass() throws IOException {
          return null;
      }
@@ -473, +467 @@

  }
  }}}
  === New Implementation ===
- 
  {{{
  public class SimpleTextStorer implements 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) {
@@ -503, +496 @@

                      Integer.valueOf(delimiter.substring(2)).byteValue();
                  break;
  
-             default:                
+             default:
                  throw new RuntimeException("Unknown delimiter " + delimiter);
              }
-         } else {            
+         } else {
              throw new RuntimeException("PigStorage delimeter must be a single character");
          }
      }
@@ -636, +629 @@

              }
              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);
          }
-         
+ 
          }
      }
  
@@ -653, +646 @@

  
      @Override
      public void prepareToWrite(RecordWriter writer) {
-         this.writer = writer;        
+         this.writer = writer;
      }
  
      @Override
@@ -686, +679 @@

      }
  
  }
- 
  }}}
+ == Notes: ==
+ 1) checkSpecs() method of user provided !OutputFormat should be side-effect free, that is it should have consistent behavior if called multiple times. This is so because Pig will call this method multiple times and assumes it is safe to do so.