You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@pig.apache.org by Alex Rovner <al...@gmail.com> on 2012/07/26 16:41:30 UTC

Trunk version does not like my macros

Gentlemen,

We have recently attempted to compile and use the latest trunk code and
have encountered a rather strange issue. Our job which is attached, has
been working fine on V11 of pig that we have compiled of trunk a while back:

Apache Pig version 0.11.0-SNAPSHOT (r1227411)
compiled Jan 04 2012, 19:34:06

When we attempted to switch to the latest trunk version yesterday, we have
encountered the following exception:


Caused by: org.apache.pig.impl.logicalLayer.FrontendException: ERROR 1000:
Error during parsing. Can not create a Path from a null string
at org.apache.pig.PigServer$Graph.parseQuery(PigServer.java:1595)
at org.apache.pig.PigServer$Graph.registerQuery(PigServer.java:1534)
at org.apache.pig.PigServer.registerQuery(PigServer.java:516)
at org.apache.pig.tools.grunt.GruntParser.processPig(GruntParser.java:987)
at
org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptParser.java:412)
at
org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:193)
at org.apache.pig.PigServer.registerScript(PigServer.java:590)
at org.apache.pig.PigServer.registerScript(PigServer.java:692)
at org.apache.pig.PigServer.registerScript(PigServer.java:665)
at com.proclivitysystems.etl.job.PIGJobRunner.run(PIGJobRunner.java:244)
... 2 more
Caused by: java.lang.IllegalArgumentException: Can not create a Path from a
null string
at org.apache.hadoop.fs.Path.checkPathArg(Path.java:78)
at org.apache.hadoop.fs.Path.<init>(Path.java:90)
at
org.apache.pig.impl.io.FileLocalizer.fetchFilesInternal(FileLocalizer.java:766)
at org.apache.pig.impl.io.FileLocalizer.fetchFile(FileLocalizer.java:733)
at
org.apache.pig.parser.QueryParserDriver.getMacroFile(QueryParserDriver.java:350)
at
org.apache.pig.parser.QueryParserDriver.makeMacroDef(QueryParserDriver.java:406)
at
org.apache.pig.parser.QueryParserDriver.expandMacro(QueryParserDriver.java:268)
at org.apache.pig.parser.QueryParserDriver.parse(QueryParserDriver.java:169)
at org.apache.pig.PigServer$Graph.parseQuery(PigServer.java:1587)
... 11 more


I have tried to step through to figure out whats going on, and it seems
like the parser is trying to load our macro named "roas" from a "null" file
thus causing this issue. As you can see in the script we are not
referencing any external macros. All macros are defined within the file.

Any help would be appreciated.

Thanks
Alex

Re: Trunk version does not like my macros

Posted by Bill Graham <bi...@gmail.com>.
I'm able to reproduce. Here's a JIRA:
https://issues.apache.org/jira/browse/PIG-2866

The issue is that QueryParserDriver assumes that the script has a file
associated with it. It doesn't handle the case where the script is read
from an InputStream. I'll update the JIRA with a workaround, which is to
write the script to a temp file and pass that to PigServer.

On Fri, Jul 27, 2012 at 7:02 AM, Alex Rovner <al...@gmail.com> wrote:

