Details

    • Type: Improvement Improvement
    • Status: Closed
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 0.12.0
    • Component/s: None
    • Labels:
      None
    • Patch Info:
      Patch Available

      Description

      The current AvroStorage implementation has a lot of issues: it requires old versions of Avro, it copies data much more than needed, and it's verbose and complicated. (One pet peeve of mine is that old versions of Avro don't support Snappy compression.)

      I rewrote AvroStorage from scratch to fix these issues. In early tests, the new implementation is significantly faster, and the code is a lot simpler. Rewriting AvroStorage also enabled me to implement support for Trevni (as TrevniStorage).

      I'm opening this ticket to facilitate discussion while I figure out the best way to contribute the changes back to Apache.

      1. PIG-3015-22June2013.diff
        182 kB
        Cheolsoo Park
      2. PIG-3015-20May2013.diff
        179 kB
        Joseph Adler
      3. PIG-3015-12.patch
        16 kB
        Joseph Adler
      4. PIG-3015-doc-2.patch
        13 kB
        Cheolsoo Park
      5. PIG-3015-11.patch
        159 kB
        Cheolsoo Park
      6. with_dates.pig
        0.4 kB
        Joseph Adler
      7. PIG-3015-10.patch
        160 kB
        Cheolsoo Park
      8. PIG-3015-9.patch
        154 kB
        Joseph Adler
      9. PIG-3015-doc.patch
        13 kB
        Cheolsoo Park
      10. PIG-3015-7.patch
        160 kB
        Cheolsoo Park
      11. PIG-3015-6.patch
        160 kB
        Joseph Adler
      12. good.avro
        2.86 MB
        Cheolsoo Park
      13. bad.avro
        2.86 MB
        Cheolsoo Park
      14. Test.java
        2 kB
        Cheolsoo Park
      15. TestInput.java
        1 kB
        Cheolsoo Park
      16. PIG-3015-5.patch
        153 kB
        Joseph Adler
      17. PIG-3015-4.patch
        163 kB
        Cheolsoo Park
      18. PIG-3015-3.patch
        154 kB
        Cheolsoo Park
      19. PIG-3015-2.patch
        153 kB
        Cheolsoo Park

        Issue Links

          Activity

          Hide
          Cheolsoo Park added a comment -

          Hi Joseph,

          Thank you very much for opening the jira. I have recently worked on AvroStorage by myself, and I totally agree with you. Since you already have code to contribute, this is even better.

          As part of re-write, I would also like to propose to migrate AvroStorge from Piggybank to the core Pig. I have 2 reasons for this:

          1. AvroStorage is widely used, so it makes sense to include it in the core Pig rather than in Piggybank.
          2. Until migration is complete, we can maintain both versions (new one in core Pig and old one in Piggybank) to avoid breaking backward compatibility. Another motivation for re-write to me is to clean up funny options that the current AvroStorage has. So I think that it's unavoidable to break backward compatibility.

          I asked this question on the user mailing list a while ago, and nobody disagreed. But please let me know if anyone has objections.

          To start with, I am wondering if you can post your code as a patch to this jira and the review board. Assuming that we're going to move AvroStorage to the core Pig, you can probably create a new package called "org.apache.pig.backend.hadoop.avro" and add your code there. If you could break your patch into smaller pieces and attach them to sub-tasks of this jira, that would be helpful too.

          Please let me know what you think.

          Thanks!

          Show
          Cheolsoo Park added a comment - Hi Joseph, Thank you very much for opening the jira. I have recently worked on AvroStorage by myself, and I totally agree with you. Since you already have code to contribute, this is even better. As part of re-write, I would also like to propose to migrate AvroStorge from Piggybank to the core Pig. I have 2 reasons for this: AvroStorage is widely used, so it makes sense to include it in the core Pig rather than in Piggybank. Until migration is complete, we can maintain both versions (new one in core Pig and old one in Piggybank) to avoid breaking backward compatibility. Another motivation for re-write to me is to clean up funny options that the current AvroStorage has. So I think that it's unavoidable to break backward compatibility. I asked this question on the user mailing list a while ago, and nobody disagreed. But please let me know if anyone has objections. To start with, I am wondering if you can post your code as a patch to this jira and the review board. Assuming that we're going to move AvroStorage to the core Pig, you can probably create a new package called "org.apache.pig.backend.hadoop.avro" and add your code there. If you could break your patch into smaller pieces and attach them to sub-tasks of this jira, that would be helpful too. Please let me know what you think. Thanks!
          Hide
          Joseph Adler added a comment -

          Here's the working version: https://github.com/josephadler/fast-avro-storage

          I can break that up into multiple Jira tickets, though that feels like a lot of extra work; I threw away all the existing code and started from scratch. I do think it's reasonable to separate AvroStorage and TrevniStorage for now (though they are very closely related)

          Show
          Joseph Adler added a comment - Here's the working version: https://github.com/josephadler/fast-avro-storage I can break that up into multiple Jira tickets, though that feels like a lot of extra work; I threw away all the existing code and started from scratch. I do think it's reasonable to separate AvroStorage and TrevniStorage for now (though they are very closely related)
          Hide
          Cheolsoo Park added a comment -

          Thanks for the link.

          You can upload the entire code as a single patch if you prefer. I suggested only because big patches usually take longer to be reviewed and committed, but I will review this one at least.

          Show
          Cheolsoo Park added a comment - Thanks for the link. You can upload the entire code as a single patch if you prefer. I suggested only because big patches usually take longer to be reviewed and committed, but I will review this one at least.
          Hide
          Joseph Adler added a comment -

          Just reading through the discussion on the user list.

          I'll check out trunk, refactor/rename as needed, make sure it passes existing tests, fix bugs, then submit the patches. That will probably take me a few days to do.

          Additionally, I'd like to get a few things correct the first time. Specifically, I'm trying to figure out how to deal with the plethora of possible options for load/store functions. I want to make sure that I cover all the important use cases regarding schemas. Here's the list that I came up with:

          LoadFunc:
          (1) Read the schema from the input file(s)
          (a) Just pick the schema from the most recent file
          (b) Check all the files to make sure the schemas are compatible
          (2) Use a schema manually provided by the user

          StoreFunc:
          (1) Automatically translate the Pig schema to an Avro Schema
          (2) Use a schema manually provided by the user
          (a) Allow the user to name the records and name space
          (b) Automatically pick a record and namespace name

          Show
          Joseph Adler added a comment - Just reading through the discussion on the user list. I'll check out trunk, refactor/rename as needed, make sure it passes existing tests, fix bugs, then submit the patches. That will probably take me a few days to do. Additionally, I'd like to get a few things correct the first time. Specifically, I'm trying to figure out how to deal with the plethora of possible options for load/store functions. I want to make sure that I cover all the important use cases regarding schemas. Here's the list that I came up with: LoadFunc: (1) Read the schema from the input file(s) (a) Just pick the schema from the most recent file (b) Check all the files to make sure the schemas are compatible (2) Use a schema manually provided by the user StoreFunc: (1) Automatically translate the Pig schema to an Avro Schema (2) Use a schema manually provided by the user (a) Allow the user to name the records and name space (b) Automatically pick a record and namespace name
          Hide
          Cheolsoo Park added a comment -

          Hi Joseph,

          The list of options that you described looks like a good start. I think that we should definitely start with a small set of options, but it may be a good idea to keep in mind what options we eventually want to add. So here are my questions:

          LoadFunc

          (a) Just pick the schema from the most recent file
          (b) Check all the files to make sure the schemas are compatible

          I haven't checked out your repository, so please correct me if I am wrong. I assume that your storage converts Avro schema to Pig schema during the load? If so, how do you convert multiple (compatible but different) schemas to one Pig schema? The current storage has an option called 'multiple_schemas' to merge multiple schemas into one.

          (2) Use a schema manually provided by the user

          Do we need this option for LoadFunc? Is this for when the input Avro files do not have an embedded schema?

          Does your storage also have limits on unions and recursive records like the current storage? In fact, recursive records are now supported by PIG-2875.

          How about corrupted files? Currently, we have an option to skip corrupted files (ignore_bad_files) instead of failing on them.

          StoreFunc

          (2) Use a schema manually provided by the user

          The current storage provides three ways of specifying the output schema:

          1. A JSON string can be given (option: schema).
          2. The schema of an existing Avro file (.avro) can be used (option: same).
          3. An Avro schema file (.avsc) can be used (option: schema_file).

          Are you going to support the same?

          How about multiple stores with different output schemas? Currently, the current storage has the 'index' option that allows the user to specify different output schemas for each store.

          Thanks!

          Show
          Cheolsoo Park added a comment - Hi Joseph, The list of options that you described looks like a good start. I think that we should definitely start with a small set of options, but it may be a good idea to keep in mind what options we eventually want to add. So here are my questions: LoadFunc (a) Just pick the schema from the most recent file (b) Check all the files to make sure the schemas are compatible I haven't checked out your repository, so please correct me if I am wrong. I assume that your storage converts Avro schema to Pig schema during the load? If so, how do you convert multiple (compatible but different) schemas to one Pig schema? The current storage has an option called 'multiple_schemas' to merge multiple schemas into one. (2) Use a schema manually provided by the user Do we need this option for LoadFunc? Is this for when the input Avro files do not have an embedded schema? Does your storage also have limits on unions and recursive records like the current storage? In fact, recursive records are now supported by PIG-2875 . How about corrupted files? Currently, we have an option to skip corrupted files (ignore_bad_files) instead of failing on them. StoreFunc (2) Use a schema manually provided by the user The current storage provides three ways of specifying the output schema: A JSON string can be given (option: schema). The schema of an existing Avro file (.avro) can be used (option: same). An Avro schema file (.avsc) can be used (option: schema_file). Are you going to support the same? How about multiple stores with different output schemas? Currently, the current storage has the 'index' option that allows the user to specify different output schemas for each store. Thanks!
          Hide
          Joseph Adler added a comment -

          Before addressing the questions, I wanted to propose a naming schema for the load and store functions. To be consistent with other Pig UDFs, I think it makes more sense to use different function names rather than passing different types of arguments to the UDF. Can I propose something like this:

          LoadFuncs:

          • AvroStorage. May be instantiated with zero, one, or two arguments. If called with no arguments, the function will load the schema from the most recent data file found in the specified path and use that schema. If called with one argument, the argument will be a String that specifies the input schema. The String may either contain the schema definition, may be a URI that refers to the location of the input schema in a file, or may be an example data file from which to read the schema. If two arguments are specified, the first argument refers to the type of the output records (the name of the type) and the second argument may be either a JSON string, a URI for a schema definition file, or a URI for an example file that contains the definition of that type.

          This function does not check schema compatibility of input files or allow recursive schema definitions. Fails when corrupted files are encountered.

          • AvroStorage.AllowRecursive. Same as above, except this function does not check schema compatibility of input files but does allow recursive schema definitions. Recursively defined records are just defined as schemaless tuples in the Pig Schema.
          • AvroStorage.IgnoreCorrupted Same as above, except this function will not allow recursive schema definitions, but will not fail on corrupted input files.
          • AvroStorage.AllowRecursiveAndIgnoreCorrupted Same as above, except this function allows recursive definitions and does not fail on corrupted input files.

          StoreFunc:

          • AvroStorage. May be instantiated with zero, one, or two arguments; the meaning of the arguments can be inferred from how they are specified. If called with no arguments, the function will translate the pig schema to an Avro schema, use a default name for the record types, and not assign a namespace to the records. If called with one argument, the argument will be a String that may specify the output schema, or may specify the record name for the output records. If the string specifies the schema definition, may be a URI that refers to the location of the input schema in a file, or may be an example data file from which to reuse the schema. If two arguments are specified, they may refer to the name and namespace for the output records. Alternately, the first argument may refer to the type of the output records (the name of the schema), and the second argument may be either a JSON string, a URI for a schema definition file, or a URI for an example file that contains the definition of that type.

          Answers to questions:

          LoadFunc 1a: Yes, the storage function will convert avro schemas to pig schemas, and vice versa.

          I haven't tried to convert multiple "compatible but different" schemas to one pig schema. I believe that if you manually supply a schema to the function that is a superset of all the schemas in the input data, the underlying Avro libraries will take care of this for you... though this brings up another question: what does "compatible" mean in this case? Personally, I do not think that the core Pig library should attempt to resolve this problem for users; I think it is best for users to load files with different load functions, cast and rename fields as appropriate in pig code, then take a union of the values. It's possible to miss real (and important) errors if Pig does a lot of type conversions and manipulations under the covers.

          LoadFunc 2: I think this is necessary for a few reasons: It's faster to supply a schema manually (the Pig run time doesn't have to read files from HDFS at planning time to detect the schema). By specifying the schema, you can also specify a subset of fields to de-serialize, reducing the size of the input data. Finally, by specifying a schema manually, you can read a set of files with compatible but different schemas.

          I think PIG-2875 is a design mistake. If I had been involved in the project, I would have argued hard against this. You can't specify a recursive schema in Pig, so why allow users to load files with recursive schemas in Pig? It is possible to load recursively defined records into pig, but that seems like a recipe for confusion and errors. By default, recursive schema definitions should result in an error, or at least a warning message. I'd propose that this be allowed only as an option.

          Storefunc 2a:

          I don't think it's hard to specfiy those three options. It's probably OK for the StoreFunc to allow the user to specify either a schema, a URI that refers to a schema file, or a URI that refers to an example file, then for the function to figure out what the argument means and do the right thing.

          Can you explain the use case for multiple stores with different output schemas? I'm having a hard time understanding why it makes sense to do something complicated like that.

          Show
          Joseph Adler added a comment - Before addressing the questions, I wanted to propose a naming schema for the load and store functions. To be consistent with other Pig UDFs, I think it makes more sense to use different function names rather than passing different types of arguments to the UDF. Can I propose something like this: LoadFuncs: AvroStorage. May be instantiated with zero, one, or two arguments. If called with no arguments, the function will load the schema from the most recent data file found in the specified path and use that schema. If called with one argument, the argument will be a String that specifies the input schema. The String may either contain the schema definition, may be a URI that refers to the location of the input schema in a file, or may be an example data file from which to read the schema. If two arguments are specified, the first argument refers to the type of the output records (the name of the type) and the second argument may be either a JSON string, a URI for a schema definition file, or a URI for an example file that contains the definition of that type. This function does not check schema compatibility of input files or allow recursive schema definitions. Fails when corrupted files are encountered. AvroStorage.AllowRecursive. Same as above, except this function does not check schema compatibility of input files but does allow recursive schema definitions. Recursively defined records are just defined as schemaless tuples in the Pig Schema. AvroStorage.IgnoreCorrupted Same as above, except this function will not allow recursive schema definitions, but will not fail on corrupted input files. AvroStorage.AllowRecursiveAndIgnoreCorrupted Same as above, except this function allows recursive definitions and does not fail on corrupted input files. StoreFunc: AvroStorage. May be instantiated with zero, one, or two arguments; the meaning of the arguments can be inferred from how they are specified. If called with no arguments, the function will translate the pig schema to an Avro schema, use a default name for the record types, and not assign a namespace to the records. If called with one argument, the argument will be a String that may specify the output schema, or may specify the record name for the output records. If the string specifies the schema definition, may be a URI that refers to the location of the input schema in a file, or may be an example data file from which to reuse the schema. If two arguments are specified, they may refer to the name and namespace for the output records. Alternately, the first argument may refer to the type of the output records (the name of the schema), and the second argument may be either a JSON string, a URI for a schema definition file, or a URI for an example file that contains the definition of that type. Answers to questions: LoadFunc 1a: Yes, the storage function will convert avro schemas to pig schemas, and vice versa. I haven't tried to convert multiple "compatible but different" schemas to one pig schema. I believe that if you manually supply a schema to the function that is a superset of all the schemas in the input data, the underlying Avro libraries will take care of this for you... though this brings up another question: what does "compatible" mean in this case? Personally, I do not think that the core Pig library should attempt to resolve this problem for users; I think it is best for users to load files with different load functions, cast and rename fields as appropriate in pig code, then take a union of the values. It's possible to miss real (and important) errors if Pig does a lot of type conversions and manipulations under the covers. LoadFunc 2: I think this is necessary for a few reasons: It's faster to supply a schema manually (the Pig run time doesn't have to read files from HDFS at planning time to detect the schema). By specifying the schema, you can also specify a subset of fields to de-serialize, reducing the size of the input data. Finally, by specifying a schema manually, you can read a set of files with compatible but different schemas. I think PIG-2875 is a design mistake. If I had been involved in the project, I would have argued hard against this. You can't specify a recursive schema in Pig, so why allow users to load files with recursive schemas in Pig? It is possible to load recursively defined records into pig, but that seems like a recipe for confusion and errors. By default, recursive schema definitions should result in an error, or at least a warning message. I'd propose that this be allowed only as an option. Storefunc 2a: I don't think it's hard to specfiy those three options. It's probably OK for the StoreFunc to allow the user to specify either a schema, a URI that refers to a schema file, or a URI that refers to an example file, then for the function to figure out what the argument means and do the right thing. Can you explain the use case for multiple stores with different output schemas? I'm having a hard time understanding why it makes sense to do something complicated like that.
          Hide
          Cheolsoo Park added a comment -

          Hi Joseph,

          1) Using different functions sounds OK to me, but couldn't we handle them via args using CommandLineParser? IMHO, this is simpler and more scalable. Another advantage of using CommandLineParser is that we don't have to infer the meaning of arguments based on the number of arguments. Other built-in storages (e.g. HBaseStorage) use CommandLineParser, so why don't we do the same to provide the universal syntax to the user across the project? Thoughts?

          2) Multiple schema support

          this brings up another question: what does "compatible" mean in this case?

          Please refer to the rules listed in PIG-2579. I did this because it asked by several people. The use case is that people define Avro schemas, but they evolve over time. Since the AvroStorage used to assume that all the input files have the exactly the same schema, they couldn't load them. PIG-2579 was trying to address that inconvenience. Do you think that we should include a similar functionality as an option in the new storage?

          3) Recursive record support

          You can't specify a recursive schema in Pig, so why allow users to load files with recursive schemas in Pig? By default, recursive schema definitions should result in an error, or at least a warning message. I'd propose that this be allowed only as an option.

          Agreed (and guilty ). In fact, this was a feature request from one of my customers. The rationale was that people couldn't change their already-defined recursive schemas, but they wanted to do some processing on non-recursive parts of data. Providing it as an option sound good to me.

          4) Multiple store support

          Can you explain the use case for multiple stores with different output schemas? I'm having a hard time understanding why it makes sense to do something complicated like that.

          I think that I wasn't clear. All I wanted to say is that if we have more than one relation to store in a script, we should be able to do it.

          set1 = load 'input1.txt' using PigStorage() as ( ... );
          store set1 into 'set1' using org.apache.pig.piggybank.storage.avro.AvroStorage('index', '1');
          
          set2 = load 'input2.txt' using PigStorage() as ( ... );
          store set2 into 'set2' using org.apache.pig.piggybank.storage.avro.AvroStorage('index', '2');
          

          The current storage supports multiple stores via the 'index' option. In fact, this is very hacky, and we should get rid of it. Nevertheless, I wanted to know if this will be still supported. On a second thought, I think that your proposal already implies multiple store support because:

          • The output schema will be derived from the Pig schema per store, or
          • The user will specify the output schema per store.

          So I don't see any problem.

          Thanks!

          Show
          Cheolsoo Park added a comment - Hi Joseph, 1) Using different functions sounds OK to me, but couldn't we handle them via args using CommandLineParser? IMHO, this is simpler and more scalable. Another advantage of using CommandLineParser is that we don't have to infer the meaning of arguments based on the number of arguments. Other built-in storages (e.g. HBaseStorage) use CommandLineParser, so why don't we do the same to provide the universal syntax to the user across the project? Thoughts? 2) Multiple schema support this brings up another question: what does "compatible" mean in this case? Please refer to the rules listed in PIG-2579 . I did this because it asked by several people. The use case is that people define Avro schemas, but they evolve over time. Since the AvroStorage used to assume that all the input files have the exactly the same schema, they couldn't load them. PIG-2579 was trying to address that inconvenience. Do you think that we should include a similar functionality as an option in the new storage? 3) Recursive record support You can't specify a recursive schema in Pig, so why allow users to load files with recursive schemas in Pig? By default, recursive schema definitions should result in an error, or at least a warning message. I'd propose that this be allowed only as an option. Agreed (and guilty ). In fact, this was a feature request from one of my customers. The rationale was that people couldn't change their already-defined recursive schemas, but they wanted to do some processing on non-recursive parts of data. Providing it as an option sound good to me. 4) Multiple store support Can you explain the use case for multiple stores with different output schemas? I'm having a hard time understanding why it makes sense to do something complicated like that. I think that I wasn't clear. All I wanted to say is that if we have more than one relation to store in a script, we should be able to do it. set1 = load 'input1.txt' using PigStorage() as ( ... ); store set1 into 'set1' using org.apache.pig.piggybank.storage.avro.AvroStorage('index', '1'); set2 = load 'input2.txt' using PigStorage() as ( ... ); store set2 into 'set2' using org.apache.pig.piggybank.storage.avro.AvroStorage('index', '2'); The current storage supports multiple stores via the 'index' option. In fact, this is very hacky, and we should get rid of it. Nevertheless, I wanted to know if this will be still supported. On a second thought, I think that your proposal already implies multiple store support because: The output schema will be derived from the Pig schema per store, or The user will specify the output schema per store. So I don't see any problem. Thanks!
          Hide
          Mike Naseef added a comment -

          We are very excited about this direction, as we were considering a private re-write to AvroStorage for some of the issues you are addressing. I want to +1 passing the schema into the LoadFunc. The old AvroStorage is very slow and a resource hog when we have a directory hierarchy to scan - even when we set the no_schema_check property. Furthermore, we occasionally have issues with pig jobs picking the old schema when we have a schema update. Manually specifying the schema would fix this (option 1a should cover this as well) and give us more flexibility in defining the data we want pig to pull from a file.

          Show
          Mike Naseef added a comment - We are very excited about this direction, as we were considering a private re-write to AvroStorage for some of the issues you are addressing. I want to +1 passing the schema into the LoadFunc. The old AvroStorage is very slow and a resource hog when we have a directory hierarchy to scan - even when we set the no_schema_check property. Furthermore, we occasionally have issues with pig jobs picking the old schema when we have a schema update. Manually specifying the schema would fix this (option 1a should cover this as well) and give us more flexibility in defining the data we want pig to pull from a file.
          Hide
          Cheolsoo Park added a comment -

          Hi Mike, thanks for your opinion. I agree that passing the input schema into the LoadFunc is a good improvement.

          Please feel free to comment on other issues too. Hopefully, we can resolve as many issues as possible while re-writing AvroStorage.

          Show
          Cheolsoo Park added a comment - Hi Mike, thanks for your opinion. I agree that passing the input schema into the LoadFunc is a good improvement. Please feel free to comment on other issues too. Hopefully, we can resolve as many issues as possible while re-writing AvroStorage.
          Hide
          Alan Gates added a comment -

          +1 for moving it into Pig proper. Avro is a common format and it makes sense to guarantee support for it in Pig.

          Show
          Alan Gates added a comment - +1 for moving it into Pig proper. Avro is a common format and it makes sense to guarantee support for it in Pig.
          Hide
          Joseph Adler added a comment -

          Started working on this now. Two questions

          (1) I'm a new contributor. What's the best way to organize the code within Pig? I have a lot of helper classes and methods, and would like to put different classes in different files to maximize readability. Should I put the helper classes in an existing package (org.apache.pig.impl.builtin seems like the closest match, though still not quite right), create a new package for the helper classes, or do something else? I couldn't find documentation on the best way to do this.

          (2) Here's what I came up with for options: the first argument is either an explicit schema or specifies the record names if a schema is automatically generated. The second argument is a list of options (like in PigStorage):

          <li><code>-namespace</code> Namespace for an automatically generated output schema.</li>
          <li><code>-ignoreerrors</code> Tells function to ignore errors in input files.</li>
          <li><code>-schemafile</code> Specifies URL for avro schema file from which to read the input schema (can be local file, hdfs, url, etc).</li>
          <li><code>-examplefile</code> Specifies URL for avro data file from which to copy the input schema (can be local file, hdfs, url, etc).</li>

          I considered providing an explicit option to provide a schema with a "-schema" flag, but would have had to do something much more complicated to correctly parse the options if an option could include a JSON schema. (Plus, I don't think the meaning of the argument will be ambiguous: it will either be a valid JSON object describing a schema or valid name.)

          Show
          Joseph Adler added a comment - Started working on this now. Two questions (1) I'm a new contributor. What's the best way to organize the code within Pig? I have a lot of helper classes and methods, and would like to put different classes in different files to maximize readability. Should I put the helper classes in an existing package (org.apache.pig.impl.builtin seems like the closest match, though still not quite right), create a new package for the helper classes, or do something else? I couldn't find documentation on the best way to do this. (2) Here's what I came up with for options: the first argument is either an explicit schema or specifies the record names if a schema is automatically generated. The second argument is a list of options (like in PigStorage): <li><code>-namespace</code> Namespace for an automatically generated output schema.</li> <li><code>-ignoreerrors</code> Tells function to ignore errors in input files.</li> <li><code>-schemafile</code> Specifies URL for avro schema file from which to read the input schema (can be local file, hdfs, url, etc).</li> <li><code>-examplefile</code> Specifies URL for avro data file from which to copy the input schema (can be local file, hdfs, url, etc).</li> I considered providing an explicit option to provide a schema with a "-schema" flag, but would have had to do something much more complicated to correctly parse the options if an option could include a JSON schema. (Plus, I don't think the meaning of the argument will be ambiguous: it will either be a valid JSON object describing a schema or valid name.)
          Hide
          Cheolsoo Park added a comment -

          Hi Joseph,

          To answer your questions:

          1) If I am not mistaken, o.a.p.impl.builtin is for internal built-in UDFs. I don't know exactly what your helper classes are like, but would o.a.p.impl.util be a better place?

          Looking at the package tree, I also noticed that there is an *.impl.util package for each sub-component of Pig. So if your helper classes are AvroStorage-specific, you may want to create two new packages called o.a.p.hadoop.avro and o.a.p.hadoop.avro.util, and add AvroStorage to hadoop.avro and helper classes to hadoop.avro.util respectively.

          Please anyone correct me if I am wrong here. I am a new committer.

          2) What you propose sounds good to me.

          Thanks!

          Show
          Cheolsoo Park added a comment - Hi Joseph, To answer your questions: 1) If I am not mistaken, o.a.p.impl.builtin is for internal built-in UDFs. I don't know exactly what your helper classes are like, but would o.a.p.impl.util be a better place? Looking at the package tree, I also noticed that there is an *.impl.util package for each sub-component of Pig. So if your helper classes are AvroStorage-specific, you may want to create two new packages called o.a.p.hadoop.avro and o.a.p.hadoop.avro.util, and add AvroStorage to hadoop.avro and helper classes to hadoop.avro.util respectively. Please anyone correct me if I am wrong here. I am a new committer. 2) What you propose sounds good to me. Thanks!
          Hide
          Joseph Adler added a comment -

          I put the code in o.a.impl.util. Not a big deal to move it later if that's the preferred style.

          Show
          Joseph Adler added a comment - I put the code in o.a.impl.util. Not a big deal to move it later if that's the preferred style.
          Hide
          Russell Jurney added a comment -

          I agree that we should replace the old AvroStorage with this one, and that we should make AvroStorage a builtin.

          However, I don't think its acceptable to break backwards-compatibility with the existing AvroStorage, and having two implementations at once seems confusing. It would be best to extend this implementation with those features required to maintain compatibility with the Piggybank AvroStorage before committing it as a builtin.

          It sounds like you're on top of this, Joe and Chelsoo I'll be a tester.

          Show
          Russell Jurney added a comment - I agree that we should replace the old AvroStorage with this one, and that we should make AvroStorage a builtin. However, I don't think its acceptable to break backwards-compatibility with the existing AvroStorage, and having two implementations at once seems confusing. It would be best to extend this implementation with those features required to maintain compatibility with the Piggybank AvroStorage before committing it as a builtin. It sounds like you're on top of this, Joe and Chelsoo I'll be a tester.
          Hide
          Cheolsoo Park added a comment -

          Hi Russell,

          Thank you very much for offering help.

          However, I don't think its acceptable to break backwards-compatibility with the existing AvroStorage, and having two implementations at once seems confusing. It would be best to extend this implementation with those features required to maintain compatibility with the Piggybank AvroStorage before committing it as a builtin.

          Sure, we can wait until completing the new AvroStorage before commit it, and I won't insist to maintain two versions of AvroStorage if that's confusing to others.

          But given that the new AvroStorage will have different options from the current AvroStorage, it seems unavoidable to introduce some backward incompatibility. For example, Joseph's proposal for new options are very different from those of the current AvroStorage. Would that be acceptable?

          Show
          Cheolsoo Park added a comment - Hi Russell, Thank you very much for offering help. However, I don't think its acceptable to break backwards-compatibility with the existing AvroStorage, and having two implementations at once seems confusing. It would be best to extend this implementation with those features required to maintain compatibility with the Piggybank AvroStorage before committing it as a builtin. Sure, we can wait until completing the new AvroStorage before commit it, and I won't insist to maintain two versions of AvroStorage if that's confusing to others. But given that the new AvroStorage will have different options from the current AvroStorage, it seems unavoidable to introduce some backward incompatibility. For example, Joseph's proposal for new options are very different from those of the current AvroStorage. Would that be acceptable?
          Hide
          Russell Jurney added a comment -

          The existing method of storing to multiple locations is so strange... let's call that part a bug fix? We can enable storing to more than one place without the weird argument workaround using the new outputSchema interface, can't we?

          Show
          Russell Jurney added a comment - The existing method of storing to multiple locations is so strange... let's call that part a bug fix? We can enable storing to more than one place without the weird argument workaround using the new outputSchema interface, can't we?
          Hide
          Joseph Adler added a comment -

          I hate breaking backwards compatibility. (One of the reaons for doing the rewrite is that Avro broke backwards compatibility.) But I think we have some good reasons to do so here:

          • Options for AvroStorage are very different than options for other storage functions in Pig. In moving AvroStorage to builtin, it makes sense for AvroStorage to behave as close as possible to PigStorage, etc.
          • The huge number of crazy options make the code slow and complicated.
          • There are good workarounds for many changes in the options. For example, all the weird stuff about selecting a schema using an index could be easily changed to explicit schema definitions.
          • It gets harder to make changes with time. This is probably the best opportunity to make the options simpler and clearer.
          Show
          Joseph Adler added a comment - I hate breaking backwards compatibility. (One of the reaons for doing the rewrite is that Avro broke backwards compatibility.) But I think we have some good reasons to do so here: Options for AvroStorage are very different than options for other storage functions in Pig. In moving AvroStorage to builtin, it makes sense for AvroStorage to behave as close as possible to PigStorage, etc. The huge number of crazy options make the code slow and complicated. There are good workarounds for many changes in the options. For example, all the weird stuff about selecting a schema using an index could be easily changed to explicit schema definitions. It gets harder to make changes with time. This is probably the best opportunity to make the options simpler and clearer.
          Hide
          Russell Jurney added a comment -

          Actually, I reverse my position. Get this in builtin as soon as possible. Give ppl one pig version to get off the pipe and then we kill the old one

          Ship it.

          Show
          Russell Jurney added a comment - Actually, I reverse my position. Get this in builtin as soon as possible. Give ppl one pig version to get off the pipe and then we kill the old one Ship it.
          Hide
          Joseph Adler added a comment -

          Progress update: I merged in the code, and am now working on test cases. I plan to submit the patches for review later this week.

          Right now, I am working on unit tests for AvroStorage. Because AvroStorage is so complicated, I am trying to find ways to make the test cases easier to manage. (I don't like seeing a single test file with dozens of distinct test cases, and dozens of test data files in one directory). I feel like it's too hard to understand what's being tested and what's not being tested, and too hard to maintain the tests. AvroStorage is very complicated, and I think it's worth changing the test strategy to be more methodical and rigorous. Here's what I'm proposing:

          (1) Test files will be kept in different directories by file type: schema (AVSC) files, raw text input files, json formatted input files, uncompressed avro files, deflate compressed avro files, snappy compressed avro files, uncompressed avro output files, deflate compressed avro output files, snappy compressed output files.
          (2) Test pig scripts will be kept in discrete files, with parameters as file names. I'll modify the test runner to set the runtime parameters correctly. (I think this increases the readability of the test cases and also helps with debugging; you can always type "java -cp pig.jar org.apache.pig.Main -x local -f test_file" to run the files outside the test harness and see what happens)
          (3) I'm thinking about modifying the build process to compile human readable files (in JSON format) into avro files before running the tests.

          What do you guys think?

          Show
          Joseph Adler added a comment - Progress update: I merged in the code, and am now working on test cases. I plan to submit the patches for review later this week. Right now, I am working on unit tests for AvroStorage. Because AvroStorage is so complicated, I am trying to find ways to make the test cases easier to manage. (I don't like seeing a single test file with dozens of distinct test cases, and dozens of test data files in one directory). I feel like it's too hard to understand what's being tested and what's not being tested, and too hard to maintain the tests. AvroStorage is very complicated, and I think it's worth changing the test strategy to be more methodical and rigorous. Here's what I'm proposing: (1) Test files will be kept in different directories by file type: schema (AVSC) files, raw text input files, json formatted input files, uncompressed avro files, deflate compressed avro files, snappy compressed avro files, uncompressed avro output files, deflate compressed avro output files, snappy compressed output files. (2) Test pig scripts will be kept in discrete files, with parameters as file names. I'll modify the test runner to set the runtime parameters correctly. (I think this increases the readability of the test cases and also helps with debugging; you can always type "java -cp pig.jar org.apache.pig.Main -x local -f test_file" to run the files outside the test harness and see what happens) (3) I'm thinking about modifying the build process to compile human readable files (in JSON format) into avro files before running the tests. What do you guys think?
          Hide
          Cheolsoo Park added a comment -

          Hi Joseph,

          Thanks for the update. I support what you're proposing. I appreciate your effort to clean up the code. Just to be clear, I have the following questions:

          Test pig scripts will be kept in discrete files, with parameters as file names. I'll modify the test runner to set the runtime parameters correctly.

          Ideally, all Pig unit test should be written this way. Currently, Pig queries are hard-coded in the test code, which is not very nice. But changing it is going to be a long-term effort. Your changes for this jira will be isolated in TestAvroStorage, won't they? If not, can you please provide more detail? I am just trying to understand the scope of your proposal.

          I'm thinking about modifying the build process to compile human readable files (in JSON format) into avro files before running the tests.

          This will be fully automated in the current framework (ant + junit), so I can run ant test -Dtestcase=TestAvroStorage to run unit test cases, right? One exception for this might be a test case for corrupted Avro files I guess.

          Thanks!

          Show
          Cheolsoo Park added a comment - Hi Joseph, Thanks for the update. I support what you're proposing. I appreciate your effort to clean up the code. Just to be clear, I have the following questions: Test pig scripts will be kept in discrete files, with parameters as file names. I'll modify the test runner to set the runtime parameters correctly. Ideally, all Pig unit test should be written this way. Currently, Pig queries are hard-coded in the test code, which is not very nice. But changing it is going to be a long-term effort. Your changes for this jira will be isolated in TestAvroStorage , won't they? If not, can you please provide more detail? I am just trying to understand the scope of your proposal. I'm thinking about modifying the build process to compile human readable files (in JSON format) into avro files before running the tests. This will be fully automated in the current framework (ant + junit), so I can run ant test -Dtestcase=TestAvroStorage to run unit test cases, right? One exception for this might be a test case for corrupted Avro files I guess. Thanks!
          Hide
          Joseph Adler added a comment -

          Just TestAvroStorage, yes. I'm not trying to rewrite the whole test system, just clean up the AvroStorage tests. And yes, I'd want to either make an exception for corrupted Avro files or have a job that corrupts the files.

          Show
          Joseph Adler added a comment - Just TestAvroStorage, yes. I'm not trying to rewrite the whole test system, just clean up the AvroStorage tests. And yes, I'd want to either make an exception for corrupted Avro files or have a job that corrupts the files.
          Hide
          Joseph Adler added a comment -

          Here is a patch with a working implementation (plus new unit tests and a bash script to generate the test data files; just run the bash script in the test/org/apache/pig/builtin/avro directory to generate all the avro files needed for testing)

          Show
          Joseph Adler added a comment - Here is a patch with a working implementation (plus new unit tests and a bash script to generate the test data files; just run the bash script in the test/org/apache/pig/builtin/avro directory to generate all the avro files needed for testing)
          Hide
          Joseph Adler added a comment -

          Here's the generated patch file.

          Show
          Joseph Adler added a comment - Here's the generated patch file.
          Hide
          Cheolsoo Park added a comment -

          Hi Joseph,

          First of all, thank you so much!

          Secondly, considering the size of the patch, would you mind uploading it to the RB? This will encourage more people to review it.
          https://reviews.apache.org/

          You can choose pig-git to upload a diff file from the github repository.

          Thirdly, I haven't fully read the patch yet and will do once it's uploaded on the RB. But I have a few minor comments as below:

          • Can you please add the Apache license header to every new file?
          • Can you please remove @author tags?
          • Can you please replace System.err.println() with common.logging.log?
          • Our indentation convention is 4 spaces and no tabs. You used 2 spaces, and I see 2 tabs in directory_test.pig.

          Lastly, your bash script probably should be replaced by a python script (or another cross-platform script) because there is an on-going effort of porting Pig to Windows (PIG-2793). In particular, TestAvroStorage is added to the unit test suites, this will be an issue. Please feel free to open a sub-task for converting it to Python if you'd like to get help.

          Show
          Cheolsoo Park added a comment - Hi Joseph, First of all, thank you so much! Secondly, considering the size of the patch, would you mind uploading it to the RB? This will encourage more people to review it. https://reviews.apache.org/ You can choose pig-git to upload a diff file from the github repository. Thirdly, I haven't fully read the patch yet and will do once it's uploaded on the RB. But I have a few minor comments as below: Can you please add the Apache license header to every new file? Can you please remove @author tags? Can you please replace System.err.println() with common.logging.log ? Our indentation convention is 4 spaces and no tabs. You used 2 spaces, and I see 2 tabs in directory_test.pig . Lastly, your bash script probably should be replaced by a python script (or another cross-platform script) because there is an on-going effort of porting Pig to Windows ( PIG-2793 ). In particular, TestAvroStorage is added to the unit test suites, this will be an issue. Please feel free to open a sub-task for converting it to Python if you'd like to get help.
          Hide
          Joseph Adler added a comment -

          I have made all the changes that you suggested (including rewriting the script that builds test cases in Python) and have uploaded the new version to the RB: https://reviews.apache.org/r/8104/

          Show
          Joseph Adler added a comment - I have made all the changes that you suggested (including rewriting the script that builds test cases in Python) and have uploaded the new version to the RB: https://reviews.apache.org/r/8104/
          Hide
          Russell Jurney added a comment -

          Suggest to check out the work Jon did in PIG-2614. One bad record out of a billion killing a job is almost always absurd.

          Show
          Russell Jurney added a comment - Suggest to check out the work Jon did in PIG-2614 . One bad record out of a billion killing a job is almost always absurd.
          Hide
          Joseph Adler added a comment -

          I just took at look at PIG-2614. It looks like the PIG-2614 patch will be compatible with this patch; PIG-2614 simply counts errors as values are read from a LoadFunc. Am I missing something? I'd be happy to drop the option to ignore bad records; I think that would make the options for this function cleaner and easier to understand.

          Show
          Joseph Adler added a comment - I just took at look at PIG-2614 . It looks like the PIG-2614 patch will be compatible with this patch; PIG-2614 simply counts errors as values are read from a LoadFunc. Am I missing something? I'd be happy to drop the option to ignore bad records; I think that would make the options for this function cleaner and easier to understand.
          Hide
          Cheolsoo Park added a comment -

          PIG-2614 lets the user configure the following properties:

          public static final String BAD_RECORD_THRESHOLD_CONF_KEY = "pig.piggybank.storage.avro.bad.record.threshold";
          public static final String BAD_RECORD_MIN_COUNT_CONF_KEY = "pig.piggybank.storage.avro.bad.record.min";
          

          I agree with replacing -ignoreerrors with these properties.

          Show
          Cheolsoo Park added a comment - PIG-2614 lets the user configure the following properties: public static final String BAD_RECORD_THRESHOLD_CONF_KEY = "pig.piggybank.storage.avro.bad.record.threshold" ; public static final String BAD_RECORD_MIN_COUNT_CONF_KEY = "pig.piggybank.storage.avro.bad.record.min" ; I agree with replacing -ignoreerrors with these properties.
          Hide
          Timothy Potter added a comment -

          Can this patch be applied to Pig 0.10?

          Show
          Timothy Potter added a comment - Can this patch be applied to Pig 0.10?
          Hide
          Joseph Adler added a comment -

          Hi Timothy:

          I have not tried the patch with Pig 0.10, but I don't know of any reason why it would not work. Give it a spin and let us know what happens.

          – Joe

          Show
          Joseph Adler added a comment - Hi Timothy: I have not tried the patch with Pig 0.10, but I don't know of any reason why it would not work. Give it a spin and let us know what happens. – Joe
          Hide
          Cheolsoo Park added a comment -

          Hi Joe,

          Sorry for the delay. I made some comments in the RB. Please let me know what you think.

          Thanks!

          Show
          Cheolsoo Park added a comment - Hi Joe, Sorry for the delay. I made some comments in the RB. Please let me know what you think. Thanks!
          Hide
          Joseph Adler added a comment -

          I made most of the recommended changes (thanks for looking this over), and have a follow up question:

          I have always assumed that AvroStorage was designed to be used with Hadoop sequence files that contained a series of records, so I implemented AvroStorage to only work with a file in this format. Are there cases where the highest level schema for a file will be another type? If so... what does that mean for pig? Is there one record per file?

          Here's a specific example: suppose that we have this schema:

          {"name" : "IntArray", "type" : "array", "items" : "int"}

          Suppose that we have 3 files to load, each with this schema, each containing an array of 10 integers. Should we load this into pig as a single bag with 30 integers? A bag containing three bags (each, in turn, containing 10 integers)? Or reject this file entirely?

          Show
          Joseph Adler added a comment - I made most of the recommended changes (thanks for looking this over), and have a follow up question: I have always assumed that AvroStorage was designed to be used with Hadoop sequence files that contained a series of records, so I implemented AvroStorage to only work with a file in this format. Are there cases where the highest level schema for a file will be another type? If so... what does that mean for pig? Is there one record per file? Here's a specific example: suppose that we have this schema: {"name" : "IntArray", "type" : "array", "items" : "int"} Suppose that we have 3 files to load, each with this schema, each containing an array of 10 integers. Should we load this into pig as a single bag with 30 integers? A bag containing three bags (each, in turn, containing 10 integers)? Or reject this file entirely?
          Hide
          Joseph Adler added a comment -

          replacing with revised patch

          Show
          Joseph Adler added a comment - replacing with revised patch
          Hide
          Joseph Adler added a comment -

          Revised patch; reflects comments and suggestions from review board

          Show
          Joseph Adler added a comment - Revised patch; reflects comments and suggestions from review board
          Hide
          Joseph Adler added a comment -

          Revised patch (compiles together all changes)

          Show
          Joseph Adler added a comment - Revised patch (compiles together all changes)
          Hide
          Cheolsoo Park added a comment -

          Hi Joe,

          Thanks for your prompt response!

          To answer your questions,

          I have always assumed that AvroStorage was designed to be used with Hadoop sequence files that contained a series of records, so I implemented AvroStorage to only work with a file in this format. Are there cases where the highest level schema for a file will be another type? If so... what does that mean for pig? Is there one record per file?

          This is a good question, and I see your argument. But this will be very different from what the current AvroStorage does. Currently, a non-record type is automatically wrapped in a tuple. For example, "1" is loaded as (1) in Pig. If a file includes multiple values, they are loaded as multiple tuples as follows:

          avro
          cheolsoo@localhost:~/workspace/avro $java -jar avro-tools-1.5.4.jar getschema multiple_int.avro 
          "int"
          cheolsoo@localhost:~/workspace/avro $java -jar avro-tools-1.5.4.jar tojson multiple_int.avro 
          1
          2
          3
          
          pig
          in = LOAD 'multiple_int.avro' USING org.apache.pig.piggybank.storage.avro.AvroStorage();
          DUMP in;
          (1)
          (2)
          (3)
          

          Agreed that we can tell users that the top-level schema must be a record type, but I am afraid that people might not agree. In my experience, people tend to think that every valid Avro file should be able to be loaded by AvroStorage. Granted, there exist some restrictions (e.g. recursive records and unions), but even these restrictions have been loosened recently. Unless there is a convincing reason to not, I think that we should keep it that way.

          In many cases, people already have data pipeline in place (e.g. Flume produces Avro files => Pig consumes Avro files), and it is not guaranteed that the top-level schema is always a record type.

          Here's a specific example: suppose that we have this schema:
          {"name" : "IntArray", "type" : "array", "items" : "int"}
          Suppose that we have 3 files to load, each with this schema, each containing an array of 10 integers. Should we load this into pig as a single bag with 30 integers? A bag containing three bags (each, in turn, containing 10 integers)? Or reject this file entirely?

          Currently, they are loaded as 3 tuples, and each tuple contains a bag of 10 integers.

          ({(1),(2), ... ,(10)})
          ({(1),(2), ... ,(10)})
          ({(1),(2), ... ,(10)})
          

          Thoughts?

          Show
          Cheolsoo Park added a comment - Hi Joe, Thanks for your prompt response! To answer your questions, I have always assumed that AvroStorage was designed to be used with Hadoop sequence files that contained a series of records, so I implemented AvroStorage to only work with a file in this format. Are there cases where the highest level schema for a file will be another type? If so... what does that mean for pig? Is there one record per file? This is a good question, and I see your argument. But this will be very different from what the current AvroStorage does. Currently, a non-record type is automatically wrapped in a tuple. For example, "1" is loaded as (1) in Pig. If a file includes multiple values, they are loaded as multiple tuples as follows: avro cheolsoo@localhost:~/workspace/avro $java -jar avro-tools-1.5.4.jar getschema multiple_int.avro " int " cheolsoo@localhost:~/workspace/avro $java -jar avro-tools-1.5.4.jar tojson multiple_int.avro 1 2 3 pig in = LOAD 'multiple_int.avro' USING org.apache.pig.piggybank.storage.avro.AvroStorage(); DUMP in; (1) (2) (3) Agreed that we can tell users that the top-level schema must be a record type, but I am afraid that people might not agree. In my experience, people tend to think that every valid Avro file should be able to be loaded by AvroStorage. Granted, there exist some restrictions (e.g. recursive records and unions), but even these restrictions have been loosened recently. Unless there is a convincing reason to not, I think that we should keep it that way. In many cases, people already have data pipeline in place (e.g. Flume produces Avro files => Pig consumes Avro files), and it is not guaranteed that the top-level schema is always a record type. Here's a specific example: suppose that we have this schema: {"name" : "IntArray", "type" : "array", "items" : "int"} Suppose that we have 3 files to load, each with this schema, each containing an array of 10 integers. Should we load this into pig as a single bag with 30 integers? A bag containing three bags (each, in turn, containing 10 integers)? Or reject this file entirely? Currently, they are loaded as 3 tuples, and each tuple contains a bag of 10 integers. ({(1),(2), ... ,(10)}) ({(1),(2), ... ,(10)}) ({(1),(2), ... ,(10)}) Thoughts?
          Hide
          Joseph Adler added a comment -

          I think that approach makes sense; each object in a file should be wrapped in a Tuple. Suppose that a file example.avro contained the data:

          {[1, 2, 3, 4, 5]} {[6, 7, 8, 9, 10]}

          and had this schema:

          {"name" : "IntArray", "type" : "array", "items" : "int"}

          , and we loaded this as

          A = LOAD 'example.avro' USING AvroStorage;

          The bag A would have the Pig schema A:{(IntArray:

          {(int)}

          )}; it would contain two tuples, which would in turn each contain one bag of integers. Does that sound correct? If so, I'll go implement that.

          Show
          Joseph Adler added a comment - I think that approach makes sense; each object in a file should be wrapped in a Tuple. Suppose that a file example.avro contained the data: {[1, 2, 3, 4, 5]} {[6, 7, 8, 9, 10]} and had this schema: {"name" : "IntArray", "type" : "array", "items" : "int"} , and we loaded this as A = LOAD 'example.avro' USING AvroStorage; The bag A would have the Pig schema A:{(IntArray: {(int)} )}; it would contain two tuples, which would in turn each contain one bag of integers. Does that sound correct? If so, I'll go implement that.
          Hide
          Cheolsoo Park added a comment -

          Yes, it does. Thank you, sir!

          Show
          Cheolsoo Park added a comment - Yes, it does. Thank you, sir!
          Hide
          Johannes Schwenk added a comment -

          First of all I want to say many thanks Joseph, for all the great work on this so far! This will be very useful for my work.

          By the way: You certainly know of PIG-2684 about the existing AvroStorage implementation havin problems with <code><alias_name>::</code> prefix that is added by pigs join operations? What is your solution to this issue in the new implementation?

          Show
          Johannes Schwenk added a comment - First of all I want to say many thanks Joseph, for all the great work on this so far! This will be very useful for my work. By the way: You certainly know of PIG-2684 about the existing AvroStorage implementation havin problems with <code><alias_name>::</code> prefix that is added by pigs join operations? What is your solution to this issue in the new implementation?
          Hide
          Joseph Adler added a comment -

          Hi Johannes,

          As you probably know, the Avro specification limits the set of valid characters in names (see http://avro.apache.org/docs/current/spec.html#Names). Names must

          • start with [A-Za-z_]
          • subsequently contain only [A-Za-z0-9_]

          So double colons aren't allowed. PIG-2684 proposes using namespaces as the solution. I think that's a poor choice; namespaces are often used for other purposes. Specifically, names spaces are essential if you are writing complicated data processing software that processes multiple types of avro serialized objects. In my experience, the avro schema and protocol compilers produce much better, more usable code if you use name spaces.

          There are two good workarounds:

          • The Pig user can rename variables in a bag before storing the bag using AvroStorage
          • The Pig user can manually specify the output schema before storing the bag with AvroStorage

          So, here's a specific suggestion:

          • By default, throw an exception if the pig schema contains a name with a double-colon and the user does not specify an output schema
          • Add an option to AvroStorage to transform double colons to something else. (Maybe double underscores? Maybe storing them in the namespace?)

          What do you think?

          Show
          Joseph Adler added a comment - Hi Johannes, As you probably know, the Avro specification limits the set of valid characters in names (see http://avro.apache.org/docs/current/spec.html#Names ). Names must start with [A-Za-z_] subsequently contain only [A-Za-z0-9_] So double colons aren't allowed. PIG-2684 proposes using namespaces as the solution. I think that's a poor choice; namespaces are often used for other purposes. Specifically, names spaces are essential if you are writing complicated data processing software that processes multiple types of avro serialized objects. In my experience, the avro schema and protocol compilers produce much better, more usable code if you use name spaces. There are two good workarounds: The Pig user can rename variables in a bag before storing the bag using AvroStorage The Pig user can manually specify the output schema before storing the bag with AvroStorage So, here's a specific suggestion: By default, throw an exception if the pig schema contains a name with a double-colon and the user does not specify an output schema Add an option to AvroStorage to transform double colons to something else. (Maybe double underscores? Maybe storing them in the namespace?) What do you think?
          Hide
          Joseph Adler added a comment -

          I added support for files that don't have records, added option for dealing with double colons in variable names.

          Show
          Joseph Adler added a comment - I added support for files that don't have records, added option for dealing with double colons in variable names.
          Hide
          Joseph Adler added a comment -

          Added test cases for TrevniStorage (and made sure the test cases all pass)

          Show
          Joseph Adler added a comment - Added test cases for TrevniStorage (and made sure the test cases all pass)
          Hide
          Russell Jurney added a comment -
          Show
          Russell Jurney added a comment - See PIG-3059
          Hide
          Johannes Schwenk added a comment -

          Hi Joseph,

          I agree, that it would be best to have an option for automatic handling of double-colons in names. I think it would be better to use the namespace, since replacing them with underscores could leed to confusion since underscores are an allowed character in names. How do you handle identical names with different namespaces when you load data?

          Show
          Johannes Schwenk added a comment - Hi Joseph, I agree, that it would be best to have an option for automatic handling of double-colons in names. I think it would be better to use the namespace, since replacing them with underscores could leed to confusion since underscores are an allowed character in names. How do you handle identical names with different namespaces when you load data?
          Hide
          Russell Jurney added a comment -

          The patch only applies at p1, not p0. Isn't p0 supposed to work?

          Show
          Russell Jurney added a comment - The patch only applies at p1, not p0. Isn't p0 supposed to work?
          Hide
          Russell Jurney added a comment -

          What Avro version does this expect? I build and I am still on 1.5.3, which seems wrong?

          Show
          Russell Jurney added a comment - What Avro version does this expect? I build and I am still on 1.5.3, which seems wrong?
          Hide
          Russell Jurney added a comment -

          I saw here: https://github.com/josephadler/fast-avro-storage/blob/master/pom.xml

          that I should use this avro jar: http://www.trieuvan.com/apache/avro/avro-1.7.3/avro-src-1.7.3.tar.gz

          Proceeding with testing with workaround, recommend patching pom.xml (although this is in another JIRA, right?)

          Show
          Russell Jurney added a comment - I saw here: https://github.com/josephadler/fast-avro-storage/blob/master/pom.xml that I should use this avro jar: http://www.trieuvan.com/apache/avro/avro-1.7.3/avro-src-1.7.3.tar.gz Proceeding with testing with workaround, recommend patching pom.xml (although this is in another JIRA, right?)
          Hide
          Joseph Adler added a comment -

          I'd recommend using 1.7.3, unless you have a compelling reason to use an older Avro version. There have been several significant bug fixes.

          Show
          Joseph Adler added a comment - I'd recommend using 1.7.3, unless you have a compelling reason to use an older Avro version. There have been several significant bug fixes.
          Hide
          Russell Jurney added a comment -

          AVRO-1218 has me blocked. I'm going to edit build.xml in your patch?

          Show
          Russell Jurney added a comment - AVRO-1218 has me blocked. I'm going to edit build.xml in your patch?
          Hide
          Joseph Adler added a comment -

          Ivy should be able to pull the jar from a maven repo. Do you need to build your own Avro jar from source?

          Show
          Joseph Adler added a comment - Ivy should be able to pull the jar from a maven repo. Do you need to build your own Avro jar from source?
          Hide
          Russell Jurney added a comment -

          Ivy is pulling 1.5.3 atm. Editing.

          Show
          Russell Jurney added a comment - Ivy is pulling 1.5.3 atm. Editing.
          Hide
          Russell Jurney added a comment -

          I see, had to edit ivy/libraries.properties to 1.7.3 instead of 1.5.3. Suggest to include in your patch.

          Show
          Russell Jurney added a comment - I see, had to edit ivy/libraries.properties to 1.7.3 instead of 1.5.3. Suggest to include in your patch.
          Hide
          Cheolsoo Park added a comment -

          The current patch already includes that change:

          diff --git a/ivy/libraries.properties b/ivy/libraries.properties
          index bfbbbc0..1e7fbc8 100644
          --- a/ivy/libraries.properties
          +++ b/ivy/libraries.properties
          @@ -16,7 +16,7 @@
           #These are the versions of our dependencies (in alphabetical order)
           apacheant.version=1.7.1
           automaton.version=1.11-8
          -avro.version=1.5.3
          +avro.version=1.7.2
           commons-beanutils.version=1.7.0
           commons-cli.version=1.0
           commons-codec.version=1.4
          
          Show
          Cheolsoo Park added a comment - The current patch already includes that change: diff --git a/ivy/libraries.properties b/ivy/libraries.properties index bfbbbc0..1e7fbc8 100644 --- a/ivy/libraries.properties +++ b/ivy/libraries.properties @@ -16,7 +16,7 @@ #These are the versions of our dependencies (in alphabetical order) apacheant.version=1.7.1 automaton.version=1.11-8 -avro.version=1.5.3 +avro.version=1.7.2 commons-beanutils.version=1.7.0 commons-cli.version=1.0 commons-codec.version=1.4
          Hide
          Russell Jurney added a comment -

          Strange. I am not able to apply that and get that result. I'll try downloading and applying again. Hmmmmmm...

          Having this problem loading my emails:

          grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/avro-1.7.3.jar
          grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/json-simple-1.1.jar
          grunt> REGISTER /me/Software/pig-trunk/contrib/piggybank/java/piggybank.jar
          grunt>
          grunt> rmf /tmp/sent_counts.avro
          grunt>
          grunt> messages = LOAD '/me/Data/test_inbox' USING AvroStorage();
          2012-12-11 13:01:41,690 [main] ERROR org.apache.pig.tools.grunt.Grunt - ERROR 2998: Unhandled internal error. org/apache/avro/io/DatumReader
          2012-12-11 13:01:41,690 [main] ERROR org.apache.pig.tools.grunt.Grunt - java.lang.NoClassDefFoundError: org/apache/avro/io/DatumReader
          at java.lang.Class.forName0(Native Method)
          at java.lang.Class.forName(Class.java:247)
          at org.apache.pig.impl.PigContext.resolveClassName(PigContext.java:510)
          at org.apache.pig.parser.LogicalPlanBuilder.validateFuncSpec(LogicalPlanBuilder.java:1206)
          at org.apache.pig.parser.LogicalPlanBuilder.buildFuncSpec(LogicalPlanBuilder.java:1194)
          at org.apache.pig.parser.LogicalPlanGenerator.func_clause(LogicalPlanGenerator.java:4766)
          at org.apache.pig.parser.LogicalPlanGenerator.load_clause(LogicalPlanGenerator.java:3183)
          at org.apache.pig.parser.LogicalPlanGenerator.op_clause(LogicalPlanGenerator.java:1315)
          at org.apache.pig.parser.LogicalPlanGenerator.general_statement(LogicalPlanGenerator.java:799)
          at org.apache.pig.parser.LogicalPlanGenerator.statement(LogicalPlanGenerator.java:517)
          at org.apache.pig.parser.LogicalPlanGenerator.query(LogicalPlanGenerator.java:392)
          at org.apache.pig.parser.QueryParserDriver.parse(QueryParserDriver.java:184)
          at org.apache.pig.PigServer$Graph.validateQuery(PigServer.java:1581)
          at org.apache.pig.PigServer$Graph.registerQuery(PigServer.java:1554)
          at org.apache.pig.PigServer.registerQuery(PigServer.java:526)
          at org.apache.pig.tools.grunt.GruntParser.processPig(GruntParser.java:991)
          at org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptParser.java:412)
          at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:194)
          at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:170)
          at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:69)
          at org.apache.pig.Main.run(Main.java:535)
          at org.apache.pig.Main.main(Main.java:154)
          at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
          at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
          at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
          at java.lang.reflect.Method.invoke(Method.java:597)
          at org.apache.hadoop.util.RunJar.main(RunJar.java:156)
          Caused by: java.lang.ClassNotFoundException: org.apache.avro.io.DatumReader
          at java.net.URLClassLoader$1.run(URLClassLoader.java:202)
          at java.security.AccessController.doPrivileged(Native Method)
          at java.net.URLClassLoader.findClass(URLClassLoader.java:190)
          at java.lang.ClassLoader.loadClass(ClassLoader.java:306)
          at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:301)
          at java.lang.ClassLoader.loadClass(ClassLoader.java:247)
          ... 27 more

          Details also at logfile: /private/tmp/pig_1355258041691.log

          The schema is:

          Avro Schema: {"fields": [

          {"doc": "", "type": ["null", "string"], "name": "message_id"}

          ,

          {"doc": "", "type": ["null", "string"], "name": "thread_id"}

          ,

          {"type": ["string", "null"], "name": "in_reply_to"}

          ,

          {"type": ["string", "null"], "name": "subject"}

          ,

          {"type": ["string", "null"], "name": "body"}

          ,

          {"type": ["string", "null"], "name": "date"}

          , {"doc": "", "type": ["null", {"items": ["null", {"fields": [

          {"doc": "", "type": ["null", "string"], "name": "real_name"}

          ,

          {"doc": "", "type": ["null", "string"], "name": "address"}

          ], "type": "record", "name": "from"}], "type": "array"}], "name": "froms"}, {"doc": "", "type": ["null", {"items": ["null", {"fields": [

          {"doc": "", "type": ["null", "string"], "name": "real_name"}

          ,

          {"doc": "", "type": ["null", "string"], "name": "address"}

          ], "type": "record", "name": "to"}], "type": "array"}], "name": "tos"}, {"doc": "", "type": ["null", {"items": ["null", {"fields": [

          {"doc": "", "type": ["null", "string"], "name": "real_name"}

          ,

          {"doc": "", "type": ["null", "string"], "name": "address"}

          ], "type": "record", "name": "cc"}], "type": "array"}], "name": "ccs"}, {"doc": "", "type": ["null", {"items": ["null", {"fields": [

          {"doc": "", "type": ["null", "string"], "name": "real_name"}

          ,

          {"doc": "", "type": ["null", "string"], "name": "address"}

          ], "type": "record", "name": "bcc"}], "type": "array"}], "name": "bccs"}, {"doc": "", "type": ["null", {"items": ["null", {"fields": [

          {"doc": "", "type": ["null", "string"], "name": "real_name"}

          ,

          {"doc": "", "type": ["null", "string"], "name": "address"}

          ], "type": "record", "name": "reply_to"}], "type": "array"}], "name": "reply_tos"}], "type": "record", "name": "Email"}

          And just to get really meta... here is a JSON output of my Avro serialized emails... one from this list:

          {u'bccs': None,
          u'body': u'\r\n [ https://issues.apache.org/jira/browse/PIG-2661?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13447110#comment-13447110 ] \r\n\r\nDmitriy V. Ryaboy commented on PIG-2661:\r\n----------------------------------------\r\n\r\nor, you know, stick a key in MemCache. #whyishadoopsohard\r\n \r\n> Pig uses an extra job for loading data in Pigmix L9\r\n> ---------------------------------------------------\r\n>\r\n> Key: PIG-2661\r\n> URL: https://issues.apache.org/jira/browse/PIG-2661\r\n> Project: Pig\r\n> Issue Type: Improvement\r\n> Affects Versions: 0.9.0\r\n> Reporter: Jie Li\r\n> Assignee: Jie Li\r\n> Attachments: PIG-2661.0.patch, PIG-2661.1.patch, PIG-2661.2.patch, PIG-2661.3.patch, PIG-2661.4.patch, PIG-2661.5.patch, PIG-2661.6.patch, PIG-2661.7.patch, PIG-2661.8.patch, PIG-2661.plan.txt\r\n>\r\n>\r\n> See https://issues.apache.org/jira/browse/PIG-200?focusedCommentId=13260155&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13260155\r\n\r\n--\r\nThis message is automatically generated by JIRA.\r\nIf you think it was sent incorrectly, please contact your JIRA administrators\r\nFor more information on JIRA, see: http://www.atlassian.com/software/jira\r\n',
          u'ccs': None,
          u'date': u'2012-09-03T15:22:07',
          u'froms': [

          {u'address': u'jira@apache.org', u'real_name': u'Dmitriy V. Ryaboy (JIRA)'}

          ],
          u'in_reply_to': u'52638020.7802.1335237294825.JavaMail.tomcat@hel.zones.apache.org',
          u'message_id': u'762728135.29484.1346646127701.JavaMail.jiratomcat@arcas',
          u'reply_tos': [

          {u'address': u'dev@pig.apache.org', u'real_name': None}

          ],
          u'subject': u'[jira] [Commented] (PIG-2661) Pig uses an extra job for loading\r\n data in Pigmix L9',
          u'thread_id': u'1400097807569590118',
          u'tos': [

          {u'address': u'pig-dev@hadoop.apache.org', u'real_name': None}

          ]}

          Show
          Russell Jurney added a comment - Strange. I am not able to apply that and get that result. I'll try downloading and applying again. Hmmmmmm... Having this problem loading my emails: grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/avro-1.7.3.jar grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/json-simple-1.1.jar grunt> REGISTER /me/Software/pig-trunk/contrib/piggybank/java/piggybank.jar grunt> grunt> rmf /tmp/sent_counts.avro grunt> grunt> messages = LOAD '/me/Data/test_inbox' USING AvroStorage(); 2012-12-11 13:01:41,690 [main] ERROR org.apache.pig.tools.grunt.Grunt - ERROR 2998: Unhandled internal error. org/apache/avro/io/DatumReader 2012-12-11 13:01:41,690 [main] ERROR org.apache.pig.tools.grunt.Grunt - java.lang.NoClassDefFoundError: org/apache/avro/io/DatumReader at java.lang.Class.forName0(Native Method) at java.lang.Class.forName(Class.java:247) at org.apache.pig.impl.PigContext.resolveClassName(PigContext.java:510) at org.apache.pig.parser.LogicalPlanBuilder.validateFuncSpec(LogicalPlanBuilder.java:1206) at org.apache.pig.parser.LogicalPlanBuilder.buildFuncSpec(LogicalPlanBuilder.java:1194) at org.apache.pig.parser.LogicalPlanGenerator.func_clause(LogicalPlanGenerator.java:4766) at org.apache.pig.parser.LogicalPlanGenerator.load_clause(LogicalPlanGenerator.java:3183) at org.apache.pig.parser.LogicalPlanGenerator.op_clause(LogicalPlanGenerator.java:1315) at org.apache.pig.parser.LogicalPlanGenerator.general_statement(LogicalPlanGenerator.java:799) at org.apache.pig.parser.LogicalPlanGenerator.statement(LogicalPlanGenerator.java:517) at org.apache.pig.parser.LogicalPlanGenerator.query(LogicalPlanGenerator.java:392) at org.apache.pig.parser.QueryParserDriver.parse(QueryParserDriver.java:184) at org.apache.pig.PigServer$Graph.validateQuery(PigServer.java:1581) at org.apache.pig.PigServer$Graph.registerQuery(PigServer.java:1554) at org.apache.pig.PigServer.registerQuery(PigServer.java:526) at org.apache.pig.tools.grunt.GruntParser.processPig(GruntParser.java:991) at org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptParser.java:412) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:194) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:170) at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:69) at org.apache.pig.Main.run(Main.java:535) at org.apache.pig.Main.main(Main.java:154) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) at java.lang.reflect.Method.invoke(Method.java:597) at org.apache.hadoop.util.RunJar.main(RunJar.java:156) Caused by: java.lang.ClassNotFoundException: org.apache.avro.io.DatumReader at java.net.URLClassLoader$1.run(URLClassLoader.java:202) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:190) at java.lang.ClassLoader.loadClass(ClassLoader.java:306) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:301) at java.lang.ClassLoader.loadClass(ClassLoader.java:247) ... 27 more Details also at logfile: /private/tmp/pig_1355258041691.log The schema is: Avro Schema: {"fields": [ {"doc": "", "type": ["null", "string"], "name": "message_id"} , {"doc": "", "type": ["null", "string"], "name": "thread_id"} , {"type": ["string", "null"], "name": "in_reply_to"} , {"type": ["string", "null"], "name": "subject"} , {"type": ["string", "null"], "name": "body"} , {"type": ["string", "null"], "name": "date"} , {"doc": "", "type": ["null", {"items": ["null", {"fields": [ {"doc": "", "type": ["null", "string"], "name": "real_name"} , {"doc": "", "type": ["null", "string"], "name": "address"} ], "type": "record", "name": "from"}], "type": "array"}], "name": "froms"}, {"doc": "", "type": ["null", {"items": ["null", {"fields": [ {"doc": "", "type": ["null", "string"], "name": "real_name"} , {"doc": "", "type": ["null", "string"], "name": "address"} ], "type": "record", "name": "to"}], "type": "array"}], "name": "tos"}, {"doc": "", "type": ["null", {"items": ["null", {"fields": [ {"doc": "", "type": ["null", "string"], "name": "real_name"} , {"doc": "", "type": ["null", "string"], "name": "address"} ], "type": "record", "name": "cc"}], "type": "array"}], "name": "ccs"}, {"doc": "", "type": ["null", {"items": ["null", {"fields": [ {"doc": "", "type": ["null", "string"], "name": "real_name"} , {"doc": "", "type": ["null", "string"], "name": "address"} ], "type": "record", "name": "bcc"}], "type": "array"}], "name": "bccs"}, {"doc": "", "type": ["null", {"items": ["null", {"fields": [ {"doc": "", "type": ["null", "string"], "name": "real_name"} , {"doc": "", "type": ["null", "string"], "name": "address"} ], "type": "record", "name": "reply_to"}], "type": "array"}], "name": "reply_tos"}], "type": "record", "name": "Email"} And just to get really meta... here is a JSON output of my Avro serialized emails... one from this list: {u'bccs': None, u'body': u'\r\n [ https://issues.apache.org/jira/browse/PIG-2661?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13447110#comment-13447110 ] \r\n\r\nDmitriy V. Ryaboy commented on PIG-2661 :\r\n----------------------------------------\r\n\r\nor, you know, stick a key in MemCache. #whyishadoopsohard\r\n \r\n> Pig uses an extra job for loading data in Pigmix L9\r\n> ---------------------------------------------------\r\n>\r\n> Key: PIG-2661 \r\n> URL: https://issues.apache.org/jira/browse/PIG-2661\r\n > Project: Pig\r\n> Issue Type: Improvement\r\n> Affects Versions: 0.9.0\r\n> Reporter: Jie Li\r\n> Assignee: Jie Li\r\n> Attachments: PIG-2661 .0.patch, PIG-2661 .1.patch, PIG-2661 .2.patch, PIG-2661 .3.patch, PIG-2661 .4.patch, PIG-2661 .5.patch, PIG-2661 .6.patch, PIG-2661 .7.patch, PIG-2661 .8.patch, PIG-2661 .plan.txt\r\n>\r\n>\r\n> See https://issues.apache.org/jira/browse/PIG-200?focusedCommentId=13260155&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13260155\r\n\r\n--\r\nThis message is automatically generated by JIRA.\r\nIf you think it was sent incorrectly, please contact your JIRA administrators\r\nFor more information on JIRA, see: http://www.atlassian.com/software/jira\r\n ', u'ccs': None, u'date': u'2012-09-03T15:22:07', u'froms': [ {u'address': u'jira@apache.org', u'real_name': u'Dmitriy V. Ryaboy (JIRA)'} ], u'in_reply_to': u'52638020.7802.1335237294825.JavaMail.tomcat@hel.zones.apache.org', u'message_id': u'762728135.29484.1346646127701.JavaMail.jiratomcat@arcas', u'reply_tos': [ {u'address': u'dev@pig.apache.org', u'real_name': None} ], u'subject': u' [jira] [Commented] ( PIG-2661 ) Pig uses an extra job for loading\r\n data in Pigmix L9', u'thread_id': u'1400097807569590118', u'tos': [ {u'address': u'pig-dev@hadoop.apache.org', u'real_name': None} ]}
          Hide
          Russell Jurney added a comment -

          I got a clean clone of trunk and applied the patch, then attempt to load the file again:

          grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/avro-1.7.2.jar
          grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/avro-ipc-1.7.2.jar
          grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/avro-mapred-1.7.2.jar
          grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/avro-tools-1.7.2.jar
          grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/trevni-avro-1.7.2.jar
          grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/json-simple-1.1.jar
          grunt> REGISTER /me/Software/pig-trunk/contrib/piggybank/java/piggybank.jar
          grunt> 
          grunt> 
          grunt> messages = LOAD '/me/Data/test_inbox' USING AvroStorage();
          2012-12-11 15:32:36,989 [main] ERROR org.apache.pig.tools.grunt.Grunt - ERROR 2998: Unhandled internal error. org/apache/avro/io/DatumReader
          2012-12-11 15:32:36,989 [main] ERROR org.apache.pig.tools.grunt.Grunt - java.lang.NoClassDefFoundError: org/apache/avro/io/DatumReader
          	at java.lang.Class.forName0(Native Method)
          	at java.lang.Class.forName(Class.java:247)
          	at org.apache.pig.impl.PigContext.resolveClassName(PigContext.java:510)
          	at org.apache.pig.parser.LogicalPlanBuilder.validateFuncSpec(LogicalPlanBuilder.java:1206)
          	at org.apache.pig.parser.LogicalPlanBuilder.buildFuncSpec(LogicalPlanBuilder.java:1194)
          	at org.apache.pig.parser.LogicalPlanGenerator.func_clause(LogicalPlanGenerator.java:4766)
          	at org.apache.pig.parser.LogicalPlanGenerator.load_clause(LogicalPlanGenerator.java:3183)
          	at org.apache.pig.parser.LogicalPlanGenerator.op_clause(LogicalPlanGenerator.java:1315)
          	at org.apache.pig.parser.LogicalPlanGenerator.general_statement(LogicalPlanGenerator.java:799)
          	at org.apache.pig.parser.LogicalPlanGenerator.statement(LogicalPlanGenerator.java:517)
          	at org.apache.pig.parser.LogicalPlanGenerator.query(LogicalPlanGenerator.java:392)
          	at org.apache.pig.parser.QueryParserDriver.parse(QueryParserDriver.java:184)
          	at org.apache.pig.PigServer$Graph.validateQuery(PigServer.java:1581)
          	at org.apache.pig.PigServer$Graph.registerQuery(PigServer.java:1554)
          	at org.apache.pig.PigServer.registerQuery(PigServer.java:526)
          	at org.apache.pig.tools.grunt.GruntParser.processPig(GruntParser.java:991)
          	at org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptParser.java:412)
          	at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:194)
          	at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:170)
          	at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:69)
          	at org.apache.pig.Main.run(Main.java:535)
          	at org.apache.pig.Main.main(Main.java:154)
          	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
          	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
          	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
          	at java.lang.reflect.Method.invoke(Method.java:597)
          	at org.apache.hadoop.util.RunJar.main(RunJar.java:156)
          Caused by: java.lang.ClassNotFoundException: org.apache.avro.io.DatumReader
          	at java.net.URLClassLoader$1.run(URLClassLoader.java:202)
          	at java.security.AccessController.doPrivileged(Native Method)
          	at java.net.URLClassLoader.findClass(URLClassLoader.java:190)
          	at java.lang.ClassLoader.loadClass(ClassLoader.java:306)
          	at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:301)
          	at java.lang.ClassLoader.loadClass(ClassLoader.java:247)
          	... 27 more
          
          Show
          Russell Jurney added a comment - I got a clean clone of trunk and applied the patch, then attempt to load the file again: grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/avro-1.7.2.jar grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/avro-ipc-1.7.2.jar grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/avro-mapred-1.7.2.jar grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/avro-tools-1.7.2.jar grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/trevni-avro-1.7.2.jar grunt> REGISTER /me/Software/pig-trunk/build/ivy/lib/Pig/json-simple-1.1.jar grunt> REGISTER /me/Software/pig-trunk/contrib/piggybank/java/piggybank.jar grunt> grunt> grunt> messages = LOAD '/me/Data/test_inbox' USING AvroStorage(); 2012-12-11 15:32:36,989 [main] ERROR org.apache.pig.tools.grunt.Grunt - ERROR 2998: Unhandled internal error. org/apache/avro/io/DatumReader 2012-12-11 15:32:36,989 [main] ERROR org.apache.pig.tools.grunt.Grunt - java.lang.NoClassDefFoundError: org/apache/avro/io/DatumReader at java.lang.Class.forName0(Native Method) at java.lang.Class.forName(Class.java:247) at org.apache.pig.impl.PigContext.resolveClassName(PigContext.java:510) at org.apache.pig.parser.LogicalPlanBuilder.validateFuncSpec(LogicalPlanBuilder.java:1206) at org.apache.pig.parser.LogicalPlanBuilder.buildFuncSpec(LogicalPlanBuilder.java:1194) at org.apache.pig.parser.LogicalPlanGenerator.func_clause(LogicalPlanGenerator.java:4766) at org.apache.pig.parser.LogicalPlanGenerator.load_clause(LogicalPlanGenerator.java:3183) at org.apache.pig.parser.LogicalPlanGenerator.op_clause(LogicalPlanGenerator.java:1315) at org.apache.pig.parser.LogicalPlanGenerator.general_statement(LogicalPlanGenerator.java:799) at org.apache.pig.parser.LogicalPlanGenerator.statement(LogicalPlanGenerator.java:517) at org.apache.pig.parser.LogicalPlanGenerator.query(LogicalPlanGenerator.java:392) at org.apache.pig.parser.QueryParserDriver.parse(QueryParserDriver.java:184) at org.apache.pig.PigServer$Graph.validateQuery(PigServer.java:1581) at org.apache.pig.PigServer$Graph.registerQuery(PigServer.java:1554) at org.apache.pig.PigServer.registerQuery(PigServer.java:526) at org.apache.pig.tools.grunt.GruntParser.processPig(GruntParser.java:991) at org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptParser.java:412) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:194) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:170) at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:69) at org.apache.pig.Main.run(Main.java:535) at org.apache.pig.Main.main(Main.java:154) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) at java.lang.reflect.Method.invoke(Method.java:597) at org.apache.hadoop.util.RunJar.main(RunJar.java:156) Caused by: java.lang.ClassNotFoundException: org.apache.avro.io.DatumReader at java.net.URLClassLoader$1.run(URLClassLoader.java:202) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:190) at java.lang.ClassLoader.loadClass(ClassLoader.java:306) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:301) at java.lang.ClassLoader.loadClass(ClassLoader.java:247) ... 27 more
          Hide
          Russell Jurney added a comment -

          Pardon the comment blitz:

          Russells-MacBook-Pro:pig-trunk rjurney$ grep -iR 'org.apache.avro.io.DatumReader' *
          PIG-3015.patch:+import org.apache.avro.io.DatumReader;
          PIG-3015.patch:+import org.apache.avro.io.DatumReader;
          Binary file build/classes/org/apache/pig/builtin/AvroStorage.class matches
          Binary file build/classes/org/apache/pig/impl/util/avro/AvroArrayReader.class matches
          Binary file build/classes/org/apache/pig/impl/util/avro/AvroRecordReader.class matches
          Binary file build/ivy/lib/Pig/avro-1.7.2.jar matches
          Binary file build/ivy/lib/Pig/avro-tools-1.7.2.jar matches
          Binary file build/pig-0.12.0-SNAPSHOT-withdependencies.jar matches
          Binary file contrib/piggybank/java/build/classes/org/apache/pig/piggybank/storage/avro/AvroStorage.class matches
          Binary file contrib/piggybank/java/build/classes/org/apache/pig/piggybank/storage/avro/PigAvroRecordReader.class matches
          Binary file pig.jar matches
          src/org/apache/pig/impl/util/avro/AvroArrayReader.java:import org.apache.avro.io.DatumReader;
          src/org/apache/pig/impl/util/avro/AvroRecordReader.java:import org.apache.avro.io.DatumReader;
          

          I'm puzzled... checking my classpath.

          Show
          Russell Jurney added a comment - Pardon the comment blitz: Russells-MacBook-Pro:pig-trunk rjurney$ grep -iR 'org.apache.avro.io.DatumReader' * PIG-3015.patch:+import org.apache.avro.io.DatumReader; PIG-3015.patch:+import org.apache.avro.io.DatumReader; Binary file build/classes/org/apache/pig/builtin/AvroStorage.class matches Binary file build/classes/org/apache/pig/impl/util/avro/AvroArrayReader.class matches Binary file build/classes/org/apache/pig/impl/util/avro/AvroRecordReader.class matches Binary file build/ivy/lib/Pig/avro-1.7.2.jar matches Binary file build/ivy/lib/Pig/avro-tools-1.7.2.jar matches Binary file build/pig-0.12.0-SNAPSHOT-withdependencies.jar matches Binary file contrib/piggybank/java/build/classes/org/apache/pig/piggybank/storage/avro/AvroStorage.class matches Binary file contrib/piggybank/java/build/classes/org/apache/pig/piggybank/storage/avro/PigAvroRecordReader.class matches Binary file pig.jar matches src/org/apache/pig/impl/util/avro/AvroArrayReader.java:import org.apache.avro.io.DatumReader; src/org/apache/pig/impl/util/avro/AvroRecordReader.java:import org.apache.avro.io.DatumReader; I'm puzzled... checking my classpath.
          Hide
          Russell Jurney added a comment -

          Nothing I do, even after mosts tests pass on the latest patch, can get around this error. I'll check back.

          Show
          Russell Jurney added a comment - Nothing I do, even after mosts tests pass on the latest patch, can get around this error. I'll check back.
          Hide
          Cheolsoo Park added a comment -

          Hi Joe,

          Can you please add the missing files to the patch?

          Exception in thread "main" java.io.FileNotFoundException: data/json/recordsWithDoubleUnderscores.json (No such file or directory)
          Exception in thread "main" java.io.FileNotFoundException: data/json/arrays.json (No such file or directory)
          Exception in thread "main" java.io.FileNotFoundException: data/json/arraysAsOutputByPig.json (No such file or directory)
          

          I can't run your test cases.

          Show
          Cheolsoo Park added a comment - Hi Joe, Can you please add the missing files to the patch? Exception in thread "main" java.io.FileNotFoundException: data/json/recordsWithDoubleUnderscores.json (No such file or directory) Exception in thread "main" java.io.FileNotFoundException: data/json/arrays.json (No such file or directory) Exception in thread "main" java.io.FileNotFoundException: data/json/arraysAsOutputByPig.json (No such file or directory) I can't run your test cases.
          Hide
          Joseph Adler added a comment -

          My apologies; forgot to add those to the patch. Replaced the patch version.

          Show
          Joseph Adler added a comment - My apologies; forgot to add those to the patch. Replaced the patch version.
          Hide
          Cheolsoo Park added a comment -

          Hi Joe,

          I found a few errors with the new patch.

          • python createtests.py fails with the following errors:
            creating data/avro/deflate/records.avro
            sh: data/avro/deflate/records.avro: No such file or directory
            creating data/avro/deflate/recordsAsOutputByPig.avro
            sh: data/avro/deflate/recordsAsOutputByPig.avro: No such file or directory
            creating data/avro/snappy/records.avro
            sh: data/avro/snappy/records.avro: No such file or directory
            creating data/avro/snappy/recordsAsOutputByPig.avro
            sh: data/avro/snappy/recordsAsOutputByPig.avro: No such file or directory
            

            These errors are due to a typo at line 85 in createtests.py: "data/avro/" should be "data/avro/compressed/".

          • After fixing this typo, I get the following failures in TestAvroStorage:
            Testcase: testStoreSnappyCompressedRecords took 4.439 sec
                FAILED
            Testcase: testLoadDeflateCompressedRecords took 0.007 sec
                FAILED
            Testcase: testStoreDeflateCompressedRecords took 3.557 sec
                FAILED
            Testcase: testLoadSnappyCompressedRecords took 0.004 sec
                FAILED
            

            These errors are due to typos in TestAvroStorage.java: "data/avro/deflate/" and "data/avro/snappy/" should be "data/avro/compressed/deflate/" and "data/avro/compressed/sanppy/" respectively.

          • Lastly, I realized that createtest.py doesn't generate the input avro file for testPartialLoadGlob, and I saw your comment:
            Please copy and run this command manually (doesn't work correctly from python right now... arg...
            

            In fact, I haven't been able to fix this myself yet. When I run the command manually, testPartialLoadGlob passes.

          Show
          Cheolsoo Park added a comment - Hi Joe, I found a few errors with the new patch. python createtests.py fails with the following errors: creating data/avro/deflate/records.avro sh: data/avro/deflate/records.avro: No such file or directory creating data/avro/deflate/recordsAsOutputByPig.avro sh: data/avro/deflate/recordsAsOutputByPig.avro: No such file or directory creating data/avro/snappy/records.avro sh: data/avro/snappy/records.avro: No such file or directory creating data/avro/snappy/recordsAsOutputByPig.avro sh: data/avro/snappy/recordsAsOutputByPig.avro: No such file or directory These errors are due to a typo at line 85 in createtests.py : " data/avro/ " should be " data/avro/compressed/ ". After fixing this typo, I get the following failures in TestAvroStorage : Testcase: testStoreSnappyCompressedRecords took 4.439 sec FAILED Testcase: testLoadDeflateCompressedRecords took 0.007 sec FAILED Testcase: testStoreDeflateCompressedRecords took 3.557 sec FAILED Testcase: testLoadSnappyCompressedRecords took 0.004 sec FAILED These errors are due to typos in TestAvroStorage.java : " data/avro/deflate/ " and " data/avro/snappy/ " should be " data/avro/compressed/deflate/ " and " data/avro/compressed/sanppy/ " respectively. Lastly, I realized that createtest.py doesn't generate the input avro file for testPartialLoadGlob , and I saw your comment: Please copy and run this command manually (doesn't work correctly from python right now... arg... In fact, I haven't been able to fix this myself yet. When I run the command manually, testPartialLoadGlob passes.
          Hide
          Joseph Adler added a comment -

          Revised patch to include bug fixes (thanks Cheolsoo Park!)

          Show
          Joseph Adler added a comment - Revised patch to include bug fixes (thanks Cheolsoo Park!)
          Hide
          Cheolsoo Park added a comment -

          Hi Joe,

          I am attaching a patch that replaces createtests.py with Java code in TestAvroStorage.java. I did this for two reasons:

          • Problem with generating evenFileNameTestDirectoryCounts.avro. Using Avro Tool API addresses it.
          • Better integration with JUnit. Now we can run the unit test with a single command (ant clean test -Dtestcase=TestAvroStorage). This auto-generates and delete input files before and after each test run.

          Please let me know what you think.

          Show
          Cheolsoo Park added a comment - Hi Joe, I am attaching a patch that replaces createtests.py with Java code in TestAvroStorage.java . I did this for two reasons: Problem with generating evenFileNameTestDirectoryCounts.avro . Using Avro Tool API addresses it. Better integration with JUnit. Now we can run the unit test with a single command ( ant clean test -Dtestcase=TestAvroStorage ). This auto-generates and delete input files before and after each test run. Please let me know what you think.
          Hide
          Cheolsoo Park added a comment -

          I am attaching a new patch that includes the following fixes:

          1. hadoop-core.jar is pulled as it's a dependency of trevni-core.jar. This causes compile errors with hadoop-2.0.x. I fixed this as follows:
            +    <dependency org="org.apache.avro" name="trevni-avro" rev="${avro.version}"
            +      conf="compile->default;checkstyle->master">
            +      <exclude org="org.apache.hadoop" module="hadoop-core"/>
            +    </dependency>
            
          2. avro-tools.jar contains hadoop classes such as Configuration. This causes compile errors with hadoop-2.0.x. I fixed this as follows:
            +    <dependency org="org.apache.avro" name="avro-tools" rev="${avro.version}"
            +      conf="test->default">
            +      <artifact name="nodeps" type="jar"/>
            +    </dependency>
            

          Now I can run TestAvroStorage with -Dhadoopversion=23, but all test cases currently fail. I haven't investigated yet.

          Show
          Cheolsoo Park added a comment - I am attaching a new patch that includes the following fixes: hadoop-core.jar is pulled as it's a dependency of trevni-core.jar . This causes compile errors with hadoop-2.0.x. I fixed this as follows: + <dependency org= "org.apache.avro" name= "trevni-avro" rev= "${avro.version}" + conf= "compile-> default ;checkstyle->master" > + <exclude org= "org.apache.hadoop" module= "hadoop-core" /> + </dependency> avro-tools.jar contains hadoop classes such as Configuration. This causes compile errors with hadoop-2.0.x. I fixed this as follows: + <dependency org= "org.apache.avro" name= "avro-tools" rev= "${avro.version}" + conf= "test-> default " > + <artifact name= "nodeps" type= "jar" /> + </dependency> Now I can run TestAvroStorage with -Dhadoopversion=23 , but all test cases currently fail. I haven't investigated yet.
          Hide
          Cheolsoo Park added a comment -

          I am uploading a new patch that includes the following changes:

          • I forgot to pass the --codec option to the Avro tool when generating compressed .avro files, so I fixed it.
          • I found that the unit test fails with MR2 because mapred.output.compress is set to true while mapred.output.compression.codec is not set.
            ERROR 0: 'mapred.output.compress' is set but no value is specified for 'mapred.output.compression.codec'.
            

            What's worse is that any test cases that run after testStoreSnappyCompressedRecords fails for the same reason. Apparently, this property remains enabled and affects other test cases. For now, I removed the following line from identity_codec.pig:

            SET mapred.output.compress true
            

            With this change, I can actually run test jobs, but they still fail. The reason is because output files do not match with expected ones. I didn't investigate why this happens with MR2.

          • Lastly, I also found that the current patch does not implement compression support. The test cases such as testStoreSnappyCompressedRecords and testStoreDeflateCompressedRecords do not check avro.codec. Please correct me if I am wrong.

          Thanks!

          Show
          Cheolsoo Park added a comment - I am uploading a new patch that includes the following changes: I forgot to pass the --codec option to the Avro tool when generating compressed .avro files, so I fixed it. I found that the unit test fails with MR2 because mapred.output.compress is set to true while mapred.output.compression.codec is not set. ERROR 0: 'mapred.output.compress' is set but no value is specified for 'mapred.output.compression.codec'. What's worse is that any test cases that run after testStoreSnappyCompressedRecords fails for the same reason. Apparently, this property remains enabled and affects other test cases. For now, I removed the following line from identity_codec.pig : SET mapred.output.compress true With this change, I can actually run test jobs, but they still fail. The reason is because output files do not match with expected ones. I didn't investigate why this happens with MR2. Lastly, I also found that the current patch does not implement compression support. The test cases such as testStoreSnappyCompressedRecords and testStoreDeflateCompressedRecords do not check avro.codec . Please correct me if I am wrong. Thanks!
          Hide
          Joseph Adler added a comment -

          Hi Cheolsoo,

          You're totally right; I don't check the compression properties. I know that the avro mapred library does check those parameters (org.apache.avro.mapred.AvroOutPutFormat), but I don't use that output format. Fixing and testing, will follow up with a patch.

          – Joe

          Show
          Joseph Adler added a comment - Hi Cheolsoo, You're totally right; I don't check the compression properties. I know that the avro mapred library does check those parameters (org.apache.avro.mapred.AvroOutPutFormat), but I don't use that output format. Fixing and testing, will follow up with a patch. – Joe
          Hide
          Joseph Adler added a comment -

          Added fixes for compression (and other metadata)

          Show
          Joseph Adler added a comment - Added fixes for compression (and other metadata)
          Hide
          Cheolsoo Park added a comment -

          Hi Joe, I think you forgot to add new files to the patch. Do you mind uploading the patch again?

          Show
          Cheolsoo Park added a comment - Hi Joe, I think you forgot to add new files to the patch. Do you mind uploading the patch again?
          Hide
          Joseph Adler added a comment -

          Oops, this one contains the changes.

          Show
          Joseph Adler added a comment - Oops, this one contains the changes.
          Hide
          Russell Jurney added a comment -

          Joe, some comments on handling errors in PIG-3059:

          Regarding Avro, in reading https://github.com/apache/avro/blob/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader.java - it looks like you can still sync to the next record under most bad reads. We should do so.

          You're right about a bad sync halting things, but in the case of a bad sync - you might try advancing by some amount using seek() and then sync'ing again? I think this would work. I could be wrong, but in looking how seeks work - I think that would be ok. Kinda neat, maybe? Worst case, we would only throw out inputsplits on a bad sync(), not a bad read(). length() should help, as might pastSync(), skip() and available()

          Show
          Russell Jurney added a comment - Joe, some comments on handling errors in PIG-3059 : Regarding Avro, in reading https://github.com/apache/avro/blob/trunk/lang/java/avro/src/main/java/org/apache/avro/file/DataFileReader.java - it looks like you can still sync to the next record under most bad reads. We should do so. You're right about a bad sync halting things, but in the case of a bad sync - you might try advancing by some amount using seek() and then sync'ing again? I think this would work. I could be wrong, but in looking how seeks work - I think that would be ok. Kinda neat, maybe? Worst case, we would only throw out inputsplits on a bad sync(), not a bad read(). length() should help, as might pastSync(), skip() and available()
          Hide
          Joseph Adler added a comment -

          Hi Russ,

          I think you're right... it looks like you could do something like this in AvroRecordReader.nextKeyValue:

            @Override
            public boolean nextKeyValue() throws IOException, InterruptedException {
          
              if (reader.pastSync(end)) {
                return false;
              }
          
              try {
                currentRecord = reader.next(new GenericData.Record(schema));
              } catch (NoSuchElementException e) {
                return false;
              } catch (IOException ioe) {
                reader.sync(reader.tell()+1);
                throw ioe;
              }
          
              return true;
            }
          

          Let me test this out to make sure it runs correctly on uncorrupted files. Would you mind creating a corrupted test file that I can use for testing?

          Show
          Joseph Adler added a comment - Hi Russ, I think you're right... it looks like you could do something like this in AvroRecordReader.nextKeyValue: @Override public boolean nextKeyValue() throws IOException, InterruptedException { if (reader.pastSync(end)) { return false ; } try { currentRecord = reader.next( new GenericData.Record(schema)); } catch (NoSuchElementException e) { return false ; } catch (IOException ioe) { reader.sync(reader.tell()+1); throw ioe; } return true ; } Let me test this out to make sure it runs correctly on uncorrupted files. Would you mind creating a corrupted test file that I can use for testing?
          Hide
          Cheolsoo Park added a comment -

          I am uploading a test program that I wrote. What it does are:

          • Reads an Avro file and prints out records to stdout.
          • Each time when it prints out records, it also prints out the current sync position (i.e. tell()).
          • When encountering an exception during a next(), it does sync(tell() + 1).

          Unfortunately, it doesn't seem that Avro files have sync positions between records. Instead, it only has a single sync position at the end of file.

          You can run my program as follows:

          • Untar Test.tar.gz:
            bad.avro
            good.avro
            Main.java
            Test.jar
            
          • There are two Avro files: good.avro and bad.avro:
            good.avro
            java -jar avro-tool.jar tojson good.avro
            "0"
            ...
            "999"
            
            bad.avro
            java -jar avro-tool.jar tojson bad.avro
            Exception in thread "main" org.apache.avro.AvroRuntimeException: java.io.IOException: Invalid sync!
            	at org.apache.avro.file.DataFileStream.hasNext(DataFileStream.java:210)
            	at org.apache.avro.tool.DataFileReadTool.run(DataFileReadTool.java:64)
            	at org.apache.avro.tool.Main.run(Main.java:74)
            	at org.apache.avro.tool.Main.main(Main.java:63)
            
            The Test.tar.gz contains the following files:
            
          • Run "java -jar Test.jar good.avro" gives the following output:
            lengh(): 3969
            tell(): 3969
            next(): 0
            ...
            next(): 999
            tell(): 3969
            

            As can be seen, the sync position never changes between records.

          • Run "java -jar Test.jar bad.avro" gives the following output:
            lengh(): 3969
            tell(): 3969
            hasNext() or next() failed
            tell(): 3970
            

            The sync(tell()+1) sets the sync position to past the end of file, so it ends the program.

          In summary, I don't think that we can recover from a bad sync() in Avro. Please correct me if my test program has a bug.

          Thanks!

          Show
          Cheolsoo Park added a comment - I am uploading a test program that I wrote. What it does are: Reads an Avro file and prints out records to stdout. Each time when it prints out records, it also prints out the current sync position (i.e. tell()). When encountering an exception during a next(), it does sync(tell() + 1). Unfortunately, it doesn't seem that Avro files have sync positions between records. Instead, it only has a single sync position at the end of file. You can run my program as follows: Untar Test.tar.gz: bad.avro good.avro Main.java Test.jar There are two Avro files: good.avro and bad.avro: good.avro java -jar avro-tool.jar tojson good.avro "0" ... "999" bad.avro java -jar avro-tool.jar tojson bad.avro Exception in thread "main" org.apache.avro.AvroRuntimeException: java.io.IOException: Invalid sync! at org.apache.avro.file.DataFileStream.hasNext(DataFileStream.java:210) at org.apache.avro.tool.DataFileReadTool.run(DataFileReadTool.java:64) at org.apache.avro.tool.Main.run(Main.java:74) at org.apache.avro.tool.Main.main(Main.java:63) The Test.tar.gz contains the following files: Run "java -jar Test.jar good.avro" gives the following output: lengh(): 3969 tell(): 3969 next(): 0 ... next(): 999 tell(): 3969 As can be seen, the sync position never changes between records. Run "java -jar Test.jar bad.avro" gives the following output: lengh(): 3969 tell(): 3969 hasNext() or next() failed tell(): 3970 The sync(tell()+1) sets the sync position to past the end of file, so it ends the program. In summary, I don't think that we can recover from a bad sync() in Avro. Please correct me if my test program has a bug. Thanks!
          Hide
          Russell Jurney added a comment -

          Linking to PIG-3059

          Show
          Russell Jurney added a comment - Linking to PIG-3059
          Hide
          Russell Jurney added a comment -

          I just asked user@avro.apache.org how to recover from bad records when using DataFileReader.

          Show
          Russell Jurney added a comment - I just asked user@avro.apache.org how to recover from bad records when using DataFileReader.
          Hide
          Joseph Adler added a comment -

          Hi Cheolsoo:

          What size file are you using? You can configure the sync interval with the parameter avro.mapred.sync.interval (defined in org.apache.avro.mapred.AvroOutputFormat), and implemented in my latest patch (the one from last week).

          – Joe

          Show
          Joseph Adler added a comment - Hi Cheolsoo: What size file are you using? You can configure the sync interval with the parameter avro.mapred.sync.interval (defined in org.apache.avro.mapred.AvroOutputFormat), and implemented in my latest patch (the one from last week). – Joe
          Hide
          Cheolsoo Park added a comment -

          Hi Joe,

          My file is pretty small. It makes sense that the sync interval is configurable.

          Please let me try to set the interval and run another test.

          Thanks!

          Show
          Cheolsoo Park added a comment - Hi Joe, My file is pretty small. It makes sense that the sync interval is configurable. Please let me try to set the interval and run another test. Thanks!
          Hide
          Cheolsoo Park added a comment -

          OK, I made two changes to my test program:

          • Set the sync interval to 32 bytes (32 bytes seems to be the possible minimum interval unless I misunderstood the Avro source code).
          • Increased the file size to ~10mb.

          I can see sync points are added after every 32 bytes or so. I also see tell() returns increasing values with the good file.

          I am mimicking a bad file by deleting a random byte in a sync point. Running avro-tool tojson gives me an invalid sync exception after reading to that corrupted sync point, so I guess that the bad file is created correctly.

          However, I cannot still recover from a bad read. I catches an exception from next() and do sync(tell() + 1). The next tell() seems to correctly return the next valid sync point. But next() still fails. In fact, it continues to fail until it hits the end of the file.

          next(): 9999
          tell(): 82133
          hasNext() or next() failed 
          tell(): 82196
          hasNext() or next() failed 
          tell(): 82250
          ...
          hasNext() or next() failed
          tell(): 10424205
          hasNext() or next() failed
          tell(): 10424258
          end of the file
          tell(): 10424259
          past the end of the file
          

          I am uploading my test program. TestInput.java generates input files, and Test.java runs the test.

          Does anyone have an idea what I am doing wrong?

          Show
          Cheolsoo Park added a comment - OK, I made two changes to my test program: Set the sync interval to 32 bytes (32 bytes seems to be the possible minimum interval unless I misunderstood the Avro source code). Increased the file size to ~10mb. I can see sync points are added after every 32 bytes or so. I also see tell() returns increasing values with the good file. I am mimicking a bad file by deleting a random byte in a sync point. Running avro-tool tojson gives me an invalid sync exception after reading to that corrupted sync point, so I guess that the bad file is created correctly. However, I cannot still recover from a bad read. I catches an exception from next() and do sync(tell() + 1). The next tell() seems to correctly return the next valid sync point. But next() still fails. In fact, it continues to fail until it hits the end of the file. next(): 9999 tell(): 82133 hasNext() or next() failed tell(): 82196 hasNext() or next() failed tell(): 82250 ... hasNext() or next() failed tell(): 10424205 hasNext() or next() failed tell(): 10424258 end of the file tell(): 10424259 past the end of the file I am uploading my test program. TestInput.java generates input files, and Test.java runs the test. Does anyone have an idea what I am doing wrong?
          Hide
          Scott Carey added a comment -

          Try corrupting the file at a point inside the data block instead of inside
          the sync marker. The ability to recover from a corrupted file was added
          in response to corrupted data, not corrupted sync.

          Show
          Scott Carey added a comment - Try corrupting the file at a point inside the data block instead of inside the sync marker. The ability to recover from a corrupted file was added in response to corrupted data, not corrupted sync.
          Hide
          Cheolsoo Park added a comment -

          Hi Scott,

          Thank you very much. That makes sense. After several tries and errors, I managed to "correctly" corrupt a data block and was able to verify the recovery.

          The output from 'java-tool.jar tojson bad.avro' is as follows:

          Caused by: java.io.IOException: Block read partially, the data may be corrupt
          	at org.apache.avro.file.DataFileStream.hasNext(DataFileStream.java:194)
          	... 3 more
          

          The output from my test program is as follows:

          next(): 685
          tell(): 8196
          next(): 686
          tell(): 8196
          hasNext() or next() failed
          tell(): 8240
          next(): 2656
          tell(): 16432
          next(): 2657
          tell(): 16432
          

          The data are sequential integers (0 ~ 1M). Here is the number of lost integers due to a single corrupted data block with different sync intervals:

          Sync interval in bytes Num. of lost values
          32 1970
          16,000 5389

          In summary,

          • Avro can recover from a data block corruption but cannot from a sync marker corruption.
          • The amount of data loss depends on the sync interval. By default, it's 16KB, but it can vary from 32 to 2^30 bytes. The greater the sync interval is, the more data loss is.

          I am attaching my test program and input files if anyone's interested.

          Thanks!

          Show
          Cheolsoo Park added a comment - Hi Scott, Thank you very much. That makes sense. After several tries and errors, I managed to "correctly" corrupt a data block and was able to verify the recovery. The output from 'java-tool.jar tojson bad.avro' is as follows: Caused by: java.io.IOException: Block read partially, the data may be corrupt at org.apache.avro.file.DataFileStream.hasNext(DataFileStream.java:194) ... 3 more The output from my test program is as follows: next(): 685 tell(): 8196 next(): 686 tell(): 8196 hasNext() or next() failed tell(): 8240 next(): 2656 tell(): 16432 next(): 2657 tell(): 16432 The data are sequential integers (0 ~ 1M). Here is the number of lost integers due to a single corrupted data block with different sync intervals: Sync interval in bytes Num. of lost values 32 1970 16,000 5389 In summary, Avro can recover from a data block corruption but cannot from a sync marker corruption. The amount of data loss depends on the sync interval. By default, it's 16KB, but it can vary from 32 to 2^30 bytes. The greater the sync interval is, the more data loss is. I am attaching my test program and input files if anyone's interested. Thanks!
          Hide
          Doug Cutting added a comment -

          > Avro can recover from a data block corruption but cannot from a sync marker corruption.

          BTW, I think that's probably a bug in Avro's Java implementation that (separately) we should fix.

          Show
          Doug Cutting added a comment - > Avro can recover from a data block corruption but cannot from a sync marker corruption. BTW, I think that's probably a bug in Avro's Java implementation that (separately) we should fix.
          Hide
          Russell Jurney added a comment -

          Make note of lack of Trevni Boolean support: AVRO-1229

          Show
          Russell Jurney added a comment - Make note of lack of Trevni Boolean support: AVRO-1229
          Hide
          Joseph Adler added a comment -

          Just got bitten by PIG-2266 while doing some performance testing with this ticket. I'm going to add that fix to this patch so that AvroStorage and TrevniStorage actually work.

          Show
          Joseph Adler added a comment - Just got bitten by PIG-2266 while doing some performance testing with this ticket. I'm going to add that fix to this patch so that AvroStorage and TrevniStorage actually work.
          Hide
          Cheolsoo Park added a comment -

          Hi Joe,

          Do you mind attaching the fix in PIG-2266 as a separate patch? I will review and commit it right away. I just think that putting everything into a single patch will slow down progress.

          Thanks,
          Cheolsoo

          Show
          Cheolsoo Park added a comment - Hi Joe, Do you mind attaching the fix in PIG-2266 as a separate patch? I will review and commit it right away. I just think that putting everything into a single patch will slow down progress. Thanks, Cheolsoo
          Hide
          Russell Jurney added a comment -

          Plz note, Avro will have Boolean soon: AVRO-1229

          Show
          Russell Jurney added a comment - Plz note, Avro will have Boolean soon: AVRO-1229
          Hide
          Santhosh Srinivasan added a comment -

          Created and linked AVRO-1235.

          Show
          Santhosh Srinivasan added a comment - Created and linked AVRO-1235 .
          Hide
          Joseph Adler added a comment -

          Some additional bug fixes:

          • Now correctly identifies recursive schema definitions
          • TrevniStorage was not correctly flushing output buffers before closing, causing files to be corrupted
          Show
          Joseph Adler added a comment - Some additional bug fixes: Now correctly identifies recursive schema definitions TrevniStorage was not correctly flushing output buffers before closing, causing files to be corrupted
          Hide
          Cheolsoo Park added a comment -

          Joseph Adler, thank you for the update. I just made very minor changes to the patch as follows:

          • Took out PIG-2266 and posted it in a separate patch in PIG-2266. I will commit it after running tests.
          • Parameterized the snappy version in build.xml and ivy.xml.
          • Down-graded Avro version to 1.7.3. I noticed that you bumped it to 1.7.4-SNAPSHOT, but when I tried your patch, 1.7.4-SNAPSHOT didn't exist on the Maven repo. Did you want Avro 1.7.4-SNAPSHOT because of recent updates in Avro?
          Show
          Cheolsoo Park added a comment - Joseph Adler , thank you for the update. I just made very minor changes to the patch as follows: Took out PIG-2266 and posted it in a separate patch in PIG-2266 . I will commit it after running tests. Parameterized the snappy version in build.xml and ivy.xml. Down-graded Avro version to 1.7.3. I noticed that you bumped it to 1.7.4-SNAPSHOT, but when I tried your patch, 1.7.4-SNAPSHOT didn't exist on the Maven repo. Did you want Avro 1.7.4-SNAPSHOT because of recent updates in Avro?
          Hide
          Joseph Adler added a comment -

          Sorry, didn't mean to submit a patch with Avro 1.7.4-SNAPSHOT. I added a couple optimizations to Trevni so that the performance was comparable with Avro. (I'll submit that patch to Avro.)

          Show
          Joseph Adler added a comment - Sorry, didn't mean to submit a patch with Avro 1.7.4-SNAPSHOT. I added a couple optimizations to Trevni so that the performance was comparable with Avro. (I'll submit that patch to Avro.)
          Hide
          Cheolsoo Park added a comment -

          I think the patch is very close to being committed. Two main obstacles are:

          1. Tests do not pass with Hadoop-2.0.x (i.e. ant clean test -Dtestcase=TestAvroStorage -Dhadoopversion=23).
          2. Documentation is missing.

          I will give another shot on debugging #1 when I get more time, but any help would be appreciated!

          Show
          Cheolsoo Park added a comment - I think the patch is very close to being committed. Two main obstacles are: Tests do not pass with Hadoop-2.0.x (i.e. ant clean test -Dtestcase=TestAvroStorage -Dhadoopversion=23). Documentation is missing. I will give another shot on debugging #1 when I get more time, but any help would be appreciated!
          Hide
          Russell Jurney added a comment -

          I'll start testing this again.

          Show
          Russell Jurney added a comment - I'll start testing this again.
          Hide
          Joseph Adler added a comment -

          Let me know what help you need. I can work on the documentation as well. Is early next week enough time? (Also, check out Avro-1241. I couldn't get adequate performance without it.)

          Show
          Joseph Adler added a comment - Let me know what help you need. I can work on the documentation as well. Is early next week enough time? (Also, check out Avro-1241. I couldn't get adequate performance without it.)
          Hide
          Cheolsoo Park added a comment -

          Joseph Adler, if you could add documentation, that would be awesome!

          Show
          Cheolsoo Park added a comment - Joseph Adler , if you could add documentation, that would be awesome!
          Hide
          Joseph Adler added a comment -

          Added description of AvroStorage and TrevniStorage to documentation. (Not finished editing yet, but wanted to share what I'd written so far.)

          Show
          Joseph Adler added a comment - Added description of AvroStorage and TrevniStorage to documentation. (Not finished editing yet, but wanted to share what I'd written so far.)
          Hide
          Joseph Adler added a comment -

          I think the method setLocation for AvroStoage is marked as "final." Does anyone object to removing the "final" modifier?

          Show
          Joseph Adler added a comment - I think the method setLocation for AvroStoage is marked as "final." Does anyone object to removing the "final" modifier?
          Hide
          Russell Jurney added a comment -

          Which patch should I be testing?

          Show
          Russell Jurney added a comment - Which patch should I be testing?
          Hide
          Cheolsoo Park added a comment -

          Hi Russell,

          PIG-3015-7.patch is the latest patch.

          PIG-3015-8.patch is a doc patch. In fact, I will rename it since it's confusing.

          Show
          Cheolsoo Park added a comment - Hi Russell, PIG-3015 -7.patch is the latest patch. PIG-3015 -8.patch is a doc patch. In fact, I will rename it since it's confusing.
          Hide
          Cheolsoo Park added a comment -

          Renaming PIG-3015-8.patch to PIG-3015-doc.patch.

          Show
          Cheolsoo Park added a comment - Renaming PIG-3015 -8.patch to PIG-3015 -doc.patch.
          Hide
          Russell Jurney added a comment -

          Pre-req for PIG-3111 which will refer to this in its implementation.

          Show
          Russell Jurney added a comment - Pre-req for PIG-3111 which will refer to this in its implementation.
          Hide
          Russell Jurney added a comment -

          Initial experience is very positive, making me highly motivated to getting Pig 0.12 out in 3 months or so if possible?

          Show
          Russell Jurney added a comment - Initial experience is very positive, making me highly motivated to getting Pig 0.12 out in 3 months or so if possible?
          Hide
          Russell Jurney added a comment -

          illustrate fails

          Show
          Russell Jurney added a comment - illustrate fails
          Hide
          Russell Jurney added a comment -

          Loading data without going to Piggybank is amazing. However, Trevnistorage fails to store my emails, schema:

          You can reproduce this data with your own gmail emails (just need a few) with these instructions: https://github.com/rjurney/Agile_Data_Code/tree/master/ch03

          grunt> describe emails
          emails: {message_id: chararray,thread_id: chararray,in_reply_to: chararray,subject: chararray,body: chararray,date: chararray,from: (real_name: chararray,address: chararray),tos:

          {to: (real_name: chararray,address: chararray)}

          ,ccs:

          {cc: (real_name: chararray,address: chararray)}

          ,bccs:

          {bcc: (real_name: chararray,address: chararray)}

          ,reply_tos: {reply_to: (real_name: chararray,address: chararray)}}

          Error:

          2013-02-17 18:03:31,574 [Thread-6] INFO org.apache.hadoop.mapred.MapTask - io.sort.mb = 100
          2013-02-17 18:03:31,680 [Thread-6] INFO org.apache.hadoop.mapred.MapTask - data buffer = 79691776/99614720
          2013-02-17 18:03:31,680 [Thread-6] INFO org.apache.hadoop.mapred.MapTask - record buffer = 262144/327680
          2013-02-17 18:03:31,699 [Thread-6] INFO org.apache.pig.data.SchemaTupleBackend - Key [pig.schematuple] was not set... will not generate code.
          2013-02-17 18:03:31,713 [Thread-6] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapReduce$Map - Aliases being processed per job phase (AliasName[line,offset]): M: emails[2,9],null[-1,-1],null[-1,-1],token_records[-1,-1],doc_word_totals[5,18],1-84[5,27] C: doc_word_totals[5,18],1-84[5,27] R: doc_word_totals[5,18]
          2013-02-17 18:03:31,748 [Thread-6] WARN org.apache.hadoop.mapred.LocalJobRunner - job_local_0001
          org.apache.pig.backend.executionengine.ExecException: ERROR 0: Exception while executing [POUserFunc (Name: POUserFunc(org.apache.pig.builtin.LuceneTokenize)[bag] - scope-19 Operator Key: scope-19) children: null at []]: java.lang.NullPointerException
          at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.getNext(PhysicalOperator.java:370)
          at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach.processPlan(POForEach.java:378)
          at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach.getNext(POForEach.java:298)
          at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.processInput(PhysicalOperator.java:314)
          at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPreCombinerLocalRearrange.getNext(POPreCombinerLocalRearrange.java:126)
          at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.processInput(PhysicalOperator.java:314)
          at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach.getNext(POForEach.java:242)
          at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.processInput(PhysicalOperator.java:314)
          at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange.getNext(POLocalRearrange.java:263)
          at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit.runPipeline(POSplit.java:254)
          at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit.processPlan(POSplit.java:236)
          at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit.getNext(POSplit.java:228)
          at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.runPipeline(PigGenericMapBase.java:283)
          at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:278)
          at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:64)
          at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144)
          at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:764)
          at org.apache.hadoop.mapred.MapTask.run(MapTask.java:370)
          at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:212)
          Caused by: java.lang.NullPointerException
          at org.apache.lucene.analysis.standard.std31.StandardTokenizerImpl31.zzRefill(StandardTokenizerImpl31.java:795)
          at org.apache.lucene.analysis.standard.std31.StandardTokenizerImpl31.getNextToken(StandardTokenizerImpl31.java:1002)
          at org.apache.lucene.analysis.standard.StandardTokenizer.incrementToken(StandardTokenizer.java:180)
          at org.apache.lucene.analysis.standard.StandardFilter.incrementToken(StandardFilter.java:49)
          at org.apache.lucene.analysis.core.LowerCaseFilter.incrementToken(LowerCaseFilter.java:54)
          at org.apache.lucene.analysis.util.FilteringTokenFilter.incrementToken(FilteringTokenFilter.java:50)
          at org.apache.pig.builtin.LuceneTokenize.exec(LuceneTokenize.java:70)
          at org.apache.pig.builtin.LuceneTokenize.exec(LuceneTokenize.java:51)
          at org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc.getNext(POUserFunc.java:336)
          at org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc.getNext(POUserFunc.java:380)
          at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.getNext(PhysicalOperator.java:341)
          ... 18 more
          2013-02-17 18:03:31,811 [main] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - HadoopJobId: job_local_0001
          2013-02-17 18:03:31,811 [main] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - Processing aliases 1-84,doc_word_totals,emails,token_records
          2013-02-17 18:03:31,811 [main] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - detailed locations: M: emails[2,9],null[-1,-1],null[-1,-1],token_records[-1,-1],doc_word_totals[5,18],1-84[5,27] C: doc_word_totals[5,18],1-84[5,27] R: doc_word_totals[5,18]
          2013-02-17 18:03:31,813 [main] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - 0% complete
          2013-02-17 18:03:31,817 [main] WARN org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - Ooops! Some job has failed! Specify -stop_on_failure if you want Pig to stop immediately on failure.
          2013-02-17 18:03:31,817 [main] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - job job_local_0001 has failed! Stop running all dependent jobs
          2013-02-17 18:03:31,817 [main] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - 100% complete
          2013-02-17 18:03:31,818 [main] ERROR org.apache.pig.tools.pigstats.PigStatsUtil - 1 map reduce job(s) failed!
          2013-02-17 18:03:31,818 [main] INFO org.apache.pig.tools.pigstats.SimplePigStats - Detected Local mode. Stats reported below may be incomplete
          2013-02-17 18:03:31,819 [main] INFO org.apache.pig.tools.pigstats.SimplePigStats - Script Statistics:

          HadoopVersion PigVersion UserId StartedAt FinishedAt Features
          1.0.3 0.12.0-SNAPSHOT rjurney 2013-02-17 18:03:31 2013-02-17 18:03:31 HASH_JOIN,GROUP_BY

          Failed!

          Failed Jobs:
          JobId Alias Feature Message Outputs
          job_local_0001 1-84,doc_word_totals,emails,token_records MULTI_QUERY,COMBINER Message: Job failed! Error - NA

          Input(s):
          Failed to read data from "/me/Data/test_mbox"

          Output(s):

          Job DAG:
          job_local_0001 -> null,
          null -> null,null,
          null -> null,
          null -> null,
          null -> null,
          null

          2013-02-17 18:03:31,819 [main] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - Failed!

          Show
          Russell Jurney added a comment - Loading data without going to Piggybank is amazing. However, Trevnistorage fails to store my emails, schema: You can reproduce this data with your own gmail emails (just need a few) with these instructions: https://github.com/rjurney/Agile_Data_Code/tree/master/ch03 grunt> describe emails emails: {message_id: chararray,thread_id: chararray,in_reply_to: chararray,subject: chararray,body: chararray,date: chararray,from: (real_name: chararray,address: chararray),tos: {to: (real_name: chararray,address: chararray)} ,ccs: {cc: (real_name: chararray,address: chararray)} ,bccs: {bcc: (real_name: chararray,address: chararray)} ,reply_tos: {reply_to: (real_name: chararray,address: chararray)}} Error: 2013-02-17 18:03:31,574 [Thread-6] INFO org.apache.hadoop.mapred.MapTask - io.sort.mb = 100 2013-02-17 18:03:31,680 [Thread-6] INFO org.apache.hadoop.mapred.MapTask - data buffer = 79691776/99614720 2013-02-17 18:03:31,680 [Thread-6] INFO org.apache.hadoop.mapred.MapTask - record buffer = 262144/327680 2013-02-17 18:03:31,699 [Thread-6] INFO org.apache.pig.data.SchemaTupleBackend - Key [pig.schematuple] was not set... will not generate code. 2013-02-17 18:03:31,713 [Thread-6] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapReduce$Map - Aliases being processed per job phase (AliasName [line,offset] ): M: emails [2,9] ,null [-1,-1] ,null [-1,-1] ,token_records [-1,-1] ,doc_word_totals [5,18] ,1-84 [5,27] C: doc_word_totals [5,18] ,1-84 [5,27] R: doc_word_totals [5,18] 2013-02-17 18:03:31,748 [Thread-6] WARN org.apache.hadoop.mapred.LocalJobRunner - job_local_0001 org.apache.pig.backend.executionengine.ExecException: ERROR 0: Exception while executing [POUserFunc (Name: POUserFunc(org.apache.pig.builtin.LuceneTokenize) [bag] - scope-19 Operator Key: scope-19) children: null at []]: java.lang.NullPointerException at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.getNext(PhysicalOperator.java:370) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach.processPlan(POForEach.java:378) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach.getNext(POForEach.java:298) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.processInput(PhysicalOperator.java:314) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPreCombinerLocalRearrange.getNext(POPreCombinerLocalRearrange.java:126) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.processInput(PhysicalOperator.java:314) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach.getNext(POForEach.java:242) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.processInput(PhysicalOperator.java:314) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange.getNext(POLocalRearrange.java:263) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit.runPipeline(POSplit.java:254) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit.processPlan(POSplit.java:236) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit.getNext(POSplit.java:228) at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.runPipeline(PigGenericMapBase.java:283) at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:278) at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:64) at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144) at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:764) at org.apache.hadoop.mapred.MapTask.run(MapTask.java:370) at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:212) Caused by: java.lang.NullPointerException at org.apache.lucene.analysis.standard.std31.StandardTokenizerImpl31.zzRefill(StandardTokenizerImpl31.java:795) at org.apache.lucene.analysis.standard.std31.StandardTokenizerImpl31.getNextToken(StandardTokenizerImpl31.java:1002) at org.apache.lucene.analysis.standard.StandardTokenizer.incrementToken(StandardTokenizer.java:180) at org.apache.lucene.analysis.standard.StandardFilter.incrementToken(StandardFilter.java:49) at org.apache.lucene.analysis.core.LowerCaseFilter.incrementToken(LowerCaseFilter.java:54) at org.apache.lucene.analysis.util.FilteringTokenFilter.incrementToken(FilteringTokenFilter.java:50) at org.apache.pig.builtin.LuceneTokenize.exec(LuceneTokenize.java:70) at org.apache.pig.builtin.LuceneTokenize.exec(LuceneTokenize.java:51) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc.getNext(POUserFunc.java:336) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc.getNext(POUserFunc.java:380) at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.getNext(PhysicalOperator.java:341) ... 18 more 2013-02-17 18:03:31,811 [main] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - HadoopJobId: job_local_0001 2013-02-17 18:03:31,811 [main] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - Processing aliases 1-84,doc_word_totals,emails,token_records 2013-02-17 18:03:31,811 [main] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - detailed locations: M: emails [2,9] ,null [-1,-1] ,null [-1,-1] ,token_records [-1,-1] ,doc_word_totals [5,18] ,1-84 [5,27] C: doc_word_totals [5,18] ,1-84 [5,27] R: doc_word_totals [5,18] 2013-02-17 18:03:31,813 [main] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - 0% complete 2013-02-17 18:03:31,817 [main] WARN org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - Ooops! Some job has failed! Specify -stop_on_failure if you want Pig to stop immediately on failure. 2013-02-17 18:03:31,817 [main] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - job job_local_0001 has failed! Stop running all dependent jobs 2013-02-17 18:03:31,817 [main] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - 100% complete 2013-02-17 18:03:31,818 [main] ERROR org.apache.pig.tools.pigstats.PigStatsUtil - 1 map reduce job(s) failed! 2013-02-17 18:03:31,818 [main] INFO org.apache.pig.tools.pigstats.SimplePigStats - Detected Local mode. Stats reported below may be incomplete 2013-02-17 18:03:31,819 [main] INFO org.apache.pig.tools.pigstats.SimplePigStats - Script Statistics: HadoopVersion PigVersion UserId StartedAt FinishedAt Features 1.0.3 0.12.0-SNAPSHOT rjurney 2013-02-17 18:03:31 2013-02-17 18:03:31 HASH_JOIN,GROUP_BY Failed! Failed Jobs: JobId Alias Feature Message Outputs job_local_0001 1-84,doc_word_totals,emails,token_records MULTI_QUERY,COMBINER Message: Job failed! Error - NA Input(s): Failed to read data from "/me/Data/test_mbox" Output(s): Job DAG: job_local_0001 -> null, null -> null,null, null -> null, null -> null, null -> null, null 2013-02-17 18:03:31,819 [main] INFO org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher - Failed!
          Hide
          Joseph Adler added a comment -

          Russell Jurney: ]Reading through the stack trace that you posted, it does not look like the null pointer exception was occurring in TrevniStorage. (It looks like it was occurring in the Tokenizer). Does your script work correctly if you use it with another format, like PigStorage?

          Show
          Joseph Adler added a comment - Russell Jurney : ]Reading through the stack trace that you posted, it does not look like the null pointer exception was occurring in TrevniStorage. (It looks like it was occurring in the Tokenizer). Does your script work correctly if you use it with another format, like PigStorage?
          Hide
          Joseph Adler added a comment -

          Added support for Pig dates to AvroStorage and TrevniStorage (they're translated to longs when storing values). Also added a new test case.

          Show
          Joseph Adler added a comment - Added support for Pig dates to AvroStorage and TrevniStorage (they're translated to longs when storing values). Also added a new test case.
          Hide
          Cheolsoo Park added a comment -

          Joseph Adler, sorry for the late reply, but you seem to have forgotten to include "with_dates.pig" in PIG-3015-9.patch. Do you mind uploading it?

          In addition, all the ivy changes are not included in PIG-3015-9.patch, which I will include in a new patch.

          Show
          Cheolsoo Park added a comment - Joseph Adler , sorry for the late reply, but you seem to have forgotten to include "with_dates.pig" in PIG-3015 -9.patch. Do you mind uploading it? In addition, all the ivy changes are not included in PIG-3015 -9.patch, which I will include in a new patch.
          Hide
          Cheolsoo Park added a comment -

          Attaching PIG-3015-10.patch that put back the ivy changes.

          Show
          Cheolsoo Park added a comment - Attaching PIG-3015 -10.patch that put back the ivy changes.
          Hide
          Joseph Adler added a comment -

          Missing test file (not a patch)

          Show
          Joseph Adler added a comment - Missing test file (not a patch)
          Hide
          Cheolsoo Park added a comment -

          I got unit tests working with both hadoop 20 and 23. In fact, the problem was very simple. The option parser code was not correct (i.e. CommandLine.getOptionValue() takes the option name not the long option name.), and thus, the "namespace" of output Avro files was not set properly.

          What's surprised me was that how come this issue only showed up with hadoop23. We need better test coverage.

          PIG-3015-11.patch includes the following changes:

          • Fixed the option parser code.
          • Removed commented-out code.
          • Added "with_dates.pig".
          Show
          Cheolsoo Park added a comment - I got unit tests working with both hadoop 20 and 23. In fact, the problem was very simple. The option parser code was not correct (i.e. CommandLine.getOptionValue() takes the option name not the long option name.), and thus, the "namespace" of output Avro files was not set properly. What's surprised me was that how come this issue only showed up with hadoop23. We need better test coverage. PIG-3015 -11.patch includes the following changes: Fixed the option parser code. Removed commented-out code. Added "with_dates.pig".
          Hide
          Cheolsoo Park added a comment -

          PIG-3015-doc-2.patch fixes "ant docs" errors.

          Show
          Cheolsoo Park added a comment - PIG-3015 -doc-2.patch fixes "ant docs" errors.
          Hide
          Aaron Klish added a comment -

          For Multiple Schema Support, I don't think we should follow the rules in PIG-2579. While this was a good first attempt
          at supporting multiple data files with different schemas, it has problems:

          1) The rules only support schema evolution for flat records. Nested records are not supported.
          2) The AVRO specification already defines a set of rules for schema evolution. In fact, it already has code inside GenericDatumReader that will resolve data between two different schemas.
          We should be using that logic instead of writing logic that only works in very limited situations.
          3) The idea of a merged schema is not as powerful as letting the user define the 'table' schema they expect the data to conform to. I would recommend looking at how LinkedIn added support for AVRO to Hive. Every partition can have a different schema, but there is a single 'table' schema. Hive will map the file/partition schema(s) to the 'table' schema using the logic already inside AVRO.

          Show
          Aaron Klish added a comment - For Multiple Schema Support, I don't think we should follow the rules in PIG-2579 . While this was a good first attempt at supporting multiple data files with different schemas, it has problems: 1) The rules only support schema evolution for flat records. Nested records are not supported. 2) The AVRO specification already defines a set of rules for schema evolution. In fact, it already has code inside GenericDatumReader that will resolve data between two different schemas. We should be using that logic instead of writing logic that only works in very limited situations. 3) The idea of a merged schema is not as powerful as letting the user define the 'table' schema they expect the data to conform to. I would recommend looking at how LinkedIn added support for AVRO to Hive. Every partition can have a different schema, but there is a single 'table' schema. Hive will map the file/partition schema(s) to the 'table' schema using the logic already inside AVRO.
          Hide
          Joseph Adler added a comment -

          Aaron Klish: I agree completely.

          I don't think this version of AvroStorage does anything special with multiple schemas. If you don't specify a schema, it picks the schema from the most recently written Avro file. If you do specify a schema, it uses that schema. Provided that you follow the rules for evolving schemas, everything should work correctly.

          Show
          Joseph Adler added a comment - Aaron Klish : I agree completely. I don't think this version of AvroStorage does anything special with multiple schemas. If you don't specify a schema, it picks the schema from the most recently written Avro file. If you do specify a schema, it uses that schema. Provided that you follow the rules for evolving schemas, everything should work correctly.
          Hide
          Cheolsoo Park added a comment -

          Here is what I am going to do. I will try to commit this patch as it is now because the patches are getting huge. I just need +1 from another committer to commit them because I modified the patches.

          Then, I will open follow up jiras to implement missing features including multiple schema support, etc. We can continue our discussion there.

          Does this make sense to everyone?

          Show
          Cheolsoo Park added a comment - Here is what I am going to do. I will try to commit this patch as it is now because the patches are getting huge. I just need +1 from another committer to commit them because I modified the patches. Then, I will open follow up jiras to implement missing features including multiple schema support, etc. We can continue our discussion there. Does this make sense to everyone?
          Hide
          Russell Jurney added a comment -

          Makes sense. Lets get this in.

          Show
          Russell Jurney added a comment - Makes sense. Lets get this in.
          Hide
          Dmitriy V. Ryaboy added a comment -

          Serious question: is there a reason to put this in Pig rather than keep elsewhere, where you can iterate without being tied to Pig's release cycle?

          Show
          Dmitriy V. Ryaboy added a comment - Serious question: is there a reason to put this in Pig rather than keep elsewhere, where you can iterate without being tied to Pig's release cycle?
          Hide
          Jakob Homan added a comment -

          Serious question: is there a reason to put this in Pig rather than keep elsewhere, where you can iterate without being tied to Pig's release cycle?

          Having tried that with the Avro Serde/Haivvreo, I'd say the code is better treated as part of Hive since it's wasn't getting the correct amount of attention it deserved in github. There's a definite cost to keeping the components in sync, but there's a strong benefit to making it easy for people to interact with Avro through Pig right out of the box.

          Show
          Jakob Homan added a comment - Serious question: is there a reason to put this in Pig rather than keep elsewhere, where you can iterate without being tied to Pig's release cycle? Having tried that with the Avro Serde/Haivvreo, I'd say the code is better treated as part of Hive since it's wasn't getting the correct amount of attention it deserved in github. There's a definite cost to keeping the components in sync, but there's a strong benefit to making it easy for people to interact with Avro through Pig right out of the box.
          Hide
          Cheolsoo Park added a comment -

          The main reason is because Avro is a file format that Pig should support, and it's more convenient to use it if AvroStorage is a built-in storage. Here is the discussion in the past:
          http://mail-archives.apache.org/mod_mbox/pig-user/201208.mbox/%3C27EE5059-F811-4E19-B1A3-951B4BB3BDDF%40hortonworks.com%3E

          Show
          Cheolsoo Park added a comment - The main reason is because Avro is a file format that Pig should support, and it's more convenient to use it if AvroStorage is a built-in storage. Here is the discussion in the past: http://mail-archives.apache.org/mod_mbox/pig-user/201208.mbox/%3C27EE5059-F811-4E19-B1A3-951B4BB3BDDF%40hortonworks.com%3E
          Hide
          Frederic Rechtenstein added a comment -

          Hi,

          This looks very nice, I am really looking forward for it to be released.

          Does it make sense to have an option to include the input file path with each tuple (like -tagsource in PigStorage) ?

          I understand that this would add one more item to an already long list of options. But there are some real use cases needing this feature and it would make AvroStorage more similar to PigStorage.

          Show
          Frederic Rechtenstein added a comment - Hi, This looks very nice, I am really looking forward for it to be released. Does it make sense to have an option to include the input file path with each tuple (like -tagsource in PigStorage) ? I understand that this would add one more item to an already long list of options. But there are some real use cases needing this feature and it would make AvroStorage more similar to PigStorage.
          Hide
          Joseph Adler added a comment -

          I like the -tagsource option idea. Should we allow the user to provide a name for the "tag source" field? (If we picked a name like "tagSource," and there was already a field in the avro Schema called "tagSource", I'm concerned that we'd have to deal with that conflict. I think it would be cleaner to let the end user resolve the naming issue.)

          Show
          Joseph Adler added a comment - I like the -tagsource option idea. Should we allow the user to provide a name for the "tag source" field? (If we picked a name like "tagSource," and there was already a field in the avro Schema called "tagSource", I'm concerned that we'd have to deal with that conflict. I think it would be cleaner to let the end user resolve the naming issue.)
          Hide
          Prashant Kommireddi added a comment -

          Joseph Adler note the "tagsource" option on PigStorage is deprecated in 0.12 and replaced with "tagFile". Additionally, there is an option "tagPath" for getting the entire path and not just the filename. See PIG-2857

          Show
          Prashant Kommireddi added a comment - Joseph Adler note the "tagsource" option on PigStorage is deprecated in 0.12 and replaced with "tagFile". Additionally, there is an option "tagPath" for getting the entire path and not just the filename. See PIG-2857
          Hide
          Jonathan Coveney added a comment -

          Joseph, as gratitude for the effort you've put into this, I spent a chunk of my day giving it some eyes. Thank you for this! It also is timely, as I need to be ramping up on avro myself, so good stuff. I gave some comments in the RB, though nothing major, mostly style stuff, some error handling, etc. Except a +1 soon

          I think it should be in the main Pig branch.

          As an aside, what is the plan w.r.t. Pig types that are not Avro types?

          Show
          Jonathan Coveney added a comment - Joseph, as gratitude for the effort you've put into this, I spent a chunk of my day giving it some eyes. Thank you for this! It also is timely, as I need to be ramping up on avro myself, so good stuff. I gave some comments in the RB, though nothing major, mostly style stuff, some error handling, etc. Except a +1 soon I think it should be in the main Pig branch. As an aside, what is the plan w.r.t. Pig types that are not Avro types?
          Hide
          Joseph Adler added a comment -

          Sorry to have taken so long to reply.

          I map any Pig type to a union of an Avro Type and Null. Here are the type mappings that I implemented:

          Bag -> Array
          Big Chararray -> String
          Byte Array -> Bytes
          Chararray -> String
          Datetime -> Long
          Double -> Double
          Float -> Float
          Integer -> Int
          Map -> Map
          Null -> Null
          Tuple -> Record

          Byte, Error, Generic Writable, Internal Map, Unknown aren't mapped to anything yet. Do we need to store these as well?

          Show
          Joseph Adler added a comment - Sorry to have taken so long to reply. I map any Pig type to a union of an Avro Type and Null. Here are the type mappings that I implemented: Bag -> Array Big Chararray -> String Byte Array -> Bytes Chararray -> String Datetime -> Long Double -> Double Float -> Float Integer -> Int Map -> Map Null -> Null Tuple -> Record Byte, Error, Generic Writable, Internal Map, Unknown aren't mapped to anything yet. Do we need to store these as well?
          Hide
          Rohini Palaniswamy added a comment -

          Joseph,
          Thanks for the good work. Planning to go over it more in detail this week. At a glance, had the question why doesn't AvroStorage (even the old one) implement LoadPushDown (for column pruning) ? If it's just a miss we can create a separate jira for it.

          Show
          Rohini Palaniswamy added a comment - Joseph, Thanks for the good work. Planning to go over it more in detail this week. At a glance, had the question why doesn't AvroStorage (even the old one) implement LoadPushDown (for column pruning) ? If it's just a miss we can create a separate jira for it.
          Hide
          Joseph Adler added a comment -

          Rohini Palaniswamy: Great question. I definitely implemented that interface in an earlier version; I'm not sure what happened to the code. Let me go through the patches to figure that one out.

          Show
          Joseph Adler added a comment - Rohini Palaniswamy : Great question. I definitely implemented that interface in an earlier version; I'm not sure what happened to the code. Let me go through the patches to figure that one out.
          Hide
          Joseph Adler added a comment -

          Rohini Palaniswamy OK, looks like I implemented the helper functions, and implemented the functionality for Trevni, but didn't implement it for AvroStorage. Will follow up with a patch.

          Show
          Joseph Adler added a comment - Rohini Palaniswamy OK, looks like I implemented the helper functions, and implemented the functionality for Trevni, but didn't implement it for AvroStorage. Will follow up with a patch.
          Hide
          Rohini Palaniswamy added a comment -

          Thanks . Would be very nice to have it. Just saw that Jonathan had asked the same question earlier in his review comments.

          Show
          Rohini Palaniswamy added a comment - Thanks . Would be very nice to have it. Just saw that Jonathan had asked the same question earlier in his review comments.
          Hide
          Paul Mazak added a comment -

          One simple workaround for us was to override AvroStorage's checkSchema this way.

          /**
           * In Pig script do:
           * REGISTER 'lib/this.jar'
           * DEFINE AvroStorage com.this.JoinableAvroStorage;
           */
          public class JoinableAvroStorage extends AvroStorage {
            
            @Override
            public void checkSchema(ResourceSchema s) throws IOException {
              try {
                super.checkSchema(s);
              }
              catch (SchemaParseException spe) {
                ResourceFieldSchema[] pigFields = s.getFields();
                for (int i = 0; i < pigFields.length; i++) {
                  String outname = pigFields[i].getName();
                  if (outname.contains("::")) {
                    String newOutname = outname.split("::")[1];
                    pigFields[i].setName(newOutname);
                  }
                }
                super.checkSchema(s);
              }
            }
          }
          
          Show
          Paul Mazak added a comment - One simple workaround for us was to override AvroStorage's checkSchema this way. /** * In Pig script do : * REGISTER 'lib/ this .jar' * DEFINE AvroStorage com. this .JoinableAvroStorage; */ public class JoinableAvroStorage extends AvroStorage { @Override public void checkSchema(ResourceSchema s) throws IOException { try { super .checkSchema(s); } catch (SchemaParseException spe) { ResourceFieldSchema[] pigFields = s.getFields(); for ( int i = 0; i < pigFields.length; i++) { String outname = pigFields[i].getName(); if (outname.contains( "::" )) { String newOutname = outname.split( "::" )[1]; pigFields[i].setName(newOutname); } } super .checkSchema(s); } } }
          Hide
          Joseph Adler added a comment -

          Incremental patch that adds support for push down projections, fixed some bugs with options, gets all the test cases working again

          Show
          Joseph Adler added a comment - Incremental patch that adds support for push down projections, fixed some bugs with options, gets all the test cases working again
          Hide
          Joseph Adler added a comment -

          I'm getting confused by the names of the diffs. This one is a diff from trunk, as of now.

          Show
          Joseph Adler added a comment - I'm getting confused by the names of the diffs. This one is a diff from trunk, as of now.
          Hide
          Cheolsoo Park added a comment -

          I updated the patch again hoping that this will be committed - PIG-3015-22June2013.diff.

          The patch that Joe uploaded (PIG-3015-20May2013.diff) was missing a couple of test files and included few unnecessary changes. I added missing files and cleaned up the patch a bit. The unit tests pass in Hadoop 20 and 23.

          AFAICT, most comments in the jira have been incorporated, so I think we should commit this patch. I am maintaining the code in my private repo at work now, but I'd love to see this going in...

          Show
          Cheolsoo Park added a comment - I updated the patch again hoping that this will be committed - PIG-3015 -22June2013.diff. The patch that Joe uploaded ( PIG-3015 -20May2013.diff) was missing a couple of test files and included few unnecessary changes. I added missing files and cleaned up the patch a bit. The unit tests pass in Hadoop 20 and 23. AFAICT, most comments in the jira have been incorporated, so I think we should commit this patch. I am maintaining the code in my private repo at work now, but I'd love to see this going in...
          Hide
          Dmitriy V. Ryaboy added a comment -

          +1

          if we find more stuff, we can open other jiras. Let's get this into trunk.

          Show
          Dmitriy V. Ryaboy added a comment - +1 if we find more stuff, we can open other jiras. Let's get this into trunk.
          Hide
          Cheolsoo Park added a comment -

          Thank you Dmitriy! I will commit it today!

          Show
          Cheolsoo Park added a comment - Thank you Dmitriy! I will commit it today!
          Hide
          Cheolsoo Park added a comment -

          It's committed to trunk!

          Thank you Joe for the great contribution! Thanks everyone who reviewed/tested the patch!

          Show
          Cheolsoo Park added a comment - It's committed to trunk! Thank you Joe for the great contribution! Thanks everyone who reviewed/tested the patch!
          Hide
          Anup Ahire added a comment -

          Shouldn't we loose the timezone information if we convert datetime to long ?

          After going through the jira for datetime, it appears that datetime->long wasn't supported to avoid timezone information loss.

          Thanks !!

          Show
          Anup Ahire added a comment - Shouldn't we loose the timezone information if we convert datetime to long ? After going through the jira for datetime, it appears that datetime->long wasn't supported to avoid timezone information loss. Thanks !!
          Hide
          Cheolsoo Park added a comment -

          Anup Ahire, can you please open a separate jira? We can address your concern there.

          Show
          Cheolsoo Park added a comment - Anup Ahire , can you please open a separate jira? We can address your concern there.
          Hide
          Anup Ahire added a comment -

          Done.PIG-3391 Thanks!!

          Show
          Anup Ahire added a comment - Done. PIG-3391 Thanks!!

            People

            • Assignee:
              Joseph Adler
              Reporter:
              Joseph Adler
            • Votes:
              6 Vote for this issue
              Watchers:
              32 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development