You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@pig.apache.org by Kevin Weil <ke...@gmail.com> on 2008/10/12 12:25:49 UTC

question on load functions in the types branch

I am having issues with a custom load function that reads protocol buffers.
It worked with pig 0.1, and now after the refactoring to support 0.2/types,
I can't get it to do anything past the line

org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher
- 0% complete

My reader uses the same getNext() and bindTo() that it used before (modulo
things like using the required TupleFactory now in getNext), and I now
implement determineSchema as well.  There's no global way to take a series
of bytes from a protobuf and turn it into, say, a tuple, since it depends on
the message, so I assert false in all the bytesTo* functions and then return
a proper schema in determineSchema.  Based on the
document<http://wiki.apache.org/pig/TrunkToTypesChanges>sent out a
couple days ago, I think this is correct, but stop me here if I'm
wrong.

A simplified version of my determineSchema is

@Override public Schema determineSchema(URL fileName) throws IOException {
        try {
            List<Schema.FieldSchema> schemaList = new
ArrayList<Schema.FieldSchema>();
            schemaList.add(new FieldSchema("version", null,
DataType.CHARARRAY));
            return new Schema(schemaList);
        }
        catch(...)
}

The constructor of my LogReader class takes a string argument, and so a
simple Pig script is

<register jars>
all_files = LOAD 'my_file' USING com.....logging.LogReader('client') AS
(version: chararray);
dump all_files;

It's at this point that I get the MapReduceLauncher at 0% complete and it
just sits there forever.  Here is what I do see, in grunt.  When I type the
all_files = ... line, one of my LogReader classes gets instantiated with the
correct string argument.  When I type the dump line, another gets
constructed with the correct string argument, then two LogReader classes get
instantiated with a no-argument constructor, and then one more gets created
with the correct string 1-argument constructor.  This is all on a Linux
setup with just one machine running in Hadoop mode to test.  I don't
understand why the two LogReaders get created with the zero-argument
constructor here -- that shouldn't need to be defined, right?  Perhaps it's
a clue to what's going wrong.

By the way, in my LogReader class, other than all the constructors getting
called, no other function calls seem to happen.  In particular, bindTo,
getNext, and determineSchema are never called.

Thanks in advance,
Kevin

p.s. In case it helps, the stacks at the point where the correct 1-argument
constructor LogReader(String) is being instantiated is

    at com....logging.LogReader.<init>(LogReader.java:71)
    at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
    at
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)
    at
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)
    at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
    at
org.apache.pig.impl.PigContext.instantiateFuncFromSpec(PigContext.java:416)
    at org.apache.pig.impl.logicalLayer.LOLoad.<init>(LOLoad.java:64)
    at
org.apache.pig.impl.logicalLayer.parser.QueryParser.LoadClause(QueryParser.java:1106)
    at
org.apache.pig.impl.logicalLayer.parser.QueryParser.BaseExpr(QueryParser.java:889)
    at
org.apache.pig.impl.logicalLayer.parser.QueryParser.Expr(QueryParser.java:748)
    at
org.apache.pig.impl.logicalLayer.parser.QueryParser.Parse(QueryParser.java:549)
    at
org.apache.pig.impl.logicalLayer.LogicalPlanBuilder.parse(LogicalPlanBuilder.java:60)
    at org.apache.pig.PigServer.parseQuery(PigServer.java:295)
    at org.apache.pig.PigServer.clonePlan(PigServer.java:330)
    at org.apache.pig.PigServer.compileLp(PigServer.java:666)
    at org.apache.pig.PigServer.compileLp(PigServer.java:655)
    at org.apache.pig.PigServer.store(PigServer.java:433)
    at org.apache.pig.PigServer.store(PigServer.java:421)
    at org.apache.pig.PigServer.openIterator(PigServer.java:384)
    at
org.apache.pig.tools.grunt.GruntParser.processDump(GruntParser.java:269)
    at
org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptParser.java:178)
    at
org.apache.pig.tools.grunt.GruntParser.parseContOnError(GruntParser.java:94)
    at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:58)
    at org.apache.pig.Main.main(Main.java:282)

while the stack at the point where the incorrect zero-argument constructor
gets called is

    at com.......logging.LogReader.<init>(LogReader.java:80)
    at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
    at
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)
    at
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)
    at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
    at java.lang.Class.newInstance0(Class.java:355)
    at java.lang.Class.newInstance(Class.java:308)
    at
org.apache.pig.impl.PigContext.instantiateFuncFromSpec(PigContext.java:418)
    at