> Alan,
>
> Any idea why it might be failing?
>
> Thanks
>
> Sent from my iPhone
>
> On Jul 26, 2012, at 4:28 PM, Alan Gates <ga...@hortonworks.com> wrote:
>
> > Apache mail servers strip attachments.  Could you post your script
> somewhere or send it inline?
> >
> > Alan.
> >
> > On Jul 26, 2012, at 7:41 AM, Alex Rovner wrote:
> >
> >> Gentlemen,
> >>
> >> We have recently attempted to compile and use the latest trunk code and
> have encountered a rather strange issue. Our job which is attached, has
> been working fine on V11 of pig that we have compiled of trunk a while back:
> >>
> >> Apache Pig version 0.11.0-SNAPSHOT (r1227411)
> >> compiled Jan 04 2012, 19:34:06
> >>
> >> When we attempted to switch to the latest trunk version yesterday, we
> have encountered the following exception:
> >>
> >>
> >> Caused by: org.apache.pig.impl.logicalLayer.FrontendException: ERROR
> 1000: Error during parsing. Can not create a Path from a null string
> >>    at org.apache.pig.PigServer$Graph.parseQuery(PigServer.java:1595)
> >>    at org.apache.pig.PigServer$Graph.registerQuery(PigServer.java:1534)
> >>    at org.apache.pig.PigServer.registerQuery(PigServer.java:516)
> >>    at
> org.apache.pig.tools.grunt.GruntParser.processPig(GruntParser.java:987)
> >>    at
> org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptParser.java:412)
> >>    at
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:193)
> >>    at org.apache.pig.PigServer.registerScript(PigServer.java:590)
> >>    at org.apache.pig.PigServer.registerScript(PigServer.java:692)
> >>    at org.apache.pig.PigServer.registerScript(PigServer.java:665)
> >>    at
> com.proclivitysystems.etl.job.PIGJobRunner.run(PIGJobRunner.java:244)
> >>    ... 2 more
> >> Caused by: java.lang.IllegalArgumentException: Can not create a Path
> from a null string
> >>    at org.apache.hadoop.fs.Path.checkPathArg(Path.java:78)
> >>    at org.apache.hadoop.fs.Path.<init>(Path.java:90)
> >>    at
> org.apache.pig.impl.io.FileLocalizer.fetchFilesInternal(FileLocalizer.java:766)
> >>    at
> org.apache.pig.impl.io.FileLocalizer.fetchFile(FileLocalizer.java:733)
> >>    at
> org.apache.pig.parser.QueryParserDriver.getMacroFile(QueryParserDriver.java:350)
> >>    at
> org.apache.pig.parser.QueryParserDriver.makeMacroDef(QueryParserDriver.java:406)
> >>    at
> org.apache.pig.parser.QueryParserDriver.expandMacro(QueryParserDriver.java:268)
> >>    at
> org.apache.pig.parser.QueryParserDriver.parse(QueryParserDriver.java:169)
> >>    at org.apache.pig.PigServer$Graph.parseQuery(PigServer.java:1587)
> >>    ... 11 more
> >>
> >>
> >> I have tried to step through to figure out whats going on, and it seems
> like the parser is trying to load our macro named "roas" from a "null" file
> thus causing this issue. As you can see in the script we are not
> referencing any external macros. All macros are defined within the file.
> >>
> >> Any help would be appreciated.
> >>
> >> Thanks
> >> Alex
> >
>



-- 
*Note that I'm no longer using my Yahoo! email address. Please email me at
billgraham@gmail.com going forward.*

Re: Trunk version does not like my macros

Posted by Alex Rovner <al...@gmail.com>.
Alan,

Any idea why it might be failing?

Thanks 

Sent from my iPhone

On Jul 26, 2012, at 4:28 PM, Alan Gates <ga...@hortonworks.com> wrote:

> Apache mail servers strip attachments.  Could you post your script somewhere or send it inline?
> 
> Alan.
> 
> On Jul 26, 2012, at 7:41 AM, Alex Rovner wrote:
> 
>> Gentlemen,
>> 
>> We have recently attempted to compile and use the latest trunk code and have encountered a rather strange issue. Our job which is attached, has been working fine on V11 of pig that we have compiled of trunk a while back:
>> 
>> Apache Pig version 0.11.0-SNAPSHOT (r1227411) 
>> compiled Jan 04 2012, 19:34:06
>> 
>> When we attempted to switch to the latest trunk version yesterday, we have encountered the following exception:
>> 
>> 
>> Caused by: org.apache.pig.impl.logicalLayer.FrontendException: ERROR 1000: Error during parsing. Can not create a Path from a null string
>>    at org.apache.pig.PigServer$Graph.parseQuery(PigServer.java:1595)
>>    at org.apache.pig.PigServer$Graph.registerQuery(PigServer.java:1534)
>>    at org.apache.pig.PigServer.registerQuery(PigServer.java:516)
>>    at org.apache.pig.tools.grunt.GruntParser.processPig(GruntParser.java:987)
>>    at org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptParser.java:412)
>>    at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:193)
>>    at org.apache.pig.PigServer.registerScript(PigServer.java:590)
>>    at org.apache.pig.PigServer.registerScript(PigServer.java:692)
>>    at org.apache.pig.PigServer.registerScript(PigServer.java:665)
>>    at com.proclivitysystems.etl.job.PIGJobRunner.run(PIGJobRunner.java:244)
>>    ... 2 more
>> Caused by: java.lang.IllegalArgumentException: Can not create a Path from a null string
>>    at org.apache.hadoop.fs.Path.checkPathArg(Path.java:78)
>>    at org.apache.hadoop.fs.Path.<init>(Path.java:90)
>>    at org.apache.pig.impl.io.FileLocalizer.fetchFilesInternal(FileLocalizer.java:766)
>>    at org.apache.pig.impl.io.FileLocalizer.fetchFile(FileLocalizer.java:733)
>>    at org.apache.pig.parser.QueryParserDriver.getMacroFile(QueryParserDriver.java:350)
>>    at org.apache.pig.parser.QueryParserDriver.makeMacroDef(QueryParserDriver.java:406)
>>    at org.apache.pig.parser.QueryParserDriver.expandMacro(QueryParserDriver.java:268)
>>    at org.apache.pig.parser.QueryParserDriver.parse(QueryParserDriver.java:169)
>>    at org.apache.pig.PigServer$Graph.parseQuery(PigServer.java:1587)
>>    ... 11 more
>> 
>> 
>> I have tried to step through to figure out whats going on, and it seems like the parser is trying to load our macro named "roas" from a "null" file thus causing this issue. As you can see in the script we are not referencing any external macros. All macros are defined within the file.
>> 
>> Any help would be appreciated.
>> 
>> Thanks
>> Alex
> 

Re: Trunk version does not like my macros

Posted by Alex Rovner <al...@gmail.com>.
Script below:

DEFINE ToMillis com.proclivitysystems.etl.pig.udf.eval.ToMillis();
DEFINE First com.proclivitysystems.etl.pig.udf.eval.NthTuple('0');

DEFINE ClickAttributionFilter
com.proclivitysystems.reporting.DurationFilter('click');
DEFINE ViewAttributionFilter
com.proclivitysystems.reporting.DurationFilter('view');

-- format date string to 'yyyymmdd'
DEFINE toCanonicalDateString
InvokeForString('com.proclivitysystems.etl.pig.udf.eval.Utils.toCanonicalDateString',
'String');

DEFINE base64Decode
InvokeForString('com.proclivitysystems.etl.pig.udf.eval.Utils.base64DecodeString',
'String');
DEFINE extractQueryValue
InvokeForString('com.proclivitysystems.utils.URIUtils.extractQueryValue',
'String String');

-- identifies order events
%default ORDER_TID 2

-- load orders from daily events
daily_events = LOAD '$DAILY_EVENTS' USING $HIVE_LOADER;
describe daily_events;
-- retrieve orders for the given date range
orders = FILTER daily_events BY client_tid == $CLIENT_TID and file_date >=
$START_DATE and file_date <= $END_DATE and event_tid == $ORDER_TID;

--
-- note that the order amount must be multiplied by quantity
--
orders = FOREACH orders GENERATE activity_timestamp AS date_time,
ToMillis(activity_timestamp) AS ms, natural_consumer_key, consumer_tid AS
ctid, order_id, order_sales * quantity AS order_sales, channel_type;

--
-- a given order_id may be associated with multiple items; we're interested
in the total price, hence the following
-- aggregation
--
by_id = GROUP orders BY order_id;

orders = FOREACH by_id {
   order_info = DISTINCT orders.(date_time, ms, natural_consumer_key, ctid,
order_id, channel_type);
   GENERATE flatten(order_info) AS (date_time, ms, natural_consumer_key,
ctid, order_id, channel_type), SUM(orders.order_sales) AS order_sales;
}

-- only for debugging
--STORE orders INTO '$OUTPUT/_orders';

