?
On Mon, May 19, 2014 at 2:30 PM, David Rosenstrauch wrote:
The reason why you want to copy to hdfs first is that hdfs splits the data
and distributes it across the nodes in the cluster. So if your input data
is large, you'll get much better efficiency/speed in processing it if
you'r
The reason why you want to copy to hdfs first is that hdfs splits the
data and distributes it across the nodes in the cluster. So if your
input data is large, you'll get much better efficiency/speed in
processing it if you're processing it in a distributed manner. (I.e.,
multiple machines eac
On 05/15/2014 04:45 PM, Steve Lewis wrote:
I have a medium size data set in the terrabytes range that currently lives
in the nfs file server of a medium institution. Every few months we want to
run a chain of five Hadoop jobs on this data.
The cluster is medium sized - 40 nodes about 200 simu
Running into an issue with some early attempts at converting over to MRv2.
I'm submitting a job using an MRv1 client installation, onto an MRv2
cluster. However, the client is displaying initialization log messages
for every single map task, like the log entries pasted below. In
addition to
I'm in the process of migrating over our Hadoop setup from MRv1 to MRv2
and have a question about interoperability.
We run our Hadoop clusters in the cloud (AWS) in a transient fashion.
I.e., start up clusters when needed, push all output from HDFS to S3,
and shut the clusters down when done.
Ran into a puzzling - and worrisome - issue late last night.
I was running a Hadoop streaming job, which reads input from 2 different
buckets in Amazon S3 (using s3n://). When the job completed, I realized
that the number of "map input records" was incorrect. (Several thousand
less than it s
I don't think you need a special input format. I think you just need to
specify your list of input files like this:
hdfs://HOST1/folder-name/file-name,hdfs://HOST2/folder-name/file-name, ...
HTH,
DR
On 04/09/2013 12:07 AM, Pedro Sá da Costa wrote:
Maybe there is some FileInputFormat class t
I've run into an issue with the reliability of distcp. Specifically, I
have a distcp job that seems to have not copied over a few files - and
yet didn't fail the job. Was hoping someone here might have some
suggestions/fix/workaround.
So I ran a distcp job. (Copying from one Amazon S3 buck
On 02/27/2013 01:42 PM, Adam Phelps wrote:
We have a job that uses a large lookup structure that gets created as a
static class during the map setup phase (and we have the JVM reused so
this only takes place once). However of late this structure has grown
drastically (due to items beyond our con
e
blacklist at the job level? I already tried restarting the tasktracker
on the machines, but that didn't do it.
Thanks,
DR
On 09/15/2011 12:02 PM, David Rosenstrauch wrote:
Thanks, but it doesn't. According to the link, restarting the task
tracker should take the node off the black
On 11/05/2012 04:02 PM, David Rosenstrauch wrote:
On 11/05/2012 03:54 PM, Steve Lewis wrote:
I need to determine what version of Hadoop is running - say under AWS - I
really want to use an API or properties in the running code but do not
know
how - any ideas
Probably not the best way, but one
On 11/05/2012 03:54 PM, Steve Lewis wrote:
I need to determine what version of Hadoop is running - say under AWS - I
really want to use an API or properties in the running code but do not know
how - any ideas
Probably not the best way, but one possible way: make a call to
Runtime.exec() and c
On 07/20/2012 09:20 AM, Dave Shine wrote:
I have a job that is emitting over 3 billion rows from the map to the reduce.
The job is configured with 43 reduce tasks. A perfectly even distribution
would amount to about 70 million rows per reduce task. However I actually got
around 60 million f
On 06/10/2012 08:39 PM, Arun C Murthy wrote:
Harsh - I'd be inclined to think it's worse than just setting
mapreduce.jobtracker.completeuserjobs.maximum - the only case this would solve
is if a single user submitted 25 *large* jobs (in terms of tasks) over a single
24-hr window.
That's actua
l users).
Try it out and let us know!
On Sat, Jun 9, 2012 at 12:37 AM, David Rosenstrauch wrote:
We're running 0.20.2 (Cloudera cdh3u4).
What configs are you referring to?
Thanks,
DR
On 06/08/2012 02:59 PM, Arun C Murthy wrote:
This shouldn't be happening at all...
What ver
ose should ensure your hadoop-1.x JT is very
reliable.
Arun
On Jun 8, 2012, at 8:26 AM, David Rosenstrauch wrote:
Our job tracker has been seizing up with Out of Memory (heap space) errors for
the past 2 nights. After the first night's crash, I doubled the heap space
(from the default
Our job tracker has been seizing up with Out of Memory (heap space)
errors for the past 2 nights. After the first night's crash, I doubled
the heap space (from the default of 1GB) to 2GB before restarting the
job. After last night's crash I doubled it again to 4GB.
This all seems a bit puzzl
On 02/16/2012 12:49 PM, ext-fabio.alme...@nokia.com wrote:
Hello All,
I wrote my own partitioner and I would like to see if it's working.
By printing the return of method getPartition I could see that the partitions
were different, but were they really working? To answer that I got the keys
tha
On 12/14/2011 09:39 AM, arun k wrote:
Hi Guys !
I want to analyse the completed Job counters like FILE/HDFS BYTES
READ/WRITTEN along with other values like average map/reduce task run time.
I see that Jobtracker GUI has this info but i want to programmatically
retrieve these values instead of ma
On 12/14/2011 08:20 AM, John Armstrong wrote:
Hi, there.
I've run into an odd situation, and I'm wondering if there's a way around
it; I'm trying to use Jackson for some JSON serialization in my program,
and I wrote/unit-tested it to work with Jackson 1.9. Then, in integration
testing, I starte
I can't think of an easy way to do this. There's a few not-so-easy
approaches:
* Implement numErrors as a Hadoop counter, and then have the application
which submitted the job check the value of that counter once the job is
complete and have the app throw an error if the counter exceeds the
On 11/14/2011 06:06 PM, Mapred Learn wrote:
Hi,
I have a use case where I want to pass a threshold value to a map-reduce
job. For eg: error records=10.
I want map-reduce job to fail if total count of error_records in the job
i.e. all mappers, is reached.
How can I implement this considering t
On 11/02/2011 03:44 AM, Dino Kečo wrote:
I have been reading about Hadoop metrics framework, and I was wondering is
it possible to create custom metrics for specific job.
You can create your own custom counters, to count particular statistics
you're interested in. That can take you a long way
On 09/30/2011 01:35 AM, Sudharsan Sampath wrote:
Hi,
We are looking to upgrade avro 1..4.1 to avro 1.5.x version. Does anyone
know if this can cause any incompatibility with hadoop cdh3 distro?
Thanks
Sudhan S
Possibly. IIRC, when I tried to use a newer version of Avro in our M/R
jobs I ran
tion.
Regards,
Subroto Sanyal
-Original Message-----
From: David Rosenstrauch [mailto:dar...@darose.net]
Sent: Wednesday, September 14, 2011 10:37 PM
To: mapreduce-user
Subject: Unblacklist a blacklisted tracker (at job level)?
I'm currently running a long-running job. One of our nodes was
mis
I'm currently running a long-running job. One of our nodes was
misconfigured, however, and so got blacklisted in the job.
I've since corrected the problem, though, and restarted the tasktracker.
But how can I "un-blacklist" the tasktracker so that the job starts
using it again?
TIA,
DR
* open a SequenceFile.Reader on the sequence file
* in a loop, call next(key,val) on the reader to read the next key/val
pair in the file (see:
http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/SequenceFile.Reader.html#next(org.apache.hadoop.io.Writable,%20org.apache.hadoop.i
On 09/06/2011 01:57 AM, Niels Basjes wrote:
Hi,
In the past i've had the same situation where I needed the data for
debugging. Back then I chose to create a second job with simply
SequenceFileInputFormat, IdentityMapper, IdentityReducer and finally
TextOutputFormat.
In my situation that worked
On 09/02/2011 09:14 AM, Panagiotis Antonopoulos wrote:
Hello guys,
I am using hadoop-0.20.2-cdh3u0 and I use MultipleOutputs to divide the HFiles
(which are the output of my MR job) so that each file can fit into one region
of the table where I am going to bulk load them.
Therefore I have on
On 07/27/2011 10:39 AM, John Armstrong wrote:
So I think I've figured out how to fix my problem with putting files on
the distributed classpath by digging through the code Hadoop uses to
process -libjars.
There is another, easier approach: if your app inherits from the Tool
class / runs via T
On 07/26/2011 06:52 PM, Mohamed Riadh Trad wrote:
Dear All,
Is it possible to set up a task with multiple reducers and merge reducers
outputs into one single file?
Bests,
Trad Mohamed Riadh, M.Sc, Ing.
Not within the map-reduce job, but you can merge it after the job is
done. At my previo
On 07/05/2011 11:24 AM, Pedro Sa Costa wrote:
Hi,
In my hadoop running example, the data ouput is compressed using gzip. I would
like to create a small java program that decompress the output. Can anyone
give an example on how to decompress the output in java using the hadoop API?
Write a Ja
On 06/29/2011 05:28 PM, Virajith Jalaparti wrote:
Hi,
I was wondering what scheduling algorithm is used in Hadoop (version
0.20.2 in particular), for a ReduceTask to determine in what order it is
supposed to read the map outputs from the various mappers that have been
run? In particular, suppose
Building on this, you could do something like the following to make it
more random:
if (numRecordsWritten < NUM_RECORDS_DESIRED) {
int n = generateARandomNumberBetween1and100();
if (n == 100) {
context.write(key, value);
}
}
The above would somewhat rando
r file but if I pass in HUGE text file or same file split
into 10 files, its choosing same no. of maps tasks (191 in my case).
Thanks Praveen
-Original Message- From: ext David Rosenstrauch
[mailto:dar...@darose.net] Sent: Monday, June 20, 2011 3:39 PM To:
mapreduce-user@hadoop.apache.org S
On 06/20/2011 03:24 PM, praveen.pe...@nokia.com wrote:
Hi there, I know client can send "mapred.reduce.tasks" to specify no.
of reduce tasks and hadoop honours it but "mapred.map.tasks" is not
honoured by Hadoop. Is there any way to control number of map tasks?
What I noticed is that Hadoop is ch
On 05/06/2011 01:12 PM, Geoffry Roberts wrote:
All,
I need for each one of my reducers to have read access to a certain object
or a clone thereof. I can instantiate this object a start up. How can I
give my reducers a copy?
Serialize it to a string, set it as a configuration setting on the j
On 05/03/2011 05:49 PM, Geoffry Roberts wrote:
David,
Thanks for the response.
Last thing first:
I am using org.apache.hadoop.mapreduce.lib.output.MultipleOutputs
which is differs from what your link points to
org.apache.hadoop.mapred.lib.MultipleOutputs. Using the class you propose,
requires
On 05/03/2011 01:21 PM, Geoffry Roberts wrote:
All,
I have three questions I would appreciate if anyone could weigh in on. I
apologise in advance if I sound whiny.
1. The namenode logs, when I view them from a browser, are displayed with
the lines wrapped upon each other as if there were no n
Is it possible to change the logging level for an individual job? (As
opposed to the cluster as a whole.) E.g., is there some key that I can
set on the job's configuration object that would allow me to bump up the
logging from info to debug just for that particular job?
Thanks,
DR
On 03/25/2011 03:07 PM, Pedro Costa wrote:
I don't know if this is what I want. I want to set the number of slots
that are available for the map and the reduce tasks to run. I don't
want to define the number of tasks.
On Fri, Mar 25, 2011 at 6:44 PM, David Rosenstrauch wrote:
On 03/
On 03/25/2011 02:26 PM, Pedro Costa wrote:
Hi,
is it possible to configure the total number of slots that a
TaskTracker has, to run the map and reduce tasks?
Thanks,
Yes. See the mapred.map.tasks and mapred.reduce.tasks settings.
HTH,
DR
On 03/24/2011 01:09 PM, Jonathan Coveney wrote:
I am not sure if this is the right listserv, forgive me if it is not. My
goal is this: monitor HDFS until a file is create, and then kick off a job.
Ideally I'd want to do this continuously, but the file would be create
hourly (with some sort of var
On 02/23/2011 07:24 PM, Mapred Learn wrote:
Thanks !
In this case, how can we print the metadata associated with the data
(sequence files), if user accessing this data wants to know it:
i) Is there any hadoop command that can do it ?
ii) Or we will have to provide some interface to the user to s
Perhaps this has been covered before, but I wasn't able to dig up any info.
Is there any way to run a custom "job cleanup" for a map/reduce job? I
know that each map and reduce has a cleanup method, which can be used to
clean up at the end of each task. But what I want is to run a single
cle
On 01/26/2011 05:43 AM, Joan wrote:
Hi,
I'm trying set Object into Hadoop's configuration but I don't know how to.
I'm want to do:
org.apache.hadoop.conf.Configuration conf = new
org.apache.hadoop.conf.Configuration();
conf.set("txt",myObject);
But It doesn't exist method like: conf.set(Stri
Take a look at CombineFileInputFormat.
DR
On 01/21/2011 09:24 AM, lei liu wrote:
There are two input direcoties:/user/test1/ and /user/test2/ , I want to
join the two direcoties content, in order to join the two directories, I
need to identity the content are handled by mapper from which direct
Maybe change "id" to be an IntWritable, and "str" to be a Text?
HTH,
DR
On 01/19/2011 09:36 AM, Joan wrote:
Hi Lance,
My custom object has Writable implement but I don't overrride toString
method?
*public class MyWritable implements DBWritable, Writable, Cloneable {
int id;
Strin
On 01/18/2011 03:09 PM, Mapred Learn wrote:
hi,
I have a basic question. How does partitioning work ?
Following is a scenario I created to put up my question.
i) A parttition function is defined as partitioning map-output based on
aphabetical sorting of the key i.e. a partition for keys startin
I assumed you were already doing this but yes, Alain is correct, you
need to set the output format too.
I initialize writing to sequence files like so:
job.setOutputFormatClass(SequenceFileOutputFormat.class);
FileOutputFormat.setOutputName(job, dataSourceName);
FileOutputFormat.setOutputPath(j
Sounds to me like your custom object isn't serializing properly.
You might want to read up on how to do it correctly here:
http://developer.yahoo.com/hadoop/tutorial/module5.html#types
FYI - here's an example of a custom type I wrote, which I'm able to
read/write successfully to/from a sequen
On 12/21/2010 09:50 PM, Chase Bradford wrote:
If you want a tmp file on a task's local host, just use java's
createTempFile from the File class. It creates a file in
java.io.tmp, which the task runner sets up in the task's workspace
and is cleaned by the TT even if the child jvm exits badly.
M
Not a very good approach for numerous reasons. (e.g., you generally
don't want to run another processing-intensive app - like a database -
on a hadoop node, you don't want to have to worry about the exact
situation you're worrying about (i.e., trying to make certain output get
routed to certai
On 12/19/2010 10:39 AM, Martin Becker wrote:
Hello everybody,
is there a possibility to make sure that certain/all reduce tasks,
i.e. the reducers to certain keys, are executed in a specified order?
This is Job internal, so the Job Scheduler is probably the wrong place to start?
Does the order i
It doesn't. But you really can't do what you're asking. Nor, I think,
would you really want to. The whole idea behind Hadoop is that it's a
distributed system whereby nodes are pretty much interchangeable.
There's nothing to be gained by trying to pin a particular reduce task
to a particular
And, as a follow-up, yes, we use the partitioner class to achieve this.
Our partioner runs a hashing algorithm which ensures that a given user
key will always map to a specific shard #.
DR
On 12/18/2010 01:16 PM, Hari Sreekumar wrote:
Hi Jane,
The partitioner class can be used to
On 12/18/2010 12:43 PM, Jane Chen wrote:
Hi All,
Is there anyway to influence where a reduce task is run? We have a case where
we'd like to choose the host to run the reduce task based on the task's input
key.
Any suggestion is greatly appreciated.
Thanks,
Jane
We don't do exactly that, b
On 12/18/2010 11:04 AM, Martin Becker wrote:
Hello everbody,
I am wondering if there is a feature allowing (in my case) reduce
tasks to communicate. For example by some volatile variables at some
centralized point. Or maybe just notify other running or to-be-running
reduce tasks of a completed r
Seems like CombineFileInputFormat.createPool() might help here. But I'm
a little unclear on usage. That method is protected ... and so then I
guess only accessible by subclasses?
Can anyone advise on usage here?
Thanks,
DR
On 12/08/2010 11:25 AM, David Rosenstrauch wrote:
Bit of a
the file paths whose data
is included in the split, but no way to identify which file path a
particular record came from.
Any workaround here?
Thanks,
DR
On 12/07/2010 11:08 PM, David Rosenstrauch wrote:
Thanks for the suggestion Shrijeet.
Same thought occurred to me on the way home from wo
t
On Tue, Dec 7, 2010 at 3:43 PM, David Rosenstrauch wrote:
Having an issue with some SequenceFiles that I generated, and I'm trying to
write a M/R job to fix them.
Situation is roughly this:
I have a bunch of directories in HDFS, each of which contains a set of 7
sequence files. Each seq
Having an issue with some SequenceFiles that I generated, and I'm trying
to write a M/R job to fix them.
Situation is roughly this:
I have a bunch of directories in HDFS, each of which contains a set of 7
sequence files. Each sequence file is of a different "type", but the
key type is the sa
On 11/22/2010 04:39 PM, praveen.pe...@nokia.com wrote:
Hi all, I am trying to figure how I can start a hadoop job
porgramatically from my Java application running in an app server. I
was able to run my map reduce job using hadoop command from hadoop
master machine but my goal is to run the same j
On 10/26/2010 03:01 AM, exception wrote:
Hi,
When launching a job in hadoop, usually we use "hadoop jar xxx.jar
input output". Can I run a job simply by a java program (In fully
distributed mode), without packing a jar. I know this will cause
problems because the remote nodes don't have the sour
We ran into this recently. Solution was to bump up the value of the
dfs.datanode.max.xcievers setting.
HTH,
DR
On 10/12/2010 03:53 PM, rakesh kothari wrote:
Hi,
My MR Job is processing gzipped files each around 450 MB and there are 24 of
them. File block size is 512 MB.
This job is faili
On 09/25/2010 10:24 AM, Martin Becker wrote:
Hello David,
thanks a lot. Yet I want java code to submit my application. I do not
want to mess with any kind of command line arguments or an executable,
neither Java nor Hadoop. I want to write a method that can set up and
submit a job to an arbitrar
On 09/24/2010 01:26 PM, Martin Becker wrote:
Hello David,
This will at best run my MapReduce process on the local Hadoop instance.
What do I do to submit it to a remote Hadoop cluster using Java code?
Martin
$ java -cp YourApp -libjars -jt
-fs
DR
On 09/24/2010 12:42 PM, Martin Becker wrote:
Hello David,
Thanks for your suggestions. I fail to see where your approach is
different from the one used in the tutorial.
The difference is that the tutorial launches the job using the "hadoop"
executable:
$ bin/hadoop jar /user/joe/wordcount.j
On 09/24/2010 11:12 AM, Martin Becker wrote:
Hi James,
I am trying to avoid to call any command line command. I want to submit
a job from within a java application. If possible without packing any
jar file at all. But I guess that will be necessary to allow Hadoop to
load the specific classes. T
Hmmm. Any idea as to why the undeprecation? I thought the intention
was to try to move everybody to the new API. Why the reversal?
Thanks,
DR
On 09/22/2010 12:29 PM, Tom White wrote:
Note that JobClient, along with the rest of the "old" API in
org.apache.hadoop.mapred, has been undeprecate
On 09/14/2010 10:10 PM, Pete Tyler wrote:
I'm trying to figure out how to achieve the following from a Java client,
1. My app (which is a web server) starts up
2. As part of startup my jar file, which includes my map reduce classes are
distributed to hadoop nodes
3. My web app uses map reduce to
On 09/12/2010 02:38 AM, Chase Bradford wrote:
I have a mapper class (extended from mapreduce.Mapper), where setup
reports to an outside resource. I want to make sure that most of the
time when the task fails or is killed, a specific chunk of cleanup
code is executed.
Just like there's a "setup
On 09/10/2010 02:18 AM, Angus Helm wrote:
Hi all, I have a task which involves loading a large amount of data
from a database and then using that data to process a large number of
small files. I'm trying to split up the file processing via mapreduce,
so each task runs as a map. However, the "load
On 09/08/2010 05:00 AM, Rita Liu wrote:
Hi :) I double check the logs. The log messages I added to mapper and
reducer in WordCount.java now appear in userlogs/, and the log messages I
added to run() and main() in WordCount.java appear on the console. Moreover
-- the log message I added to JobClie
On 08/25/2010 10:36 AM, Anfernee Xu wrote:
Thanks all for your help.
The challenge is that suppose I have 4 datanodes in cluster, but for a given
input, I have 2 splits, therefore only 2 nodes out of 4 will run M/R job,
say nodeA and nodeB, after the job completes, the data from input has been
s
On 08/25/2010 09:07 AM, Anfernee Xu wrote:
I'm new to Hadoop and I want to use it for my data processing. My
understanding is that each Split will be processed by a mapper task, so for
my application I have mapper in which I populate backend data store with
data from splits, after all splits are
On 08/23/2010 12:25 PM, Steve Hoffman wrote:
It appears that Job replaces JobClient/JobConf/etc. and you simply
call submit() to do a submit and return (similar to
JobClient.submitJob() did).
However, after submit() is called on Job, a call to getJobID() return
null. This seems very wrong... Ho
I had a job that I ran a few days ago that rolled over to the Job
tracker history. Now when I go view it in the history viewer although I
can see basic stats such as total # records in/out, I can no longer see
all the counter values (i.e, most notably my own custom counter values).
Is there a
Was reading up a bit today on configuring the settings for # task slots,
namely:
mapred.tasktracker.map.tasks.maximum
mapred.tasktracker.reduce.tasks.maximum
Was just wondering: couldn't (shouldn't?) this be done dynamically by
default? i.e., couldn't/shouldn't a slave node be able to comput
On 08/16/2010 05:48 PM, Ted Yu wrote:
No.
On Mon, Aug 16, 2010 at 1:25 PM, David Rosenstrauchwrote:
Is it possible for a M/R job to have no mapper? i.e.:
job.setMapperClass(null)? Or is it required that one at least use an
"identity mapper" (i.e., plain vanilla org.apache.hadoop.mapreduce.Ma
Is it possible for a M/R job to have no mapper? i.e.:
job.setMapperClass(null)? Or is it required that one at least use an
"identity mapper" (i.e., plain vanilla org.apache.hadoop.mapreduce.Mapper)?
Thanks,
DR
On 08/12/2010 07:02 PM, Ted Yu wrote:
How about hack #3:
maintain your installation of hadoop where you replace jackson jar with
v1.5.4 jar ?
Thanks for the reply Ted.
If I understand correctly, you're suggesting we keep our own customized
hadoop installation, which we'd install on all the bo
Anyone have any ideas how I might be able to work around
https://issues.apache.org/jira/browse/MAPREDUCE-1700 ? It's quite a
thorny issue!
I have a M/R job that's using Avro (v1.3.3). Avro, in turn, has a
dependency on Jackson (of which I'm using v1.5.4). I'm able to add the
jars to the di
On 08/12/2010 12:10 AM, Harsh J wrote:
On Thu, Aug 12, 2010 at 7:57 AM, David Rosenstrauch wrote:
On 08/11/2010 08:08 PM, Aaron Kimball wrote:
On a related note, if there's actually no difference between the 2
methods,
would anybody have any idea what could make the "mapred.j
On 08/11/2010 08:08 PM, Aaron Kimball wrote:
On Wed, Aug 11, 2010 at 3:13 PM, David Rosenstrauchwrote:
What's the preferred way to submit a job these days?
org.apache.hadoop.mapreduce.Job.submit() ? Or
org.apache.hadoop.mapred.JobClient.runJob()? Or does it even matter? (i.e.,
is there any di
On 08/11/2010 08:14 PM, Aaron Kimball wrote:
David,
Since you are directly instantiating the Mapper and Reducer (not using
ReflectionUtils), you are free to call setConf() yourself before you run the
test.
Sort of. What would wind up happening is that setConf would get called
twice: once by
What's the preferred way to submit a job these days?
org.apache.hadoop.mapreduce.Job.submit() ? Or
org.apache.hadoop.mapred.JobClient.runJob()? Or does it even matter?
(i.e., is there any difference between them?)
I've been trying to run a job using
org.apache.hadoop.mapreduce.Job.submit()
On 08/10/2010 04:54 PM, David Rosenstrauch wrote:
On 08/10/2010 04:24 PM, David Rosenstrauch wrote:
What I'm looking for is a way to initialize the reducer using a
particular Configuration object, which contains the parms/properties
that the reducer needs. I looked at the ReduceDriver
On 08/10/2010 04:24 PM, David Rosenstrauch wrote:
What I'm looking for is a way to initialize the reducer using a
particular Configuration object, which contains the parms/properties
that the reducer needs. I looked at the ReduceDriver class, however, and
there doesn't seem to be any
I'm trying to write some tests with the mrunit framework, but running
into a snag.
It seems that the mock Context objects that are being created are always
using a new, empty Configuration object. However, I've written my
reducer to implement Configurable, since in production I'm going to be
On 08/09/2010 09:14 PM, Harsh J wrote:
Another solution would be to create a custom named output using
mapred.lib.MultipleOutputs and collecting to that instead of the
job-set output format (which one can set to NullOutputFormat so it
doesn't complain about existing paths, etc.).
So if you'd wan
On 08/09/2010 05:45 PM, David Rosenstrauch wrote:
On 08/09/2010 04:01 PM, David Rosenstrauch wrote:
On a similar note, it looks like if I want to customize the name/path of
the generated SequenceFile my only option currently is to override
FileOutputFormat.getDefaultWorkFile().
a) Again, have
On 08/09/2010 04:01 PM, David Rosenstrauch wrote:
On a similar note, it looks like if I want to customize the name/path of
the generated SequenceFile my only option currently is to override
FileOutputFormat.getDefaultWorkFile().
a) Again, have I got this correct, or am I overlooking something
On 08/07/2010 02:06 AM, Harsh J wrote:
On Sat, Aug 7, 2010 at 11:20 AM, David Rosenstrauch wrote:
I'm using a SequenceFileOutputFormat. But I'd like to be able to set some
SequenceFile.Metadata on the SequenceFile.Writer that's getting created.
Doesn't look like there&
ndle that for you pretty easily :)
See github.com/cloudera/sqoop and archive.cloudera.com/cdh/3/sqoop for more
info.
Cheers,
- Aaron
On Wed, Aug 4, 2010 at 7:41 PM, Harsh J wrote:
AFAIK you don't really need serialization if your job is a map-only
one; the OutputFormat/RecWriter (
d the existing SFOP to allow this on JIRA
or the dev mailing list :)
On Mon, Aug 9, 2010 at 8:09 PM, David Rosenstrauch wrote:
On 08/07/2010 02:06 AM, Harsh J wrote:
On Sat, Aug 7, 2010 at 11:20 AM, David Rosenstrauch
wrote:
I'm using a SequenceFileOutputFormat. But I'd like to
On 08/07/2010 02:06 AM, Harsh J wrote:
On Sat, Aug 7, 2010 at 11:20 AM, David Rosenstrauch wrote:
I'm using a SequenceFileOutputFormat. But I'd like to be able to set some
SequenceFile.Metadata on the SequenceFile.Writer that's getting created.
Doesn't look like there&
I'm using a SequenceFileOutputFormat. But I'd like to be able to set
some SequenceFile.Metadata on the SequenceFile.Writer that's getting
created. Doesn't look like there's any easy way to do that, other than
overriding the SequenceFileOutputFormat.getRecordWriter() method.
Am I overlooking
I'm working on a M/R job which uses DBInputFormat. So I have to create
my own DBWritable for this. I'm a little bit confused about how to
implement this though.
In the sample code in the Javadoc for the DBWritable class, the
MyWritable implements both DBWritable and Writable - thereby forcin
On 08/04/2010 01:55 PM, Wilkes, Chris wrote:
On Aug 4, 2010, at 10:50 AM, David Rosenstrauch wrote:
On 08/04/2010 12:30 PM, Owen O'Malley wrote:
On Aug 4, 2010, at 8:38 AM, David Rosenstrauch wrote:
Anyone know if there's any particular reason why the new Partitioner
cla
1 - 100 of 106 matches
Mail list logo