org.apache.pig.impl.PigContext.instantiateFuncFromSpec(PigContext.java:454)
    at
org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POCast.instantiateFunc(POCast.java:66)
    at
org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POCast.setLoadFSpec(POCast.java:71)
    at
org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogToPhyTranslationVisitor.visit(LogToPhyTranslationVisitor.java:1153)
    at org.apache.pig.impl.logicalLayer.LOCast.visit(LOCast.java:58)
    at org.apache.pig.impl.logicalLayer.LOCast.visit(LOCast.java:27)
    at
org.apache.pig.impl.plan.DependencyOrderWalkerWOSeenChk.walk(DependencyOrderWalkerWOSeenChk.java:68)
    at
org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogToPhyTranslationVisitor.visit(LogToPhyTranslationVisitor.java:805)
    at org.apache.pig.impl.logicalLayer.LOForEach.visit(LOForEach.java:105)
    at org.apache.pig.impl.logicalLayer.LOForEach.visit(LOForEach.java:40)
    at
org.apache.pig.impl.plan.DependencyOrderWalker.walk(DependencyOrderWalker.java:68)
    at org.apache.pig.impl.plan.PlanVisitor.visit(PlanVisitor.java:51)
    at
org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.compile(HExecutionEngine.java:232)
    at org.apache.pig.PigServer.compilePp(PigServer.java:731)
    at
org.apache.pig.PigServer.executeCompiledLogicalPlan(PigServer.java:644)
    at org.apache.pig.PigServer.store(PigServer.java:452)
    at org.apache.pig.PigServer.store(PigServer.java:421)
    at org.apache.pig.PigServer.openIterator(PigServer.java:384)
    at
org.apache.pig.tools.grunt.GruntParser.processDump(GruntParser.java:269)
    at
org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptParser.java:178)
    at
org.apache.pig.tools.grunt.GruntParser.parseContOnError(GruntParser.java:94)
    at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:58)
    at org.apache.pig.Main.main(Main.java:282)

RE: question on load functions in the types branch

Posted by Pradeep Kamath <pr...@yahoo-inc.com>.
Hi Kevin,

  Currently this is work in progress but here is what the
determineSchema API might look like once done:

/**

     * Find the schema from the loader.  This function will be called at
parse time

     * (not run time) to see if the loader can provide a schema for the
data.  The

     * loader may be able to do this if the data is self describing
(e.g. JSON).  If

     * the loader cannot determine the schema, it can return a null.

     * LoadFunc implementations which need to open the input "fileName",
can use 

     * FileLocalizer.open(String fileName, ExecType execType,
DataStorage storage) to get

     * an InputStream which they can use to initialize their loader
implementation. They

     * can then use this to read the input data to discover the schema.
Note: this will

     * work only when the fileName represents a file on Local File
System or Hadoop file 

     * system

     * @param fileName Name of the file to be read.(this will be the
same as the filename 

     * in the "load statement of the script)

     * @param execType - execution mode of the pig script - one of
ExecType.LOCAL or ExecType.MAPREDUCE

     * @param storage - the DataStorage object corresponding to the
execType

     * @return a Schema describing the data if possible, or null
otherwise.

     * @throws IOException.

     */

    public Schema determineSchema(String fileName, ExecType execType,
DataStorage storage) throws IOException;

 

If you will be having a static schema and you don't need access to the
input file to determine it, then you will not be affected by these API
changes.

 

Hope this helps.

 

Thanks,

Pradeep

 

-----Original Message-----
From: Kevin Weil [mailto:kevinweil@gmail.com] 
Sent: Monday, October 13, 2008 6:31 PM
To: pig-user@incubator.apache.org
Subject: Re: question on load functions in the types branch

 

Hi Pradeep,

 

Great, I'm looking forward to seeing the checkin!  Will there be a

significant difference in the determineSchema API?  I'm writing a bunch
of

code that uses it because my data is position dependent, and I don't
think

bytesTo* will work.  My load function does properly return typed tuples

rather than bytearrays.  If the change will require significant
refactoring,

maybe you can let me know what the determineSchema method signature will

look like so I can plan ahead a bit?

 

Thanks,

Kevin

 

On Mon, Oct 13, 2008 at 4:13 PM, Pradeep Kamath
<pr...@yahoo-inc.com>wrote:

 

> Hi Kevin,

> 

>  determineSchema() is currently never being called since the code in

> LOLoad which is supposed to call it is broken. I am fixing that as
part

> of https://issues.apache.org/jira/browse/PIG-468. However

> determineSchema() may not be the cause of the issue you are describing