view_logs = LOAD '$IMPRESSIONS_TABLE' USING $HIVE_LOADER;
view_logs = FILTER view_logs BY client_tid==$CLIENT_TID AND
date_partition>=$OBSERVATION_START AND date_partition<=$END_DATE;

-- TBD: need to add pub_payout but it's currently missing from click log
(want to keep the schemas uniform)
view_logs = FOREACH view_logs GENERATE date_time, ToMillis(date_time) AS
ms, uuid, bid_id, campaign_id, adgroup_id, advertiser_id, advertiser_payout;

click_logs = LOAD '$CLICKS_TABLE' USING $HIVE_LOADER;
click_logs = FILTER click_logs BY client_tid==$CLIENT_TID AND
date_partition>=$OBSERVATION_START AND date_partition<=$END_DATE;
describe click_logs;

click_logs = FOREACH click_logs GENERATE date_time, ToMillis(date_time) AS
ms, uuid, bid_id, campaign_id, adgroup_id, advertiser_id, advertiser_payout;

by_uuid = GROUP view_logs BY uuid;
-- used to improve accuracy of conversions since internally iponweb uses
bid_id
uuid_to_bid_id = FOREACH by_uuid GENERATE group AS uuid,
flatten(view_logs.bid_id) AS bid_id;

DESCRIBE uuid_to_bid_id;

--
-- psguid -> ctid map  (used for offline order attribution)
--
psguid_map = LOAD '$PSGUID_MAP' USING PigStorage(',') AS (psguid:chararray,
ctid: long);

--
-- computes temporal roas corresponding to click/view conversions
--
DEFINE roas(orders, psguidmap, clicks_or_views, attribution_filter,
output_path, conversion_type, client_tid, uuid_to_bid_id) RETURNS
conversions {
-- de-multiplex orders
SPLIT $orders INTO offline_orders IF UPPER(channel_type) == 'OFFLINE',
online_orders IF UPPER(channel_type) == 'ONLINE';

-- For online attribution, we join on natural_consumer_key instead of ctid;
the reason is "business logic".  This essentially is tantamount to
under-reporting.
-- Here is why.  Consider a user cookied with multiple psguids, e.g., x, y,
z.  Suppose there is a single click event which corresponds to x.
        -- Thus, orders corresponding to y and z are not attributable.
--
online_orders = FOREACH online_orders GENERATE date_time AS date_time, ms
AS ms, natural_consumer_key AS natural_consumer_key, ctid AS ctid, order_id
AS order_id,
                channel_type AS channel_type, order_sales AS order_sales;

-- tmp = JOIN online_orders BY natural_consumer_key, clicks_or_views BY
uuid;

        -- iponweb uses bid_id in case uuid gets reset between imp/click
and order;
-- we thus use uuid_to_bid_id in order to join on bid_id
-- =======================begin
workaround==========================================
tmp = JOIN online_orders BY natural_consumer_key, $uuid_to_bid_id BY uuid;

        online_orders = FOREACH tmp GENERATE online_orders::date_time AS
date_time, online_orders::ms AS ms, online_orders::natural_consumer_key AS
natural_consumer_key, bid_id AS bid_id, online_orders::ctid AS ctid,
online_orders::order_id AS order_id, online_orders::channel_type AS
channel_type, online_orders::order_sales AS order_sales;

tmp = JOIN online_orders BY bid_id, $clicks_or_views BY bid_id;
        -- =======================end of
workaround=========================================

-- potential online conversions
online = FOREACH tmp GENERATE online_orders::date_time AS date_time,
'$conversion_type' AS conversion_type, channel_type,
                  campaign_id, adgroup_id, order_id, order_sales,
                                      (online_orders::ms -
$clicks_or_views::ms) AS delta,
                                      $clicks_or_views::uuid AS uuid,
online_orders::ctid AS ctid, advertiser_id, '$client_tid' AS client_tid:
int,
      advertiser_payout AS advertiser_payout;
        --
        -- For offline attribution, we need to go through psguidmap.  That
is, map consumer tid into psguid.

--
-- resolve consumer tids using psguid map
--
clicks_or_views = JOIN $clicks_or_views BY uuid, $psguidmap BY psguid;
 clicks_or_views = FOREACH clicks_or_views GENERATE ms AS ms,
psguid_map::ctid AS ctid, psguid_map::psguid AS uuid, campaign_id,
adgroup_id, advertiser_id, advertiser_payout;
 -- this step is stupid but it's needed to work around pig's
idiosyncracies; if the fields are not renamed, then we can't refer to them
below...
offline_orders = FOREACH offline_orders GENERATE date_time AS date_time, ms
AS ms, natural_consumer_key AS natural_consumer_key, ctid AS ctid, order_id
AS order_id,
          channel_type AS channel_type, order_sales AS order_sales;

tmp = JOIN offline_orders BY ctid, clicks_or_views BY ctid;
-- potential offline conversions
offline = FOREACH tmp GENERATE offline_orders::date_time AS date_time,
'$conversion_type' AS conversion_type, channel_type,
                   campaign_id, adgroup_id, order_id, order_sales,
       (offline_orders::ms - clicks_or_views::ms) AS delta,
                                       clicks_or_views::uuid AS uuid,
clicks_or_views::ctid AS ctid, advertiser_id, '$client_tid' AS client_tid:
int,
       advertiser_payout AS advertiser_payout;
-- all potential conversions
online_or_offline = UNION offline, online;

--
-- filter by click/view attribution window  (yields all attributable orders)
--
conversions = FILTER online_or_offline BY $attribution_filter(delta,
campaign_id);
--
-- if a conversion is attributable to > 1 clicks/views within the same
campaign or across different campaigns,
-- attribute the conversion to the click/view with the smallest delta
(conversion time - click/view time).
--
by_id = GROUP conversions BY order_id;

$conversions = FOREACH by_id {
   ordered = ORDER conversions BY delta ASC;
   GENERATE flatten(First(ordered)) AS (conversions::date_time,
conversions::conversion_type, conversions::channel_type,
conversions::campaign_id, conversions::adgroup_id,
               conversions::order_id, conversions::order_sales,
conversions::delta,
conversions::uuid, conversions::ctid, conversions::advertiser_id,
conversions::client_tid, conversions::advertiser_payout);
}
}

