leton of)
your script?
--jacob
@thedatachef
On Jul 20, 2014, at 6:43 AM, Rodrigo Ferreira wrote:
> Hi everyone,
>
> I found the answer here:
> http://pig.apache.org/docs/r0.9.1/perf.html#Implicit-Dependencies
>
> It seems that when you have implicit dependencies you have to us
Rodrigo,
Write your own StoreFunc and use it instead of the udf. PigStorage can already
write to s3 so it should be straightforward to simply subclass that.
--jacob
@thedatachef
On Jul 16, 2014, at 2:20 AM, Rodrigo Ferreira wrote:
> Hey Jacob, thanks for your reply.
>
it seems that Pig optimizer simply discards this UDF.
> What I did was to make this UDF return a boolean value and I store it on S3
> (a lightweight file). This way it gets executed. Any thoughts on this?
Can you explain further? It's not clear what you're trying to do/what isn't
working.
--jacob
@thedatachef
(field5max.tuple_schema.$0) as field5max;
};
Generates: (1,2,3,(a),(x)) for your input. You can do further projections to
rearrange how you like downstream.
Best of luck.
--jacob
@thedatachef
On Mar 20, 2014, at 5:59 AM, Adamantios Corais
wrote:
> Hi,
>
> I have the followi
g and clearly shows what logical operators map
to map-reduce jobs. It could be a starting point for managing complexity at
least.
--jacob
@thedatachef
On Mar 5, 2014, at 7:33 AM, Christopher Petrino wrote:
> Hi all, what is everyone's approach for managing a Pig scripts that has
>
token),
COUNT(flattened) as freq;
Of course, this will spawn another map-reduce job. However, since COUNT is
algebraic, pig will make use of combiners drastically reducing the amount of
data sent to the reducers.
--jacob
@thedatachef
On Nov 19, 2013, at 5:45 PM, jamal sasha wrote:
> Hi,
>
as reporting_dt;
Note that I haven't actually ran this yet but it's pretty straightforward pig.
--jacob
@thedatachef
On Sep 10, 2013, at 12:07 AM, "Ambastha, Abhishek"
wrote:
> Hi,
>
> I want to write a pig query for the below SQL:
>
>CREAT
one_rec = limit ordered 1;
generate
flatten(one_rec) as (f11, f2, f13, f14);
};
--jacob
@thedatachef
On Sat, 2013-08-24 at 18:03 +, Ambastha, Abhishek wrote:
> Hi,
>
> How can I sort and dedupe on multiple columns ?
>
> I have a 5 GB fil
ank.storage.MultiStorage('/data', '0');
where '/data' is the directory on hdfs where the files will go and 0 is the
numerical index of the field to use as the file name, called filename in this
case. I would warn you though, this will open many streams at once and co
erate on tuples during execution
(map-reduce or whatever emulates map-reduce in local mode) and _after_ the
input location has been resolved.
--jacob
@thedatachef
On Jul 30, 2013, at 1:30 AM, Xuri Nagarin wrote:
> Thanks Jacob.
>
> I threw in a little bash shell hack to make it s
t; to accomplish this. Perhaps
another Pig user can illuminate the situation better?
I'll see about publishing a simple version of FilterBag if it seems the pig
community would use it.
--jacob
@thedatachef
On Jul 28, 2013, at 8:34 PM, Xuri Nagarin wrote:
> Hi,
&g
};
dump all_matches;
(q1,2,{(d1),(d2)})
(q2,0,{})
(q3,0,{})
The empty curly braces indicate bags that contain no tuples.
--jacob
@thedatachef
On Jun 21, 2013, at 6:14 AM, Siddhi Borkar wrote:
> Thanks a lot the solution worked fine. Is it possible also to display the
>
h (group counts by q) generate group as q,
SUM(counts.num_matches) as total_matches;
dump all_matches;
(q1,2)
(q2,0)
(q3,0)
--jacob
@thedatachef
On 06/20/2013 02:06 PM, Barclay Dunn wrote:
Jacob,
If I run that code with an added row in file2.txt, e.g.,
$ cat file2.txt
q1 d1
q1 d2
q3 d3
generate
group as q,
num_matches as num_matches;
};
dump counts;
(q1,2)
(q2,0)
(q3,0)
--jacob
@thedatachef
On Jun 20, 2013, at 4:00 AM, Siddhi Borkar wrote:
> Hi,
>
> I have a problem statement where in I have to compare two files and
hosts_exploded;
spread = foreach (join ranked_users by $0, ranked_hosts by $0) generate
host_id, user_id;
dump spread;
Hope that helps!
--jacob
@thedatachef
On Sun, 2013-03-31 at 12:06 -0400, John Meek wrote:
> hey all,
>
> Can anyone let me know how I can accomplish
Michael,
Why not just:
D = foreach (join C by datapoint2, B by datapoint1) generate
B::datapoint1, B::datapoint2;
Does that get you what you need?
--jacob
@thedatachef
On Mon, 2012-03-19 at 15:55 -0400, Michael Moore wrote:
> Really sorry folks. Hotmail stinks. In case this does
used about there though is the ability to
register jars. Is there a simple example of using PigRunner that
demonstrates this?
Thanks!
--jacob
@thedatachef
ser_type=='rich';
B = COGROUP happy by user_id, rich by user_id;
rich_and_not_happy = foreach (filter B by IsEmpty(happy) and NOT
IsEmpty(rich)) generate group as user_id;
DUMP rich_and_not_happy;
--jacob
@thedatachef
On Tue, 2012-02-28 at 16:49 +0100, Marco Cadetg wrote:
> Hi there,
&g
#x27; as (loc:chararray, stuff:int);
c = cogroup a by loc, b by loc;
d = foreach (filter c by IsEmpty(b)) generate FLATTEN(a);
which will result in d containing only the records from a where the
'loc' field doesn't match with the 'loc' field in b.
--jacob
@thedatachef
,cache_hit', columns) AS (url:chararray,
cache_hit:chararray);
DUMP what_cassandrastorage_should_really_produce;
(key1, http://www.google.com, hit)
(key2, http://www.google.com, hit)
Does that work for your use case?
--jacob
@thedatachef
On Fri, 2011-11-04 at 08:51 -0400, AD wrote:
>
You'll have to run a FOREACH...GENERATE over the data first and generate
a single key to look like the filename you want. Then you can use
MultiStorage() from the piggybank. See:
org.apache.pig.piggybank.storage.MultiStorage
in the pig api docs.
--jacob
@thedatachef
On Mon, 2011-10-10 at
urious, in the same java doc I see no reference to SQLLOADER, where
did you find that?
--jacob
@thedatachef
On Tue, 2011-07-26 at 10:00 -0300, Juan Martin Pampliega wrote:
> From what I read, data from a RDBMS is read by the command:
> LOAD 'sql://mytable' USING SQLLOADER();
>
> fred3 120
> elsie 4 45
>
> The output is as:
> ({(sally)},{(82)})
> ({(fred)},{(120)})
> ({},{})
>
> As you see, I got an empty tuple? why?
There are three tuples, one for each group (1, 3, and 4). The filter
condition left the bags from group 4 empty sin
name, score));
};
DUMP C;
on the following data:
$: cat foo.tsv
henrietta 1 25
sally 1 82
fred3 120
elsie 4 45
yields:
(sally,82)
(fred,120)
Does that work for you?
--jacob
@thedatachef
On Tue, 2011-07-19 at 15:00 +0200, 勇胡 wrote:
> A = L
he new key from file 1 and col2 from file
2.
This works even if your ranges are smaller, just round to whatever makes
sense. Eg, nearest 10. What this does not work for is if your ranges are
variable. Are your ranges variable? :)
--jacob
@thedatachef
On Fri, 2011-07-15 at 01:23 -0700, Laksh
Dmitriy,
I see your point. It would definitely be nice to have a builtin for
returning a bag though. I'd actually be happy if
TOBAG(FLATTEN(STRSPLIT(X,','))) worked.
--jacob
@thedatachef
On Sat, 2011-05-07 at 18:41 -0700, Dmitriy Ryaboy wrote:
> FWIW -- the reason STRSPL
es this fine by passing in various things like {(),()} and so on.
What makes it yet more strange is that deleting a single _arbitrary_
entry from the line above gets rid of the error.
I really have no idea what's going on.
Any ideas?
--jacob
@thedatachef
On Fri, 2011-05-06 at 16:06 -0600, Christian wrote:
> Thank you for taking the time to explain this to me Jacob!
>
> Am I stuck with hard-coding for my other question?
>
> Instead of:
> 2011-05-01DIRECTIVE132423DIRECTIVE23433DIRECTIVE3
> 1983
>
e2
> > 201101,directive2
> > 201101,directive2
> > 201101,directive3
> > 201102,directive2
> > 201102,directive4
> > 201103,directive1
> >
> > This is how my data looks (row and column wise)
>
> >
> > then, a simple:
> >
> > Y = GROUP X BY (date,directive);
> > Z = FOREACH Y GENERATE FLATTEN(group) AS (date,directive), COUNT(X) AS
> > num_occurrences;
> >
> > would result in:
> >
> > 201101,directive1,2
> > 201101,directive2,3
> > 201101,directive3,1
> > 201102,directive2,1
> > 201102,directive4,1
> > 201103,directive1,1
> >
> > At least, that's what it _seems_ like you're asking for.
> >
> > I've gotten that far. I'm actually asking for the being able to put those
> into columns and not rows.
>
> >
> > --jacob
> > @thedatachef
> >
> > Thanks Jacob!
>
> -Christian
>
> >
> > > Thanks very much for you help,
> > > Christian
> >
> >
> >
FOREACH Y GENERATE FLATTEN(group) AS (date,directive), COUNT(X) AS
num_occurrences;
would result in:
201101,directive1,2
201101,directive2,3
201101,directive3,1
201102,directive2,1
201102,directive4,1
201103,directive1,1
At least, that's what it _seems_ like you're asking for.
--jacob
@thedatachef
> Thanks very much for you help,
> Christian
_distinct_users
;
};
And I just tested it in local mode with Pig 0.8, works great.
--jacob
@thedatachef
On Fri, 2011-05-06 at 11:30 -0700, Kim Vogt wrote:
> I think you're missing a SUM and/or COUNT and that's the part I'm stuck on.
>
> -Kim
>
> On Fri, May
hould work. Haven't tested it though.
--jacob
@thedatachef
On Fri, 2011-05-06 at 11:08 -0700, Kim Vogt wrote:
> Hi,
>
> I'm stuck on a query for counting distinct users. Say I have data that looks
> like this:
>
> book, user1
> book, user2
> book, user1
> mov
This works, though I'm not sure what the difference is really:
A = LOAD '1line.txt' AS (a:chararray);
B = FOREACH A {
extract = REGEX_EXTRACT(a,'2',1);
boolean = ((extract IS NULL) ? 1 : 0);
GENERATE boolean;
};
DUMP B;
--jacob
@thedatachef
On
What you've indicated does require access to the whole relation at once
or at least a way of incrementing a counter and assigning its value to
each tuple. This kind of shared/synchronized state isn't possible with
Pig at the moment as far as I know.
--jacob
@thedatachef
On Tue, 2011-0
you might look at the MAX and MIN operations. They do require a GROUP
ALL but are algebraic so it's not actually going to bring all the data
to one machine as it otherwise would.
--jacob
@thedatachef
On Tue, 2011-04-26 at 19:07 -0700, Arun A K wrote:
> Hi
>
> I have the following
JIRA filed, see:
https://issues.apache.org/jira/browse/PIG-2014
--jacob
@thedatachef
On Mon, 2011-04-25 at 09:02 -0700, Alan Gates wrote:
> You are not insane. Pig rewrites sample into filter, and then pushes
> that filter in front of the group. It shouldn't push that filter
Nested cross is not supported at the moment, though it is in the works,
see: https://issues.apache.org/jira/browse/PIG-1916
Meanwhile, you could write a UDF to cross the items manually.
--jacob
@thedatachef
On Tue, 2011-04-26 at 15:04 +0300, Shai Harel wrote:
> I have a bag of items (a res
2;
DUMP sampled;
Running this results in 1,513 reduce output records. So, am I insane or
shouldn't the reduce output records be much much closer to 22 or 23
records (eg. 0.0012*18863)?
--jacob
@thedatachef
https://github.com/jeromatron/pygmalion
Best of luck!
--jacob
@thedatachef
On Sun, 2011-04-24 at 18:31 +0200, pob wrote:
> Maybe I forget one more thing, rows are taken from Cassandra.
>
> rows = LOAD 'cassandra://emailArchive/messagesMetaData' USING
> CassandraStorage() AS (key,
Strange, that looks right to me. What happens if you try the 'AS'
statement anyhow?
--jacob
@thedatachef
On Sun, 2011-04-24 at 18:22 +0200, pob wrote:
> Hello,
>
> pom = foreach rows generate myUDF.toTuple($1); -- reading data
> describe pom
> pom: {y: {t: (domain:
the UDF or
just use the 'AS' statement. For example, say you have a UDF that sums
two numbers:
data = LOAD 'foobar' AS (int:a, int:b);
summed = FOREACH data GENERATE MyFancySummingUDF(a,b) AS (sum:int);
DUMP summed;
--jacob
@thedatachef
On Sun, 2011-04-24 at 18:02 +0200, po
I wrote a post on a similar problem with pig. Finding similarity between
comic book characters ;)
http://thedatachef.blogspot.com/2011/02/brute-force-graph-crunching-with-pig.html
--jacob
@thedatachef
On Sun, 2011-04-03 at 20:49 +, Diallo Mamadou Bobo wrote:
> Hi There.
> We need as p
We're still using a fork unfortunately. Jeremy is referencing the on in
trunk as far as I know though. Here we're waiting for when we switch
from our weird version of hbase (0.89somethingsomething) to 0.90 to make
the switch.
--jacob
On Thu, 2011-03-24 at 15:10 -0700, Dmitriy Ry
e to go in and explicitly check the types of each column and
cast manually.
--jacob
On Thu, 2011-03-24 at 13:11 -0500, Jeremy Hanna wrote:
> I see that there are a few LoadCaster implementations in pig 0.8. There's
> the Utf8StorageConverter, the HBaseBinaryConverter, and a couple o
Then you can just flatten the bag like
normal.
--jacob
@thedatachef
On Sun, 2011-02-27 at 22:16 -0800, Bill Graham wrote:
> Hi,
>
> I'd like to be able to flatten a map, so each K->V is flattened into
> it's own row. Basically something like this:
>
> cat map_
Had this. It's because Pig packages the Hadoop 0.20 classes into its
jar. Tail the namenode logs when you run the Pig shell and you'll see an
error about version. Try the instructions here:
http://thedatachef.blogspot.com/2011/01/apache-pig-08-with-cloudera-cdh3.html
--jacob
@thedatach
grunt> DUMP data;
(98384,559)
(98385,587)
(98386,573)
(98387,587)
(98388,589)
(98389,584)
(98390,572)
(98391,567)
Looks great. I'm going to blame it on your version? I'm using pig-0.8
and hadoop 0.20.2.
--jacob
@thedatachef
On Tue, 2011-02-22 at 08:21 -0500, Eric Lubow wrote:
> I apo
Your mac needs to have the hadoop configuration (eg. hdfs-site.xml,
mapred-site.xml, core-site.xml, depending on the version of hadoop)
files available somewhere in pig's classpath. It may do to simply copy
them directly from one of the remote machines.
--jacob
@thedatachef
On Tue, 2011-02-
d.
Lastly, there's an additional class (for writing to hbase only) called
'DynamicFamilyStorage' that gets the column family and column name from
the records themselves that you might find useful.
--jacob
@thedatachef
On Mon, 2011-02-14 at 15:03 -0700, Matt Davies wrote:
> Thank
Thanks for the mention. Off the top of my head I got these sorts of
errors when trying to store either null records or records with null
fields. What happens if you FILTER out any null values you might have.
Does the problem persist?
--jacob
@thedatachef
On Mon, 2011-02-14 at 14:57 -0700, Matt
To be clearer about the ambiguous +1, we're using Pig 0.8, hbase 0.89
something something, and a modified version of HBaseStorage. Anything
to clean that up gets a +1 from me.
--jacob
@thedatachef
Sent from my iPhone
On Feb 14, 2011, at 9:10 AM, Dan Harvey wrote:
We're still u
+1
--jacob
@thedatachef
Sent from my iPhone
On Feb 13, 2011, at 7:41 PM, Dmitriy Ryaboy wrote:
Hi folks,
Is anyone who uses HBaseStorage in Pig still on hbase 0.20.6?
There are a number of tickets outstanding to improve HBaseStorage and
I've suggested that we should add a shim lay
Thanks, that's ultimately what I went with. (Saw how it was done in the
AvroStorage class). Thought there might be a cleaner/simpler/better way
I was missing.
--jacob
@thedatachef
On Tue, 2011-02-01 at 21:22 +0530, Harsh J wrote:
> I remember facing this problem when trying to implemen
ed. I suspect (any Pig devs wanna help
me out here?) that it's called exactly once per task (ie. during the
call to 'setup()' in vanilla map-reduce land). If that's true then all
you'd have to do is set it the first time then read it on all subsequent
calls to setStoreLocation
ll by the time putNext is called. Is there some other way or am I
missing something obvious?
--jacob
@thedatachef
doesn't do anything more complex than launch a
hadoop streaming job and use the output in the subsequent steps
btw I write 90% of my mr jobs using either wukong or Pig. Only when
it's absolutely required do I use a language with as much overhead as
java :)
--jacob
@thedatachef
Sen
ear. This should at least get you on the right track.
--jacob
@thedatachef
On Sun, 2011-01-30 at 16:43 +0200, Cam Bazz wrote:
> Hello,
>
> I did look over to the GROUP operator, but could not find what i was
> looking for.
>
> I need to sum the values given by keys. This i
See the COGROUP operator
--jacob
@thedatachef
Sent from my iPhone
On Jan 30, 2011, at 5:10 AM, Cam Bazz wrote:
Hello,
Lets say I have two tables like
A:
1,11
2,15
and
B:
1,10
4,11
5,10
and joinin them
J = JOIN A by $0 FULL, B by $0
I get J:
1,11,1,21
2,16,,
,,4,11
,,5,10
which is a
p-and.html
then use the STREAM operator to stream your raw records (load them as
chararrays first) through your wukong script. It's not perfect but it
gets the job done.
--jacob
@thedatachef
On Sat, 2011-01-29 at 12:12 +, Alex McLintock wrote:
> I wonder if discussion of the Piggyb
at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:921)
at
org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper.readAddressOrThrow(ZooKeeperWrapper.java:477)
... 6 more
Even more confusing...
--jacob
@thedat
tion.receiveResponse(HBaseClient.java:521)
at
org.apache.hadoop.hbase.ipc.HBaseClient$Connection.run(HBaseClient.java:459)
hdfs://ip-10-113-50-207.ec2.internal/tmp/temp-1162684483/tmp278494015
I'm not sure what resource, exactly, it's looking for and not finding. Any
ideas? Thanks,
--jacob
@thedatachef
at, at this phase, output data is finally being written
to the hdfs. Do you have write permissions where you're trying to write?
What do the namenode logs say? The datanode logs?
--jacob
On Wed, 2011-01-26 at 13:32 -0300, Martin Z wrote:
>
>
>
>
>
>
>
>
> Hi
Martin,
When you look at the task logs for the particular reducer that's stuck,
what do you see? What kind of operations do you have going on in the
script, possibly a GROUP ALL?
--jacob
On Wed, 2011-01-26 at 12:44 -0300, Martin Z wrote:
> Hi all,
>
> I'm running a
at:chararray,sn:chararray,prot:chararray,foll:chararray,friend:chararray,status:chararray,favo:chararray,crat:chararray,sid:chararray,is_full:chararray,health:chararray);
cut_fields = FOREACH twuid GENERATE uid, sn;
STORE twuid INTO 'my_table' USING
org.apache.pig.backend.hadoop.hbase.HBaseStorage(
#x27;my_col_fam:screen_name');
Thanks for any help.
--jacob
@thedatachef
65 matches
Mail list logo