> here. A related note on determineSchema() is that the Loader's
getNext()

> should return data with types matching the schema in determineSchema()

> (if determineSchema() is implemented, if it's not, then loader's

> getNext() should return DataByteArrays).

> 

> 

> 

> The determineSchema() method's signature will also change as part of
the

> patch for PIG-468.

> 

> 

> 

> For now, if your loader's getNext() returns a tuple of DataByteArrays

> then you should not implement determineSchema() and provide suitable

> bytesTo* implementations. Utf8StorageConvertor has implementations for

> Utf8 input data.

> 

> 

> 

> Thanks,

> 

> Pradeep

> 

> 

> 

> -----Original Message-----

> From: Kevin Weil [mailto:kevinweil@gmail.com]

> Sent: Sunday, October 12, 2008 3:26 AM

> To: pig-user@incubator.apache.org

> Subject: question on load functions in the types branch

> 

> 

> 

> I am having issues with a custom load function that reads protocol

> buffers.

> 

> It worked with pig 0.1, and now after the refactoring to support

> 0.2/types,

> 

> I can't get it to do anything past the line

> 

> 

> 

>
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLa

> uncher

> 

> - 0% complete

> 

> 

> 

> My reader uses the same getNext() and bindTo() that it used before

> (modulo

> 

> things like using the required TupleFactory now in getNext), and I now

> 

> implement determineSchema as well.  There's no global way to take a

> series

> 

> of bytes from a protobuf and turn it into, say, a tuple, since it

> depends on

> 

> the message, so I assert false in all the bytesTo* functions and then

> return

> 

> a proper schema in determineSchema.  Based on the

> 

> document<http://wiki.apache.org/pig/TrunkToTypesChanges>sent out a

> 

> couple days ago, I think this is correct, but stop me here if I'm

> 

> wrong.

> 

> 

> 

> A simplified version of my determineSchema is

> 

> 

> 

> @Override public Schema determineSchema(URL fileName) throws
IOException

> {

> 

>        try {

> 

>            List<Schema.FieldSchema> schemaList = new

> 

> ArrayList<Schema.FieldSchema>();

> 

>            schemaList.add(new FieldSchema("version", null,

> 

> DataType.CHARARRAY));

> 

>            return new Schema(schemaList);

> 

>        }

> 

>        catch(...)

> 

> }

> 

> 

> 

> The constructor of my LogReader class takes a string argument, and so
a

> 

> simple Pig script is

> 

> 

> 

> <register jars>

> 

> all_files = LOAD 'my_file' USING com.....logging.LogReader('client')
AS

> 

> (version: chararray);

> 

> dump all_files;

> 

> 

> 

> It's at this point that I get the MapReduceLauncher at 0% complete and

> it

> 

> just sits there forever.  Here is what I do see, in grunt.  When I
type

> the

> 

> all_files = ... line, one of my LogReader classes gets instantiated
with

> the

> 

> correct string argument.  When I type the dump line, another gets

> 

> constructed with the correct string argument, then two LogReader
classes

> get

> 

> instantiated with a no-argument constructor, and then one more gets

> created

> 

> with the correct string 1-argument constructor.  This is all on a
Linux

> 

> setup with just one machine running in Hadoop mode to test.  I don't

> 

> understand why the two LogReaders get created with the zero-argument

> 

> constructor here -- that shouldn't need to be defined, right?  Perhaps

> it's

> 

> a clue to what's going wrong.

> 

> 

> 

> By the way, in my LogReader class, other than all the constructors

> getting

> 

> called, no other function calls seem to happen.  In particular,
bindTo,

> 

> getNext, and determineSchema are never called.

> 

> 

> 

> Thanks in advance,

> 

> Kevin

> 

> 

> 

> p.s. In case it helps, the stacks at the point where the correct

> 1-argument

> 

> constructor LogReader(String) is being instantiated is

> 

> 

> 

>    at com....logging.LogReader.<init>(LogReader.java:71)

> 

>    at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native

> Method)

> 

>    at

> 

>
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorA

> ccessorImpl.java:39)

> 

>    at

> 

>
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCons

> tructorAccessorImpl.java:27)

> 

>    at java.lang.reflect.Constructor.newInstance(Constructor.java:513)

> 

>    at

> 

>
org.apache.pig.impl.PigContext.instantiateFuncFromSpec(PigContext.java:4

> 16)

> 

>    at org.apache.pig.impl.logicalLayer.LOLoad.<init>(LOLoad.java:64)

> 

>    at

> 