DEFINE compute_stats(clicks_or_views, log_type) RETURNS stats {

clicks_or_views = FOREACH $clicks_or_views GENERATE
toCanonicalDateString(date_time) AS date_time, uuid;

by_id = GROUP clicks_or_views BY date_time;

$stats = FOREACH by_id GENERATE group AS date_time, '$log_type',
COUNT(clicks_or_views);
}


-- ============================Stats========================================

view_stats = compute_stats(view_logs, 'view');
click_stats = compute_stats(click_logs, 'click');

stats = UNION view_stats, click_stats;

STORE stats INTO '$OUTPUT/_stats' USING
com.proclivitysystems.etl.pig.udf.storage.PathPartitioningStorage('-partitions=date_time
-format=text');

-- ============================End of
Stats====================================

-- compute click conversions
click_conversions = roas(orders, psguid_map, click_logs,
ClickAttributionFilter, '$OUTPUT', 'pc', '$CLIENT_TID', uuid_to_bid_id);

--
-- now compute conversions attributed to views, i.e., impressions
--
--

--
-- subtract click_conversions
--
orders = JOIN orders BY order_id LEFT OUTER, click_conversions BY order_id;

orders = FILTER orders BY conversions::order_id is null;

orders = FOREACH orders GENERATE orders::date_time, orders::ms,
orders::natural_consumer_key, orders::ctid, orders::order_id,
orders::channel_type, orders::order_sales;

-- compute view conversions
view_conversions = roas(orders, psguid_map, view_logs,
ViewAttributionFilter, '$OUTPUT', 'pv', '$CLIENT_TID', uuid_to_bid_id);

ps_conversions = UNION click_conversions, view_conversions;

ps_conversions = FOREACH ps_conversions GENERATE conversions::date_time AS
timestamp, toCanonicalDateString(conversions::date_time) AS date_time,
conversions::conversion_type AS conversion_type, conversions::channel_type
AS channel_type, conversions::campaign_id AS campaign_id,
conversions::adgroup_id AS adgroup_id,
               conversions::order_id AS order_id, conversions::order_sales
AS order_sales,
conversions::delta AS delta,
conversions::uuid AS uuid, conversions::ctid AS ctid,
conversions::advertiser_id AS advertiser_id, conversions::client_tid AS
client_tid,
'ps' AS conversion_provenance;


-- include  iponweb's conversions for verification purposes
--
-- conversion logs are only used for verification
--
iponweb_conversions = LOAD '$CONVERSIONS_TABLE' USING $HIVE_LOADER;
iponweb_conversions = FILTER iponweb_conversions BY client_tid==$CLIENT_TID
AND date_partition>=$START_DATE AND date_partition<=$END_DATE;
describe iponweb_conversions;

-- ===================== join segment_id with client_tid
======================
clients = LOAD '$CLIENTS_TABLE' USING $HIVE_LOADER;
clients = FOREACH clients GENERATE client_tid, wec_id;

tmp = JOIN iponweb_conversions BY segment_id, clients BY wec_id USING
'replicated';
-- ===================== end of clients
join===================================

iponweb_conversions = FOREACH tmp GENERATE date_time AS timestamp,
toCanonicalDateString(date_time) AS date_time, conversion_type, 'ONLINE' AS
channel_type, campaign_id, adgroup_id,
         extractQueryValue(base64Decode(request_uri), 'order_id') AS
order_id, total_order_price AS order_sales: double, '-1' AS delta: long,
uuid, 'NA' AS ctid: long, advertiser_id, clients::client_tid AS client_tid,
'iponweb' AS conversion_provenance;

--
-- filter out conversions by other clients
--
iponweb_conversions = FILTER iponweb_conversions BY client_tid ==
$CLIENT_TID;

all_conversions = UNION ps_conversions, iponweb_conversions;

DESCRIBE all_conversions;

STORE all_conversions INTO '$OUTPUT/conversions' USING
com.proclivitysystems.etl.pig.udf.storage.PathPartitioningStorage('-partitions=date_time
-format=text');







On Thu, Jul 26, 2012 at 4:28 PM, Alan Gates <ga...@hortonworks.com> wrote:

> Apache mail servers strip attachments.  Could you post your script
> somewhere or send it inline?
>
> Alan.
>
> On Jul 26, 2012, at 7:41 AM, Alex Rovner wrote:
>
> > Gentlemen,
> >
> > We have recently attempted to compile and use the latest trunk code and
> have encountered a rather strange issue. Our job which is attached, has
> been working fine on V11 of pig that we have compiled of trunk a while back:
> >
> > Apache Pig version 0.11.0-SNAPSHOT (r1227411)
> > compiled Jan 04 2012, 19:34:06
> >
> > When we attempted to switch to the latest trunk version yesterday, we
> have encountered the following exception:
> >
> >
> > Caused by: org.apache.pig.impl.logicalLayer.FrontendException: ERROR
> 1000: Error during parsing. Can not create a Path from a null string
> >       at org.apache.pig.PigServer$Graph.parseQuery(PigServer.java:1595)
> >       at
> org.apache.pig.PigServer$Graph.registerQuery(PigServer.java:1534)
> >       at org.apache.pig.PigServer.registerQuery(PigServer.java:516)
> >       at
> org.apache.pig.tools.grunt.GruntParser.processPig(GruntParser.java:987)
> >       at
> org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptParser.java:412)
> >       at
> org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:193)
> >       at org.apache.pig.PigServer.registerScript(PigServer.java:590)
> >       at org.apache.pig.PigServer.registerScript(PigServer.java:692)
> >       at org.apache.pig.PigServer.registerScript(PigServer.java:665)
> >       at
> com.proclivitysystems.etl.job.PIGJobRunner.run(PIGJobRunner.java:244)
> >       ... 2 more
> > Caused by: java.lang.IllegalArgumentException: Can not create a Path
> from a null string
> >       at org.apache.hadoop.fs.Path.checkPathArg(Path.java:78)
> >       at org.apache.hadoop.fs.Path.<init>(Path.java:90)
> >       at
> org.apache.pig.impl.io.FileLocalizer.fetchFilesInternal(FileLocalizer.java:766)
> >       at
> org.apache.pig.impl.io.FileLocalizer.fetchFile(FileLocalizer.java:733)
> >       at
> org.apache.pig.parser.QueryParserDriver.getMacroFile(QueryParserDriver.java:350)
> >       at
> org.apache.pig.parser.QueryParserDriver.makeMacroDef(QueryParserDriver.java:406)
> >       at
> org.apache.pig.parser.QueryParserDriver.expandMacro(QueryParserDriver.java:268)
> >       at
> org.apache.pig.parser.QueryParserDriver.parse(QueryParserDriver.java:169)
> >       at org.apache.pig.PigServer$Graph.parseQuery(PigServer.java:1587)
> >       ... 11 more
> >
> >
> > I have tried to step through to figure out whats going on, and it seems
> like the parser is trying to load our macro named "roas" from a "null" file
> thus causing this issue. As you can see in the script we are not
> referencing any external macros. All macros are defined within the file.
> >
> > Any help would be appreciated.
> >
> > Thanks
> > Alex
>
>

