ote:
> OK, then just have an input file with one line of dummy data, and do
> something like this…
>
> DUMMY_DATA = load 'dummy_data';
>
> REAL_DATA = foreach DUMMY_DATA generate '$my_param';
>
> On Wed, Nov 4, 2015 at 3:20 PM, Norbert Burger >
> w
ass strA as a parameter,
> you should also be able to write it to a file that can be loaded in as an
> alias in the Pig script. Perhaps you are looking for something more
> direct?
>
> On Wed, Nov 4, 2015 at 2:46 PM, Norbert Burger
> wrote:
>
> > Hi folks - let's
Hi folks - let's say I have incoming param which has value "strA". Is
there a clever way to initialize an alias within Pig that contains a single
tuple with value "strA"?
In other words, is it possible to initialize an alias?
Norbert
I'll usually do something like this:
CLASSPATH=$(hadoop classpath):/usr/lib/pig/pig-withouthadoop.jar java -jar
...
Norbert
On Sat, Mar 22, 2014 at 8:14 PM, Jay Vyas wrote:
> What is the common way to add pig to the hadoop classpath?
>
> Do you folks generally put "pit-without-hadoop" into /u
Yes - take a look at PigCounterHelper. Instantiate a variable of this
type, and then you can call the method incrCounter() on it:
_counter = PigCounterHelper()
_counter.incrCounter("Stats","Total sessions",1)
Norbert
On Tue, Jul 30, 2013 at 11:05 AM, Serega Sheypak
wrote:
> Hi, is there any po
:
> Thanks for your reply.
>
> My goal is actually to AVOID using PARALLEL toi let PIG guess a good
> number of reducer by itself.
> Usually it works well for me, so I don't understadn why in that case it
> does not.
>
> Le 19/05/13 15:37, Norbert Burger a écrit :
>
Take a look at the PARALLEL clause:
http://pig.apache.org/docs/r0.7.0/cookbook.html#Use+the+PARALLEL+Clause
On Fri, May 17, 2013 at 10:48 AM, Vincent Barat wrote:
> Hi,
>
> I use this request to remove duplicated entries from a set of input files
> (I cannot use DISTINCT since some fields can be
Perhaps the general way to do this is to write a custom loader, but for
this simpler usecase, can you just filter out the record?
FILTER ... BY $0 MATCHES '^[0-9]+'
Norbert
On Wed, Mar 13, 2013 at 8:10 PM, Mix Nin wrote:
> I have a file as follows:
>
>
> event_timesession_id
> 12334545
Looking at your sample, it seems you have a GROUPBY generating these
bags...? Could you just insert a DISTINCT before this GROUP BY?
Norbert
On Fri, Mar 8, 2013 at 5:00 PM, Chan, Tim wrote:
> If I have a bag and would like to remove dupes, while saving the first
> occurrence, is this possible?
I thought Todd Lipcon's Hadoop Summit presentation [1] had some good info
on this topic.
[1] http://www.slideshare.net/cloudera/mr-perf
Norbert
On Thu, Mar 7, 2013 at 7:25 PM, Prashant Kommireddi wrote:
> You can do a few things here
>
>
>1. Increase mapred.child.java.opts to a higher numbe
We're using HBaseStorage to read some large rows (50k cols) and hitting
some perf issues (responseTooSlow and responseTooLarge in RS logs).
Looking through the code, I see that there's not an option that wraps
setBatch(), as opposed to the existing setCaching(). Small change, but I'm
wondering if
FILTER SIZE(tuple) == 14 won't work for your use case?
On Thu, Aug 30, 2012 at 3:39 PM, Sam William wrote:
> HI,
> I was wondering if it is possible validate records by checking the tuple
> length. I expect every record to have 14 fields, but some records might be
> corrupt. I want to filte
Yogesh -- based your log info you provided, it seems like your input
data is not tab-delimited, which is the default delimiter when using
PigStorage. As a result, your 3 space-separated fields are being
pulled as one into name:chararray, and then can't being split out
again when your try to store
Have you registered the JAR in your Pig script (for local mode) and
also added it to PIG_CLASSPATH (for remote mode, to get it into the
distributed cache)?
Norbert
On Mon, Jul 23, 2012 at 8:33 PM, Russell Jurney
wrote:
> The email package is a part of Jython, I believe:
> http://www.jython.org/d
Yang -- I think you'll get the representation you're looking for by
applying the FLATTEN a second time. Each instance of a FLATTEN strips off
a single layer.
Norbert
On Sun, Jun 24, 2012 at 5:57 PM, Jonathan Coveney wrote:
> generate K.(x1), K.(x2), K.(x3) , K.(x100); and generate
> K(x1,.
While it may be fine for many cases, If I'm reading the Nectar code
correctly, that transpose doesn't guarantee anything about the order of
rows within each column. In other words, transposing:
a - b -c
d - e - f
g - h - i
may give you different permutations of "a - d - g" as the first row,
depe
Yang -- have you seen Hortonworks' blogpost on this?
http://hortonworks.com/blog/transitive-closure-in-apache-pig/
Norbert
On Wed, Jun 20, 2012 at 10:15 PM, Prashant Kommireddi
wrote:
> Would embedding Pig in java or other languages work?
>
> http://pig.apache.org/docs/r0.10.0/cont.html#embed-j
Have you included the jython.jar in your PIG_CLASSPATH and registered from
within your script? The combination instructs the Pig backend to ship the
JAR to your cluster nodes.
Norbert
On Tue, Jun 19, 2012 at 8:28 PM, Chris Diehl wrote:
> yes indeed.
>
> On Tue, Jun 19, 2012 at 5:27 PM, Duckwor
gt; I have a trivial working UDF that just calls the pattern().matcher().find()
> but maybe there is something that I could use?
>
>
> --
> regards,
> Jakub Glapa
>
>
> On Mon, Jun 18, 2012 at 3:49 PM, Norbert Burger >wrote:
>
> > Jakub -- The MATCHE
Jakub -- The MATCHES operator accepts regexes as input. You can add a NOT
to invert the logic.
http://pig.apache.org/docs/r0.7.0/piglatin_ref2.html
Norbert
On Mon, Jun 18, 2012 at 7:14 AM, Jakub Glapa wrote:
> Hi all,
> I found in pig latin a 'matches' operator for pattern matching.
> I didn'
ould be the same even when using HbaseStorage ?
>
> Thanks again for your explications
>
> regards
>
> Baraa
>
> On Thu, Jun 14, 2012 at 6:57 PM, Norbert Burger >wrote:
>
> > Baraa -- this is standard Hadoop job functionality (that your job
> > destination is onl
Baraa -- this is standard Hadoop job functionality (that your job
destination is only a target dir, and the actual file names are
systematically generated). Pig inherits this functionality. You will have
1 "part" file per reduce task of your final Hadoop job in the target
directory.
Probably the
tion, group all to put all of them in a single bag..
>- in fact you can use RANDOM() to batch 10 ids in a bag (this
> avoids buffering all of the output in UDF).
>- or put them in a json and load json..
>
> etc...
>
> On Tue, May 29, 2012 at 10:20 AM, Norbert Burg
We're analyzing session(s) using Pig and HBase, and this session data is
currently stored in a single HBase table, where rowkey is a
sessionid-eventid combo (tall table). I'm trying to optimize the
"extract-all-events-for-a-given-session" step of our workflow.
This could be a simple JOIN. But th
Right - this was my point. Dropping the 'as' clause forces you to use
positional specifiers, which don't seem to have the same issue. Seems like
this would warrant a JIRA, if only to document the distinction a bit better.
Norbert
On Fri, May 18, 2012 at 1:13 PM, Nerius Landys wrote:
> > From
>From what I can tell, this does seem like a bug. Switching to positional
specifiers seems to work around the issue:
TEST = FOREACH MOVEMENT GENERATE $3;
POSA = FOREACH TEST GENERATE STRSPLIT($0, '/');
Possibly some casting is being applied in one case (positional specifiers)
but not the other?
e:
>
>
> hbase.zookeeper.quorum
> myzookeeper1
>
>
>
> Thanks Norbert, that was the exact tip I needed.
>
>
> Neil Yalowitz
> neilyalow...@gmail.com
>
> 2012-05-15 22:41:06,157 [main] INFO org.apache.zookeeper.ZooKeeper -
> Client environment
Is your HBase conf dir part of your Hadoop classpath? HBase configuration
settings are not pushed down to the mapreduce task level by default:
http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/package-summary.html#classpath
Norbert
On Tue, May 15, 2012 at 8:28 PM, Neil Yalowitz
This is a config/classpath issue, no? At the lowest level, Hadoop MR tasks
don't pick up settings from the HBase conf directory unless they're
explicitly added to the classpath, usually via hadoop/conf/hadoop-env.sh:
http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/package-summar
On Wed, Apr 18, 2012 at 7:57 PM, Jeremy Hanna wrote:
> Just curious - is there some way to do a remote connection? we have a few
> people here in Austin and one in Colorado at the Dachis Group that may want
> to participate.
>
+1 this.
Norbert
SUB_ID
> string,seg_ids
> array');
> opt = foreach mt generate C_SUB_ID, FLATTEN(STRSPLIT(seg_ids,':')) as
> s_seg_id;
>
> I don't seem to be getting the cross product, just something like the
> following
>
> 1133957209,61,0,1
> 4524524233,21,0
&
Have you taken a look already at CSVExcelStorage from the piggybank?
PigDump may work ok for basic datatypes, but seems like you'd quickly run
into issues with quoted strings and more complex types.
http://pig.apache.org/docs/r0.9.1/api/org/apache/pig/piggybank/storage/CSVExcelStorage.html
Norbe
Not sure if this will work in your use-case, but adding a FLATTEN to strip
the outer tuple before the FOREACHs seems to detour Pig enough to work
around the bug:
B = FOREACH A GENERATE FLATTEN(a);
B1 = FOREACH B GENERATE x, y;
B2 = FOREACH B GENERATE x, y;
Norbert
On Tue, Apr 10, 2012 at 2:42 AM
So in this case, it seems like JsonStringToMap is properly catching the
parse exception; in fact, it's the catch clause of the UDF that's
generating the "Could not json-decode string" message in your task tracker
logs.
Take a look at line 63 here:
https://github.com/kevinweil/elephant-bird/blob/ma
It seems to me that Alan is only interested in writing a loader which has a
non-default constructor (takes arguments), he doesn't need to create a UDF
which has this property.
Besides SimpleTextLoader, there are a number of examples of this in the Pig
codebase, including HBaseStorage. My own atte
Anurag - once you have elephant-bird parse the JSON into maps, then you
extract the nested JSON elements just as you would with any another map,
using the '#' projection operator. In other words, the following generates
3-element tuples containing id, name, and link:
A = LOAD ...
B = FOREACH A GE
Malcolm -- typically, you'd use a STRSPLIT and optional FLATTEN to tokenize
a chararray on some delimeter. So the following should work:
opt = foreach mt generate C_SUB_ID, flatten(STRSPLIT(seg_ids,':')) as
s_seg_id;
Norbert
On Thu, Apr 5, 2012 at 8:58 AM, Malcolm Tye wrote:
> Hi,
>I'm sto
Ben -- every non-local Pig run is converted into one or more Hadoop jobs,
so you can always use the jobtracker interface at http://namenode:50030 to
review job status and progress. The CLI interface to this is via the
command "hadoop job", eg. "hadoop job -list". Sometimes it's tricky to
match up
Are you able to connect on tcp/54311 to that machine? The higher-numbered
ports used by Hadoop/HBase are often blocked by firewalls.
Try using netcat -- "nc NAMENODEHOST 54311". Success would be indicated by
a hanging connection that's waiting for input.
Norbert
On Wed, Mar 28, 2012 at 1:48 PM
Pig uses Java's regular expression format, which anchors the regex at the
beginning and end of your string-to-be-searched. This means that the
predicate ...matches 'battery' only returns strings that are exactly
"battery", instead of strings that contain "battery".
Try using ...matches '.*battery
m/2166187
> >
> > Also, for ease of reference, here is the query:
> >
> > raw = LOAD 'hbase://events' USING
> > org.apache.pig.backend.hadoop.hbase.HBaseStorage('event:*', '-loadKey
> > true') AS (id:bytearray, events_map:map[]);
> >
> > Maybe I need
You're encountering problems connecting to HBase (presumably your Pig
script uses HBaseStorage). How does your hbase/conf/hbase-site.xml look?
Norbert
On Thu, Mar 22, 2012 at 9:16 PM, Ryan Cole wrote:
> Hello,
>
> I'm new to these lists. I'm trying to get Pig working, for my first time. I
> ha
mory intensive.
>
> Thanks,
> Prashant
>
> On Mar 21, 2012, at 6:50 AM, Norbert Burger
> wrote:
>
> > Folks -- how are folks handling the "productionalization" of their Pig
> > submit nodes?
> >
> > For our PROD environment, I originally thoug
Folks -- how are folks handling the "productionalization" of their Pig
submit nodes?
For our PROD environment, I originally thought we'd just have a few VMs
from which Pig jobs would be submitted onto our cluster. But on our 8GB
VMs, I found that we were often hitting heap OOM errors on a relativ
Markus - you might take a look at Sqoop; it's a package developed by
Cloudera to bridge the gap between Hadoop and relational databases. Your
workflow might be to kick off a Sqoop job that retrieves from your RDBMS
and dumps the results in HDFS. From there, you can use standard Pig to
process the
Pig has support for macros; take a look at DEFINE (macro) and IMPORT here:
http://pig.apache.org/docs/r0.9.1/cont.html#define-macros
Norbert
On Wed, Mar 14, 2012 at 8:41 AM, Markus Resch wrote:
> Hi everyone,
>
> In future we are going to have a bunch of Pig scripts in parallel which
> are doin
What prevents you from collecting your embedded PIg imports/calls into a
Python module, and calling out to it from there? As an example,
http://techblug.wordpress.com/2011/07/29/pagerank-implementation-in-pig/has
only a single file, but seems that it could be easily externalized if
needed.
Norber
Out of curiosity, is there an equivalent to .exec() for Python UDFs? We
had the same issue recently.
Norbert
On Wed, Mar 7, 2012 at 3:27 AM, Raghu Angadi wrote:
> On Tue, Mar 6, 2012 at 5:02 PM, Mark Kerzner >wrote:
>
> > Hi,
> >
> > I need to initialize the HBase connection, which I normally
Hi folks --
For a very sparse HBase table (2 column families, 1000s of columns) what's
the expected performance difference in using HBaseStorage with the
following two STORE methods? Note that in our use case, there only a
handful of unique rowkeys (approx 10).
1) GROUP BY the 1000s of columns b
DUMP sends your Pig script output to stdout of wherever you launched the
Grunt shell from. If you want to write to a file, check out the STORE
command instead.
On Fri, Mar 2, 2012 at 7:38 PM, Mohit Anchlia wrote:
> I run a job with "DUMP A". At the end it says output in "
> hdfs://db1:54310/tmp/
Prash, you can just model this tree as a simple graph adjacency list:
A1,A2
A2,A3
A3,A4
A4,Am
...
For nodes with more than one child, you simply extend each row
horizontally. Child/parent/descendant/ancestor are straightforward
applications of a traversal on this graph (BFS would be a good choic
Hi Mohit - you can use the "SET" keyword in Pig to make job configuration
changes:
http://pig.apache.org/docs/r0.9.2/cmds.html#set
Norbert
On Sun, Feb 26, 2012 at 8:20 PM, Mohit Anchlia wrote:
> How can I set map reduce parameters like io.sort.mb in pig like one can do
> using Configuration in
veney wrote:
> I bet it would be more efficient to just make a udf that goes from tuple to
> bag. This is not an uncommon request, though, and probably something we
> should build into pig.
>
> 2012/2/23 Norbert Burger
>
> > Try adding a FLATTEN before applying TOBAG:
&
TOKENIZE from STRSPLIT. I.e. all items
> ending up in a bag.
> Is there a way to further split these out such that they're elements of a
> bag? The TOBAG function just places the entire tuple in a bag...
>
> Thanks!
>
> On Wed, Feb 22, 2012 at 7:59 PM, Norbert Burger >wrot
Hi Flo - in your example data, it seems like the STRSPLIT() is working as
expected -- the function returns back a tuple which is being serialized in
the shell as "(t1,t2,t3,t4)".
When you mention "hostinglist isn't split properly", which part are you
referring to?
Norbert
On Wed, Feb 22, 2012 at
gt; > Thanks Bill and Norbert that seems like what I was looking for. I'm a bit
> > worried about
> > how much data/io this could create. But I'll see ;)
> >
> > Cheers
> > -Marco
> >
> > On Thu, Oct 27, 2011 at 6:03 PM, Norbert Burger <
> n
In case what you're looking for is an analysis over the full learning
duration, and not just the start interval, then one further insight is
that each original record can be transformed into a sequence of
records, where the size of the sequence corresponds to the session
duration. In other words,
Also the output schema for dummy3() doesn't match what's being returned.
You're returning a list of strings, but the outputschema specifies a bag,
which translates into a list of tuples (of something, eg. strings).
Norbert
On Wed, Oct 12, 2011 at 7:29 PM, Stan Rosenberg <
srosenb...@proclivitysy
For a more detailed explanation, take a look also at
http://pig.apache.org/docs/r0.8.1/cookbook.html#Use+the+Parallel+Features.
In summary:
* The PARALLEL keyword at the operator level overrides any other setting
* SET default_parallel determines reducer count for all blocking operators
(ones tha
Adding FLATTEN to your "grouped-by-multiple-cols" relation
(iq_per_region_per_gender) will make it much easier to join and visualize.
Once your join keys are flat string literals ("gender"), then it's just a
straightforward JOIN/FOREACH.
Here's a fragment that seems to do what you need:
A = LOAD
Take a look at the pig-withouthadoop target in the build.xml from your pig
release. Usage of the target is documented here (for a different goal,
although):
http://thedatachef.blogspot.com/2011/01/apache-pig-08-with-cloudera-cdh3.html
Essentially, the target allows you to build pig without hadoo
In case it's not obvious, you'd also need a FLATTEN(group) in there before
the FOREACH to break the tuple apart so that the fields could by synthesized
into a filename.
Norbert
On Mon, Oct 10, 2011 at 12:57 PM, Jacob Perkins
wrote:
> You'll have to run a FOREACH...GENERATE over the data first an
Is there an index or job ID available at the Pig level for each of the child
MR jobs that a single script can spawn? It'd be great, for example, if I
could write:
SET job.name rolluptask-pig.job.index
Where pig.job.index is either an index value (1st child job, 2nd child job,
etc.) or the actual
Hi Matt and the rest of the LinkedIn team:
Thanks for open-sourcing this project!
Norbert
On Tue, Sep 27, 2011 at 1:35 PM, Lakshminarayana Motamarri <
narayana.gupta...@gmail.com> wrote:
> Thank you for sharing the UDF's...
>
> On Tue, Sep 27, 2011 at 10:19 AM, Matthew Hayes
> wrote:
>
> > Hi
BASH exit status of 127 is a "command not found":
http://tldp.org/LDP/abs/html/exitcodes.html
Does geostream.pl shell out to other commands, or otherwise rely on $PATH?
Using fully-qualified paths may help.
Norbert
On Fri, Sep 23, 2011 at 6:59 PM, Deepak Reddy wrote:
> This is from the job tr
On Thu, Sep 15, 2011 at 1:16 PM, jiang licht wrote:
> Lower priority jobs then have to either wait for availability of reducer
> resources and idle, or get smaller number of slots than configured...
This last part ("or get smaller number...") isn't right, is it? The number
of reducer tasks is a
Do you have the hbase rebalancer turned on?
>
> On Sep 12, 2011, at 7:51 AM, Norbert Burger
> wrote:
>
> > Folks -- we have a timeseries-based table we recently converted to a
> salted
> > key schema [1] in order to avoid region hotspotting. The rowkey format
> i
Folks -- we have a timeseries-based table we recently converted to a salted
key schema [1] in order to avoid region hotspotting. The rowkey format is:
salt-timestamp-sessionid-eventtype, where:
salt has the form 00..13, and the timestamp is a Unix timestamp (epoch
based).
With the version 0.10.
e same sort cost? Seems that you'd have pay for a
sort as long as the requirement is top N.
Norbert
> On Thu, Sep 8, 2011 at 6:42 AM, Norbert Burger
> wrote:
>
>> Hi Ruslan -- no need to write your own UDF. There is a built-in
>> function TOP() which will extract for
Hi Ruslan -- no need to write your own UDF. There is a built-in
function TOP() which will extract for you the top N tuples of a
relation, where N is a configurable parameter. Take a look at:
http://pig.apache.org/docs/r0.9.0/func.html#topx
Norbert
On Thu, Sep 8, 2011 at 9:13 AM, Ruslan Al-Faki
Hi Ruihong -- if you can coalesce your data into (key1, value1, key2,
value2, key3, value3...) format, then you can use the TOMAP built-in
function:
https://issues.apache.org/jira/browse/PIG-1809
If you have more complex conversion requirements, then you can pass
the data off to a UDF which retur
On Tue, Sep 6, 2011 at 7:58 AM, shazz Ng wrote:
> So from Pig when I want to retrieve only the metrics and their value (= id
> for the data table) I do :
> tsd_metrics = LOAD 'hbase://tsdb-uid' using
> org.apache.pig.backend.hadoop.hbase.HBaseStorage('id:metrics', '-loadKey
> true') AS (metric
On Fri, Sep 2, 2011 at 11:26 AM, ugo jardonnet wrote:
> Is it safe and efficient to re-use relation names with pig >= 0.8.1:
>
> A = load 'NYSE_dividends' (exchange, symbol, date, dividends);
> A = filter A by dividends > 0;
It's safe, but not usually recommended. Take a look at:
http://ofps.or
I'm using HBaseStorage to load a large column family (many columns)
into a relation, which generates a map[] on each row. The maps are
wide and sparse (only a few keys exist on each row), and I'd ideally
like to GROUP all maps together by similar columns before passing off
to a UDF for further pro
On Mon, Aug 29, 2011 at 1:41 PM, Dmitriy Ryaboy wrote:
> Asif,
> Pig can only work with JVM-based languages (so, Jython for python and Rhino
> for javascript).
> You can use streaming to process data using R or non-jvm python versions.
Another option might be something like renjin
(http://code.go
I have a need within a larger Pig script to pull just a few records from an
Hbase table. I know the exact key, so it'd be trivial with a get() from a
UDF. Another alternative is use to a custom LOAD/FILTER combo, but this
would involve filtering off all but 3 of about 50 records.
>From a perform
finitely is (see
> HBaseTableInputFormat and PIG-1205). If that's not working, then it's a
> bug. Is there anything helpful in the logs?
>
>
>
> On Mon, Aug 15, 2011 at 9:19 AM, Norbert Burger >wrote:
>
> > Hi folks,
> >
> > We have a ~35 GB Hb
Hi folks,
We have a ~35 GB Hbase table that's split across several hundred regions.
I'm using the Pig version bundled with CDH3u1, which is 0.8.1 plus a few
patches. In particular, it includes PIG-1680.
With the push down filters from PIG-1680, my thought was that a LOAD/FILTER
combo like [1] wo
Isn't the query parser getting confused because you've chosen the same
column name mappings for the LHS and RHS of your join?
What happens if you change this:
AjD = JOIN ACCT BY (year,month), DIM BY (year,month) USING 'replicated';
to:
AjD = JOIN ACCT BY ($0,$1), DIM BY ($0,$1) USING 'replicate
00.000Z
> 1970-01-14T04:29:13.200Z,1970-01-01T00:00:00.000Z
> ...
>
> I am running a built version of pig from your repository and I downloaded
> the joda-time-1.6.jar from sourceforge.net website
>
> ,M
>
> -Ursprüngliche Nachricht-
> Von: Norbert Burger [mailto:nor
Seems like the Piggybank's UnixToISO should do what you're looking for - if
not long, in what datatype is your unixtimestamp arriving to your Pig
script?
Pig supports explicit casts [1], but I think the conversion you're referring
to should be handled implicitly.
[1] http://ofps.oreilly.com/title
What's the full stack trace you get when running with "pig -v"?
Norbert
On Thu, Jul 28, 2011 at 4:28 PM, Thiago Veiga wrote:
> hi,
>
> when i run my script pig i get the error message bellow
> ERROR 2017: Internal error creating job configuration
>
> does someone know this error message ?
>
> t
2 cents to the issue concerning loading by timestamp.
>
> Le 28/07/11 13:19, Norbert Burger a écrit :
>
> You can instruct HBaseStorage to load a subset of the rows using the "-gt"
>> and "-lt" options to HBaseStorage, documented here [1].
>>
>> I
You can instruct HBaseStorage to load a subset of the rows using the "-gt"
and "-lt" options to HBaseStorage, documented here [1].
I don't believe querying by timestamp is currently supported in Pig, based
on the comments to [2]. There is a standalone JIRA that's been created [3].
Norbert
[1]
h
Hi all - I'm having some difficulty returning a dict/map from a Python UDF.
The following code returns "java.lang.String cannot be cast to
org.python.core.PyObject". What am I missing?
Thanks,
Norbert
Here's my test Pig script:
REGISTER 'pigfuncs.py' USING jython AS pigfuncs;
cvps = LOAD 'test.
85 matches
Mail list logo