>
org.apache.pig.impl.logicalLayer.parser.QueryParser.LoadClause(QueryPars

> er.java:1106)

> 

>    at

> 

>
org.apache.pig.impl.logicalLayer.parser.QueryParser.BaseExpr(QueryParser

> .java:889)

> 

>    at

> 

>
org.apache.pig.impl.logicalLayer.parser.QueryParser.Expr(QueryParser.jav

> a:748)

> 

>    at

> 

>
org.apache.pig.impl.logicalLayer.parser.QueryParser.Parse(QueryParser.ja

> va:549)

> 

>    at

> 

>
org.apache.pig.impl.logicalLayer.LogicalPlanBuilder.parse(LogicalPlanBui

> lder.java:60)

> 

>    at org.apache.pig.PigServer.parseQuery(PigServer.java:295)

> 

>    at org.apache.pig.PigServer.clonePlan(PigServer.java:330)

> 

>    at org.apache.pig.PigServer.compileLp(PigServer.java:666)

> 

>    at org.apache.pig.PigServer.compileLp(PigServer.java:655)

> 

>    at org.apache.pig.PigServer.store(PigServer.java:433)

> 

>    at org.apache.pig.PigServer.store(PigServer.java:421)

> 

>    at org.apache.pig.PigServer.openIterator(PigServer.java:384)

> 

>    at

> 

>
org.apache.pig.tools.grunt.GruntParser.processDump(GruntParser.java:269)

> 

>    at

> 

>
org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptPar

> ser.java:178)

> 

>    at

> 

>
org.apache.pig.tools.grunt.GruntParser.parseContOnError(GruntParser.java

> :94)

> 

>    at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:58)

> 

>    at org.apache.pig.Main.main(Main.java:282)

> 

> 

> 

> while the stack at the point where the incorrect zero-argument

> constructor

> 

> gets called is

> 

> 

> 

>    at com.......logging.LogReader.<init>(LogReader.java:80)

> 

>    at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native

> Method)

> 

>    at

> 

>
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorA

> ccessorImpl.java:39)

> 

>    at

> 

>
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCons

> tructorAccessorImpl.java:27)

> 

>    at java.lang.reflect.Constructor.newInstance(Constructor.java:513)

> 

>    at java.lang.Class.newInstance0(Class.java:355)

> 

>    at java.lang.Class.newInstance(Class.java:308)

> 

>    at

> 

>
org.apache.pig.impl.PigContext.instantiateFuncFromSpec(PigContext.java:4

> 18)

> 

>     at

> 

>
org.apache.pig.impl.PigContext.instantiateFuncFromSpec(PigContext.java:4

> 54)

> 

>    at

> 

>
org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOp

> erators.POCast.instantiateFunc(POCast.java:66)

> 

>    at

> 

>
org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOp

> erators.POCast.setLoadFSpec(POCast.java:71)

> 

>    at

> 

>
org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogToPhyTran

> slationVisitor.visit(LogToPhyTranslationVisitor.java:1153)

> 

>    at org.apache.pig.impl.logicalLayer.LOCast.visit(LOCast.java:58)

> 

>    at org.apache.pig.impl.logicalLayer.LOCast.visit(LOCast.java:27)

> 

>    at

> 

>
org.apache.pig.impl.plan.DependencyOrderWalkerWOSeenChk.walk(DependencyO

> rderWalkerWOSeenChk.java:68)

> 

>    at

> 

>
org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogToPhyTran

> slationVisitor.visit(LogToPhyTranslationVisitor.java:805)

> 

>    at

> org.apache.pig.impl.logicalLayer.LOForEach.visit(LOForEach.java:105)

> 

>    at

> org.apache.pig.impl.logicalLayer.LOForEach.visit(LOForEach.java:40)

> 

>    at

> 

>
org.apache.pig.impl.plan.DependencyOrderWalker.walk(DependencyOrderWalke

> r.java:68)

> 

>    at org.apache.pig.impl.plan.PlanVisitor.visit(PlanVisitor.java:51)

> 

>    at

> 

>
org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.compile(H

> ExecutionEngine.java:232)

> 

>    at org.apache.pig.PigServer.compilePp(PigServer.java:731)

> 

>    at

> 

>
org.apache.pig.PigServer.executeCompiledLogicalPlan(PigServer.java:644)

> 

>    at org.apache.pig.PigServer.store(PigServer.java:452)

> 

>    at org.apache.pig.PigServer.store(PigServer.java:421)

> 

>    at org.apache.pig.PigServer.openIterator(PigServer.java:384)

> 

>    at