Re: Trunk version does not like my macros

Posted by Alan Gates <ga...@hortonworks.com>.
Apache mail servers strip attachments.  Could you post your script somewhere or send it inline?

Alan.

On Jul 26, 2012, at 7:41 AM, Alex Rovner wrote:

> Gentlemen,
> 
> We have recently attempted to compile and use the latest trunk code and have encountered a rather strange issue. Our job which is attached, has been working fine on V11 of pig that we have compiled of trunk a while back:
> 
> Apache Pig version 0.11.0-SNAPSHOT (r1227411) 
> compiled Jan 04 2012, 19:34:06
> 
> When we attempted to switch to the latest trunk version yesterday, we have encountered the following exception:
> 
> 
> Caused by: org.apache.pig.impl.logicalLayer.FrontendException: ERROR 1000: Error during parsing. Can not create a Path from a null string
> 	at org.apache.pig.PigServer$Graph.parseQuery(PigServer.java:1595)
> 	at org.apache.pig.PigServer$Graph.registerQuery(PigServer.java:1534)
> 	at org.apache.pig.PigServer.registerQuery(PigServer.java:516)
> 	at org.apache.pig.tools.grunt.GruntParser.processPig(GruntParser.java:987)
> 	at org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptParser.java:412)
> 	at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:193)
> 	at org.apache.pig.PigServer.registerScript(PigServer.java:590)
> 	at org.apache.pig.PigServer.registerScript(PigServer.java:692)
> 	at org.apache.pig.PigServer.registerScript(PigServer.java:665)
> 	at com.proclivitysystems.etl.job.PIGJobRunner.run(PIGJobRunner.java:244)
> 	... 2 more
> Caused by: java.lang.IllegalArgumentException: Can not create a Path from a null string
> 	at org.apache.hadoop.fs.Path.checkPathArg(Path.java:78)
> 	at org.apache.hadoop.fs.Path.<init>(Path.java:90)
> 	at org.apache.pig.impl.io.FileLocalizer.fetchFilesInternal(FileLocalizer.java:766)
> 	at org.apache.pig.impl.io.FileLocalizer.fetchFile(FileLocalizer.java:733)
> 	at org.apache.pig.parser.QueryParserDriver.getMacroFile(QueryParserDriver.java:350)
> 	at org.apache.pig.parser.QueryParserDriver.makeMacroDef(QueryParserDriver.java:406)
> 	at org.apache.pig.parser.QueryParserDriver.expandMacro(QueryParserDriver.java:268)
> 	at org.apache.pig.parser.QueryParserDriver.parse(QueryParserDriver.java:169)
> 	at org.apache.pig.PigServer$Graph.parseQuery(PigServer.java:1587)
> 	... 11 more
> 
> 
> I have tried to step through to figure out whats going on, and it seems like the parser is trying to load our macro named "roas" from a "null" file thus causing this issue. As you can see in the script we are not referencing any external macros. All macros are defined within the file.
> 
> Any help would be appreciated.
> 
> Thanks
> Alex