> 

>
org.apache.pig.tools.grunt.GruntParser.processDump(GruntParser.java:269)

> 

>    at

> 

>
org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptPar

> ser.java:178)

> 

>    at

> 

>
org.apache.pig.tools.grunt.GruntParser.parseContOnError(GruntParser.java

> :94)

> 

>    at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:58)

> 

>    at org.apache.pig.Main.main(Main.java:282)

> 

> 


Re: question on load functions in the types branch

Posted by Kevin Weil <ke...@gmail.com>.
Hi Pradeep,

Great, I'm looking forward to seeing the checkin!  Will there be a
significant difference in the determineSchema API?  I'm writing a bunch of
code that uses it because my data is position dependent, and I don't think
bytesTo* will work.  My load function does properly return typed tuples
rather than bytearrays.  If the change will require significant refactoring,
maybe you can let me know what the determineSchema method signature will
look like so I can plan ahead a bit?

Thanks,
Kevin

On Mon, Oct 13, 2008 at 4:13 PM, Pradeep Kamath <pr...@yahoo-inc.com>wrote:

> Hi Kevin,
>
>  determineSchema() is currently never being called since the code in
> LOLoad which is supposed to call it is broken. I am fixing that as part
> of https://issues.apache.org/jira/browse/PIG-468. However
> determineSchema() may not be the cause of the issue you are describing
> here. A related note on determineSchema() is that the Loader's getNext()
> should return data with types matching the schema in determineSchema()
> (if determineSchema() is implemented, if it's not, then loader's
> getNext() should return DataByteArrays).
>
>
>
> The determineSchema() method's signature will also change as part of the
> patch for PIG-468.
>
>
>
> For now, if your loader's getNext() returns a tuple of DataByteArrays
> then you should not implement determineSchema() and provide suitable
> bytesTo* implementations. Utf8StorageConvertor has implementations for
> Utf8 input data.
>
>
>
> Thanks,
>
> Pradeep
>
>
>
> -----Original Message-----
> From: Kevin Weil [mailto:kevinweil@gmail.com]
> Sent: Sunday, October 12, 2008 3:26 AM
> To: pig-user@incubator.apache.org
> Subject: question on load functions in the types branch
>
>
>
> I am having issues with a custom load function that reads protocol
> buffers.
>
> It worked with pig 0.1, and now after the refactoring to support
> 0.2/types,
>
> I can't get it to do anything past the line
>
>
>
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLa
> uncher
>
> - 0% complete
>
>
>
> My reader uses the same getNext() and bindTo() that it used before
> (modulo
>
> things like using the required TupleFactory now in getNext), and I now
>
> implement determineSchema as well.  There's no global way to take a
> series
>
> of bytes from a protobuf and turn it into, say, a tuple, since it
> depends on
>
> the message, so I assert false in all the bytesTo* functions and then
> return
>
> a proper schema in determineSchema.  Based on the
>
> document<http://wiki.apache.org/pig/TrunkToTypesChanges>sent out a
>
> couple days ago, I think this is correct, but stop me here if I'm
>
> wrong.
>
>
>
> A simplified version of my determineSchema is
>
>
>
> @Override public Schema determineSchema(URL fileName) throws IOException
> {
>
>        try {
>
>            List<Schema.FieldSchema> schemaList = new
>
> ArrayList<Schema.FieldSchema>();
>
>            schemaList.add(new FieldSchema("version", null,
>
> DataType.CHARARRAY));
>
>            return new Schema(schemaList);
>
>        }
>
>        catch(...)
>
> }
>
>
>
> The constructor of my LogReader class takes a string argument, and so a
>
> simple Pig script is
>
>
>
> <register jars>
>
> all_files = LOAD 'my_file' USING com.....logging.LogReader('client') AS
>
> (version: chararray);
>
> dump all_files;
>
>
>
> It's at this point that I get the MapReduceLauncher at 0% complete and
> it
>
> just sits there forever.  Here is what I do see, in grunt.  When I type
> the
>
> all_files = ... line, one of my LogReader classes gets instantiated with
> the
>
> correct string argument.  When I type the dump line, another gets
>
> constructed with the correct string argument, then two LogReader classes
> get
>
> instantiated with a no-argument constructor, and then one more gets
> created
>
> with the correct string 1-argument constructor.  This is all on a Linux
>
> setup with just one machine running in Hadoop mode to test.  I don't
>
> understand why the two LogReaders get created with the zero-argument
>
> constructor here -- that shouldn't need to be defined, right?  Perhaps
> it's
>
> a clue to what's going wrong.
>
>
>
> By the way, in my LogReader class, other than all the constructors
> getting
>
> called, no other function calls seem to happen.  In particular, bindTo,
>
> getNext, and determineSchema are never called.
>
>
>
> Thanks in advance,
>
> Kevin
>
>
>
> p.s. In case it helps, the stacks at the point where the correct
> 1-argument
>
> constructor LogReader(String) is being instantiated is
>
>
>
>    at com....logging.LogReader.<init>(LogReader.java:71)
>
>    at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
> Method)
>
>    at
>
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorA
> ccessorImpl.java:39)
>
>    at
>
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCons
> tructorAccessorImpl.java:27)
>
>    at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
>
>    at
>
> org.apache.pig.impl.PigContext.instantiateFuncFromSpec(PigContext.java:4
> 16)
>
>    at org.apache.pig.impl.logicalLayer.LOLoad.<init>(LOLoad.java:64)
>
>    at
>
> org.apache.pig.impl.logicalLayer.parser.QueryParser.LoadClause(QueryPars
> er.java:1106)
>
>    at
>
> org.apache.pig.impl.logicalLayer.parser.QueryParser.BaseExpr(QueryParser
> .java:889)
>
>    at
>
> org.apache.pig.impl.logicalLayer.parser.QueryParser.Expr(QueryParser.jav
> a:748)
>
>    at
>
> org.apache.pig.impl.logicalLayer.parser.QueryParser.Parse(QueryParser.ja
> va:549)
>
>    at
>
> org.apache.pig.impl.logicalLayer.LogicalPlanBuilder.parse(LogicalPlanBui
> lder.java:60)
>
>    at org.apache.pig.PigServer.parseQuery(PigServer.java:295)
>
>    at org.apache.pig.PigServer.clonePlan(PigServer.java:330)
>
>    at org.apache.pig.PigServer.compileLp(PigServer.java:666)
>
>    at org.apache.pig.PigServer.compileLp(PigServer.java:655)
>
>    at org.apache.pig.PigServer.store(PigServer.java:433)
>
>    at org.apache.pig.PigServer.store(PigServer.java:421)
>
>    at org.apache.pig.PigServer.openIterator(PigServer.java:384)
>
>    at
>
> org.apache.pig.tools.grunt.GruntParser.processDump(GruntParser.java:269)
>
>    at
>
> org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptPar
> ser.java:178)
>
>    at
>
> org.apache.pig.tools.grunt.GruntParser.parseContOnError(GruntParser.java
> :94)
>
>    at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:58)
>
>    at org.apache.pig.Main.main(Main.java:282)
>
>
>
> while the stack at the point where the incorrect zero-argument
> constructor
>
> gets called is
>
>
>
>    at com.......logging.LogReader.<init>(LogReader.java:80)
>
>    at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
> Method)
>
>    at
>
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorA
> ccessorImpl.java:39)
>
>    at
>
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCons
> tructorAccessorImpl.java:27)
>
>    at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
>
>    at java.lang.Class.newInstance0(Class.java:355)
>
>    at java.lang.Class.newInstance(Class.java:308)
>
>    at
>
> org.apache.pig.impl.PigContext.instantiateFuncFromSpec(PigContext.java:4
> 18)
>
>     at
>
> org.apache.pig.impl.PigContext.instantiateFuncFromSpec(PigContext.java:4
> 54)
>
>    at
>
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOp
> erators.POCast.instantiateFunc(POCast.java:66)
>
>    at
>
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOp
> erators.POCast.setLoadFSpec(POCast.java:71)
>
>    at
>
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogToPhyTran
> slationVisitor.visit(LogToPhyTranslationVisitor.java:1153)
>
>    at org.apache.pig.impl.logicalLayer.LOCast.visit(LOCast.java:58)
>
>    at org.apache.pig.impl.logicalLayer.LOCast.visit(LOCast.java:27)
>
>    at
>
> org.apache.pig.impl.plan.DependencyOrderWalkerWOSeenChk.walk(DependencyO
> rderWalkerWOSeenChk.java:68)
>
>    at
>
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogToPhyTran
> slationVisitor.visit(LogToPhyTranslationVisitor.java:805)
>
>    at
> org.apache.pig.impl.logicalLayer.LOForEach.visit(LOForEach.java:105)
>
>    at
> org.apache.pig.impl.logicalLayer.LOForEach.visit(LOForEach.java:40)
>
>    at
>
> org.apache.pig.impl.plan.DependencyOrderWalker.walk(DependencyOrderWalke
> r.java:68)
>
>    at org.apache.pig.impl.plan.PlanVisitor.visit(PlanVisitor.java:51)
>
>    at
>
> org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.compile(H
> ExecutionEngine.java:232)
>
>    at org.apache.pig.PigServer.compilePp(PigServer.java:731)
>
>    at
>
> org.apache.pig.PigServer.executeCompiledLogicalPlan(PigServer.java:644)
>
>    at org.apache.pig.PigServer.store(PigServer.java:452)
>
>    at org.apache.pig.PigServer.store(PigServer.java:421)
>
>    at org.apache.pig.PigServer.openIterator(PigServer.java:384)
>
>    at
>
> org.apache.pig.tools.grunt.GruntParser.processDump(GruntParser.java:269)
>
>    at
>
> org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptPar
> ser.java:178)
>
>    at
>
> org.apache.pig.tools.grunt.GruntParser.parseContOnError(GruntParser.java
> :94)
>
>    at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:58)
>
>    at org.apache.pig.Main.main(Main.java:282)
>
>

RE: question on load functions in the types branch

Posted by Pradeep Kamath <pr...@yahoo-inc.com>.
Hi Kevin,

  determineSchema() is currently never being called since the code in
LOLoad which is supposed to call it is broken. I am fixing that as part
of https://issues.apache.org/jira/browse/PIG-468. However
determineSchema() may not be the cause of the issue you are describing
here. A related note on determineSchema() is that the Loader's getNext()
should return data with types matching the schema in determineSchema()
(if determineSchema() is implemented, if it's not, then loader's
getNext() should return DataByteArrays). 

 

The determineSchema() method's signature will also change as part of the
patch for PIG-468.

 

For now, if your loader's getNext() returns a tuple of DataByteArrays
then you should not implement determineSchema() and provide suitable
bytesTo* implementations. Utf8StorageConvertor has implementations for
Utf8 input data.

 

Thanks,

Pradeep

 

-----Original Message-----
From: Kevin Weil [mailto:kevinweil@gmail.com] 
Sent: Sunday, October 12, 2008 3:26 AM
To: pig-user@incubator.apache.org
Subject: question on load functions in the types branch

 

I am having issues with a custom load function that reads protocol
buffers.

It worked with pig 0.1, and now after the refactoring to support
0.2/types,

I can't get it to do anything past the line

 

org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLa
uncher

- 0% complete

 

My reader uses the same getNext() and bindTo() that it used before
(modulo

things like using the required TupleFactory now in getNext), and I now

implement determineSchema as well.  There's no global way to take a
series

of bytes from a protobuf and turn it into, say, a tuple, since it
depends on

the message, so I assert false in all the bytesTo* functions and then
return

a proper schema in determineSchema.  Based on the

document<http://wiki.apache.org/pig/TrunkToTypesChanges>sent out a

couple days ago, I think this is correct, but stop me here if I'm

wrong.

 

A simplified version of my determineSchema is

 

@Override public Schema determineSchema(URL fileName) throws IOException
{

        try {

            List<Schema.FieldSchema> schemaList = new

ArrayList<Schema.FieldSchema>();

            schemaList.add(new FieldSchema("version", null,

DataType.CHARARRAY));

            return new Schema(schemaList);

        }

        catch(...)

}

 

The constructor of my LogReader class takes a string argument, and so a

simple Pig script is

 

<register jars>

all_files = LOAD 'my_file' USING com.....logging.LogReader('client') AS

(version: chararray);

dump all_files;

 

It's at this point that I get the MapReduceLauncher at 0% complete and
it

just sits there forever.  Here is what I do see, in grunt.  When I type
the

all_files = ... line, one of my LogReader classes gets instantiated with
the

correct string argument.  When I type the dump line, another gets

constructed with the correct string argument, then two LogReader classes
get

instantiated with a no-argument constructor, and then one more gets
created

with the correct string 1-argument constructor.  This is all on a Linux

setup with just one machine running in Hadoop mode to test.  I don't

understand why the two LogReaders get created with the zero-argument

constructor here -- that shouldn't need to be defined, right?  Perhaps
it's

a clue to what's going wrong.

 

By the way, in my LogReader class, other than all the constructors
getting

called, no other function calls seem to happen.  In particular, bindTo,

getNext, and determineSchema are never called.

 

Thanks in advance,

Kevin

 

p.s. In case it helps, the stacks at the point where the correct
1-argument

constructor LogReader(String) is being instantiated is

 

    at com....logging.LogReader.<init>(LogReader.java:71)

    at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
Method)

    at

sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorA
ccessorImpl.java:39)

    at

sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCons
tructorAccessorImpl.java:27)

    at java.lang.reflect.Constructor.newInstance(Constructor.java:513)

    at

org.apache.pig.impl.PigContext.instantiateFuncFromSpec(PigContext.java:4
16)

    at org.apache.pig.impl.logicalLayer.LOLoad.<init>(LOLoad.java:64)

    at

org.apache.pig.impl.logicalLayer.parser.QueryParser.LoadClause(QueryPars
er.java:1106)

    at

org.apache.pig.impl.logicalLayer.parser.QueryParser.BaseExpr(QueryParser
.java:889)

    at

org.apache.pig.impl.logicalLayer.parser.QueryParser.Expr(QueryParser.jav
a:748)

    at

org.apache.pig.impl.logicalLayer.parser.QueryParser.Parse(QueryParser.ja
va:549)

    at

org.apache.pig.impl.logicalLayer.LogicalPlanBuilder.parse(LogicalPlanBui
lder.java:60)

    at org.apache.pig.PigServer.parseQuery(PigServer.java:295)

    at org.apache.pig.PigServer.clonePlan(PigServer.java:330)

    at org.apache.pig.PigServer.compileLp(PigServer.java:666)

    at org.apache.pig.PigServer.compileLp(PigServer.java:655)

    at org.apache.pig.PigServer.store(PigServer.java:433)

    at org.apache.pig.PigServer.store(PigServer.java:421)

    at org.apache.pig.PigServer.openIterator(PigServer.java:384)

    at

org.apache.pig.tools.grunt.GruntParser.processDump(GruntParser.java:269)

    at

org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptPar
ser.java:178)

    at

org.apache.pig.tools.grunt.GruntParser.parseContOnError(GruntParser.java
:94)

    at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:58)

    at org.apache.pig.Main.main(Main.java:282)

 

while the stack at the point where the incorrect zero-argument
constructor

gets called is

 

    at com.......logging.LogReader.<init>(LogReader.java:80)

    at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
Method)

    at

sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorA
ccessorImpl.java:39)

    at

sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingCons
tructorAccessorImpl.java:27)

    at java.lang.reflect.Constructor.newInstance(Constructor.java:513)

    at java.lang.Class.newInstance0(Class.java:355)

    at java.lang.Class.newInstance(Class.java:308)

    at

org.apache.pig.impl.PigContext.instantiateFuncFromSpec(PigContext.java:4
18)

    at

org.apache.pig.impl.PigContext.instantiateFuncFromSpec(PigContext.java:4
54)

    at

org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOp
erators.POCast.instantiateFunc(POCast.java:66)

    at

org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOp
erators.POCast.setLoadFSpec(POCast.java:71)

    at

org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogToPhyTran
slationVisitor.visit(LogToPhyTranslationVisitor.java:1153)

    at org.apache.pig.impl.logicalLayer.LOCast.visit(LOCast.java:58)

    at org.apache.pig.impl.logicalLayer.LOCast.visit(LOCast.java:27)

    at

org.apache.pig.impl.plan.DependencyOrderWalkerWOSeenChk.walk(DependencyO
rderWalkerWOSeenChk.java:68)

    at

org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogToPhyTran
slationVisitor.visit(LogToPhyTranslationVisitor.java:805)

    at
org.apache.pig.impl.logicalLayer.LOForEach.visit(LOForEach.java:105)

    at
org.apache.pig.impl.logicalLayer.LOForEach.visit(LOForEach.java:40)

    at

org.apache.pig.impl.plan.DependencyOrderWalker.walk(DependencyOrderWalke
r.java:68)

    at org.apache.pig.impl.plan.PlanVisitor.visit(PlanVisitor.java:51)

    at

org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.compile(H
ExecutionEngine.java:232)

    at org.apache.pig.PigServer.compilePp(PigServer.java:731)

    at

org.apache.pig.PigServer.executeCompiledLogicalPlan(PigServer.java:644)

    at org.apache.pig.PigServer.store(PigServer.java:452)

    at org.apache.pig.PigServer.store(PigServer.java:421)

    at org.apache.pig.PigServer.openIterator(PigServer.java:384)

    at

org.apache.pig.tools.grunt.GruntParser.processDump(GruntParser.java:269)

    at

org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptPar
ser.java:178)

    at

org.apache.pig.tools.grunt.GruntParser.parseContOnError(GruntParser.java
:94)

    at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:58)

    at org.apache.pig.Main.main(Main.java:282)