Re: I need advice on whether my starting data needs to be in HDFS

2014-05-19 Thread David Rosenstrauch
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

Re: I need advice on whether my starting data needs to be in HDFS

2014-05-19 Thread David Rosenstrauch
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

Re: I need advice on whether my starting data needs to be in HDFS

2014-05-19 Thread David Rosenstrauch
, May 19, 2014 at 2:30 PM, David Rosenstrauch dar...@darose.netwrote: 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

Task inititalizations logging to console in MRv2

2014-04-14 Thread David Rosenstrauch
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

MRV1 / MRV2 interoperability question

2014-04-11 Thread David Rosenstrauch
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.

Inconsistent results when reading from s3n

2013-05-07 Thread David Rosenstrauch
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

Re: Combine data from different HDFS FS

2013-04-09 Thread David Rosenstrauch
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

Re: Large static structures in M/R heap

2013-02-27 Thread David Rosenstrauch
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

Distcp reliability issue

2013-02-27 Thread David Rosenstrauch
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

Re: Unblacklist a blacklisted tracker (at job level)?

2013-02-15 Thread David Rosenstrauch
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 blacklist. That didn't happen

Re: How do find the version of Hadoop inside a running Map or reduce task

2012-11-05 Thread David Rosenstrauch
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

Re: How do find the version of Hadoop inside a running Map or reduce task

2012-11-05 Thread David Rosenstrauch
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

Re: OK to run data node on same machine as secondary name node?

2012-08-16 Thread David Rosenstrauch
have https://issues.apache.org/jira/browse/HDFS-1457 (image transfer throttler) in the version of Hadoop you use, you can set it to a proper value and keep the SNN on a slave node without worrying about it hogging all the available bandwidth. On Thu, Aug 16, 2012 at 3:41 AM, David Rosenstrauch dar

OK to run data node on same machine as secondary name node?

2012-08-15 Thread David Rosenstrauch
I have a Hadoop cluster that's a little tight on resources. I was thinking one way I could solve this could be by running an additional data node on the same machine as the secondary name node. I wouldn't dare do that on the primary name node, since that machine needs to be extremely

Re: Distributing Keys across Reducers

2012-07-20 Thread David Rosenstrauch
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

Re: Namenode hangs on startup

2012-07-02 Thread David Rosenstrauch
A couple of thoughts: 1) It shouldn't take more than a couple of minutes for the data nodes to re-register after the name node comes back up. If the data nodes aren't registering, you might try restarting the data node daemons on each machine. They should all register soon after. (If not,

Re: Hadoop topology not working (all servers belongs to default rack)

2012-06-28 Thread David Rosenstrauch
On 06/27/2012 11:32 PM, Ben Kim wrote: Hi I got my topology script from http://wiki.apache.org/hadoop/topology_rack_awareness_scripts I checked that the script works correctly. But, in the hadoop cluster, all my servers get assigned to the default rack. I'm using hadoop 1.0.3, but had

Re: Hadoop topology not working (all servers belongs to default rack)

2012-06-28 Thread David Rosenstrauch
On 06/28/2012 10:46 AM, David Rosenstrauch wrote: On 06/27/2012 11:32 PM, Ben Kim wrote: Hi I got my topology script from http://wiki.apache.org/hadoop/topology_rack_awareness_scripts I checked that the script works correctly. But, in the hadoop cluster, all my servers get assigned

Re: Out of memory (heap space) errors on job tracker

2012-06-12 Thread David Rosenstrauch
at 12:37 AM, David Rosenstrauch dar...@darose.net 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 version of hadoop are you running? Potentially you need

Re: Out of memory (heap space) errors on job tracker

2012-06-12 Thread David Rosenstrauch
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

Out of memory (heap space) errors on job tracker

2012-06-08 Thread David Rosenstrauch
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

Re: Partitioners - How to know if they are working

2012-02-16 Thread David Rosenstrauch
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

Re: Overriding remote classes

2011-12-14 Thread David Rosenstrauch
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

Re: Analysing Completed Job info programmatically apart from Jobtracker GUI

2011-12-14 Thread David Rosenstrauch
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

Re: how to implement error thresholds in a map-reduce job ?

2011-11-15 Thread David Rosenstrauch
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

Unblacklist a blacklisted tracker (at job level)?

2011-09-14 Thread David Rosenstrauch
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

Re: Problem when using MultipleOutputs with many files

2011-09-02 Thread David Rosenstrauch
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

Re: HDFS File being written

2011-08-17 Thread David Rosenstrauch
On 08/17/2011 12:57 PM, Adam Shook wrote: Hello All, Is there any clean way to tell from the API (v0.20.2) that a file in HDFS is currently being written to? I've seen some exceptions before related to it, but I was hoping there is a clean way and Google isn't turning anything up for me.

Re: Merge Reducers Outputs

2011-07-27 Thread David Rosenstrauch
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

Re: Deep Magic on the distributed classpath

2011-07-27 Thread David Rosenstrauch
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

Re: Example on how to decompress the output file?

2011-07-05 Thread David Rosenstrauch
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

Re: How does a ReduceTask determine which MapTask output to read?

2011-06-29 Thread David Rosenstrauch
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,

Re: controlling no. of mapper tasks

2011-06-20 Thread David Rosenstrauch
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

Re: controlling no. of mapper tasks

2011-06-20 Thread David Rosenstrauch
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 Subject: Re

Re: How do I create a sequence file on my local harddrive?

2011-04-25 Thread David Rosenstrauch
On 04/22/2011 09:09 PM, W.P. McNeill wrote: I want to create a sequence file on my local harddrive. I want to write something like this: LocalFileSystem fs = new LocalFileSystem(); Configuration configuration = new Configuration(); Try doing this instead: Configuration

How to change logging level for an individual job

2011-04-13 Thread David Rosenstrauch
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

Re: How to abort a job in a map task

2011-04-07 Thread David Rosenstrauch
On 04/06/2011 08:40 PM, Haruyasu Ueda wrote: Hi all, I'm writing M/R java program. I want to abort a job itself in a map task, when the map task found irregular data. I have two idea to do so. 1. execulte bin/hadoop -kill jobID in map task, from slave machine. 2. raise an IOException to

Re: Developing, Testing, Distributing

2011-04-07 Thread David Rosenstrauch
On 04/07/2011 03:39 AM, Guy Doulberg wrote: Hey, I have been developing Map/Red jars for a while now, and I am still not comfortable with the developing environment I gathered for myself (and the team) I am curious how other Hadoop developers out-there, are developing their jobs... What IDE

Re: What does Too many fetch-failures mean? How do I debug it?

2011-03-31 Thread David Rosenstrauch
On 03/31/2011 05:13 PM, W.P. McNeill wrote: I'm running a big job on my cluster and a handful of attempts are failing with a Too many fetch-failures error message. They're all on the same node, but that node doesn't appear to be down. Subsequent attempts succeed, so this looks like a transient

Re: change number of slots in MR

2011-03-25 Thread David Rosenstrauch
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

Re: change number of slots in MR

2011-03-25 Thread David Rosenstrauch
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 Rosenstrauchdar...@darose.net wrote:

Re: A way to monitor HDFS for a file to come live, and then kick off a job?

2011-03-24 Thread David Rosenstrauch
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

Re: CDH and Hadoop

2011-03-24 Thread David Rosenstrauch
They do, but IIRC, they recently announced that they're going to be discontinuing it. DR On Thu, March 24, 2011 8:10 pm, Rita wrote: Thanks everyone for your replies. I knew Cloudera had their release but never knew Y! had one too... On Thu, Mar 24, 2011 at 5:04 PM, Eli Collins

Re: Writable Class with an Array

2011-03-17 Thread David Rosenstrauch
I would try implementing this using an ArrayWritable, which contains an array of IntWritables. HTH, DR On 03/17/2011 05:04 PM, maha wrote: Hello, I'm stuck with this for two days now ...I found a previous post discussing this, but not with arrays. I know how to write Writable class with

Re: hadoop fs -rmr /*?

2011-03-16 Thread David Rosenstrauch
On 03/16/2011 01:35 PM, W.P. McNeill wrote: On HDFS, anyone can run hadoop fs -rmr /* and delete everything. Not sure how you have your installation set but on ours (we installed Cloudera CDH), only user hadoop has full read/write access to HDFS. Since we rarely either login as user hadoop,

Re: Custom Input format...

2011-02-11 Thread David Rosenstrauch
On 02/11/2011 05:43 AM, Nitin Khandelwal wrote: Hi, I want to give a folder as input path to Map Red. Each Task should read one file out of that folder at once . i was using it before in 0.19 using multiFileSplit Format and my own Input format extending it. can u plz tell how to do the same in

Re: why is it invalid to have non-alphabet characters as a result of MultipleOutputs?

2011-02-08 Thread David Rosenstrauch
On 02/08/2011 05:01 AM, Jun Young Kim wrote: Hi, Multipleoutputs supports to have named outputs as a result of a hadoop. but, it has inconvenient restrictions to have it. only, alphabet characters are valid as a named output. A ~ Z a ~ z 0 ~ 9 are only characters we can take. I believe if I

Job-wide cleanup functionality?

2011-02-03 Thread David Rosenstrauch
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

Re: Streaming data locality

2011-02-03 Thread David Rosenstrauch
On 02/03/2011 12:16 PM, Keith Wiley wrote: I've seen this asked before, but haven't seen a response yet. If the input to a streaming job is not actual data splits but simple HDFS file names which are then read by the mappers, then how can data locality be achieved. Likewise, is there any

Re: How to set object in Configuration

2011-01-26 Thread David Rosenstrauch
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:

Re: how to write custom object using M/R

2011-01-18 Thread David Rosenstrauch
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

Re: how to write custom object using M/R

2011-01-18 Thread David Rosenstrauch
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);

Re: question for understanding partitioning

2011-01-18 Thread David Rosenstrauch
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

Re: MultipleOutputs Performance?

2011-01-13 Thread David Rosenstrauch
On 12/10/2010 02:16 PM, Harsh J wrote: Hi, On Thu, Dec 2, 2010 at 10:40 PM, Matt Tanquarymatt.tanqu...@gmail.com wrote: I am using MultipleOutputs to split a mapper input into about 20 different files. Adding this split has had an extremely adverse effect on performance. Is MultipleOutputs

Re: Getting a temporary directory in map jobs

2010-12-22 Thread David Rosenstrauch
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.

Re: Passing messages

2010-12-19 Thread David Rosenstrauch
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

Re: How to Influence Reduce Task Location.

2010-12-19 Thread David Rosenstrauch
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,

Re: How to Influence Reduce Task Location.

2010-12-19 Thread David Rosenstrauch
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

Re: How to Influence Reduce Task Location.

2010-12-19 Thread David Rosenstrauch
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

Re: Reduce Task Priority / Scheduler

2010-12-19 Thread David Rosenstrauch
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

Re: M/R file gather/scatter issue

2010-12-08 Thread David Rosenstrauch
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 work after I sent

Re: M/R file gather/scatter issue

2010-12-08 Thread David Rosenstrauch
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 snag

M/R file gather/scatter issue

2010-12-07 Thread David Rosenstrauch
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

Re: M/R file gather/scatter issue

2010-12-07 Thread David Rosenstrauch
Thanks for the suggestion Shrijeet. Same thought occurred to me on the way home from work after I sent this mail. Not sure why, but my brain was kinda locked onto the concept of the mapper being a no-op in this situation. Obviously doesn't have to be. Let me try hacking this together and

Re: Starting a Hadoop job programtically

2010-11-22 Thread David Rosenstrauch
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

Re: monitor the hadoop cluster

2010-11-11 Thread David Rosenstrauch
On 11/11/2010 02:52 PM, Da Zheng wrote: Hello, I wrote a MapReduce program and ran it on a 3-node hadoop cluster, but its running time varies a lot, from 2 minutes to 3 minutes. I want to understand how time is used by the map phase and the reduce phase, and hope to find the place to improve

HDFS file disappeared

2010-11-11 Thread David Rosenstrauch
Had a really peculiar thing happen today: a file that a job of mine created on HDFS seems to have disappeared, and I'm scratching my head as to how this could have happened without any errors getting thrown. I ran a M/R job that created a big bunch of files. Job completed without errors,

Re: HDFS file disappeared

2010-11-11 Thread David Rosenstrauch
Saw a couple more references to that block before the to delete blk messages: 2010-11-10 21:42:33,389 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addToInvalidates: blk_-4237880568969698703 is added to invalidSet of our ip prefix.169:50010 2010-11-10 21:42:33,389 INFO

Re: HDFS file disappeared

2010-11-11 Thread David Rosenstrauch
On 11/11/2010 05:10 PM, David Rosenstrauch wrote: Saw a couple more references to that block before the to delete blk messages: 2010-11-10 21:42:33,389 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addToInvalidates: blk_-4237880568969698703 is added to invalidSet of our ip prefix

Re: running job without jar

2010-10-26 Thread David Rosenstrauch
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

Re: Testing zookeeper outside the source distribution?

2010-10-18 Thread David Rosenstrauch
On 10/18/2010 08:17 AM, Anthony Urso wrote: Anyone have any pointers on how to test against ZK outside of the source distribution? All the fun classes (e.g. ClientBase) do not make it into the ZK release jar. Right now I am manually running a ZK node for the unit tests to connect to prior to

Re: Failures in the reducers

2010-10-13 Thread David Rosenstrauch
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

Re: Create node with ancestors?

2010-10-07 Thread David Rosenstrauch
On 10/05/2010 06:20 PM, David Rosenstrauch wrote: The ZK create method explicitly states in the documentation If the parent node does not exist in the ZooKeeper, a KeeperException with error code KeeperException.NoNode will be thrown. ( (http://hadoop.apache.org/zookeeper/docs/current/api/org

Re: Create node with ancestors?

2010-10-07 Thread David Rosenstrauch
if you're doing it frequently (which we don't do), but it works for us. ~Jared On Thu, Oct 7, 2010 at 11:35 AM, David Rosenstrauchdar...@darose.net wrote: On 10/05/2010 06:20 PM, David Rosenstrauch wrote: The ZK create method explicitly states in the documentation If the parent node does

Create node with ancestors?

2010-10-05 Thread David Rosenstrauch
The ZK create method explicitly states in the documentation If the parent node does not exist in the ZooKeeper, a KeeperException with error code KeeperException.NoNode will be thrown. (

Re: JobClient using deprecated JobConf

2010-09-27 Thread David Rosenstrauch
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

Re: JobClient using deprecated JobConf

2010-09-22 Thread David Rosenstrauch
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 undeprecated

Re: jobtracker: Cannot assign requested address

2010-09-21 Thread David Rosenstrauch
On 09/21/2010 03:17 AM, Jing Tie wrote: I am still suffering from the problem... Did anyone encounter it before? Or any suggestions? Many thanks in advance! Jing On Fri, Sep 17, 2010 at 5:19 PM, Jing Tietiej...@gmail.com wrote: Dear all, I am having this exception when starting jobtracker,

Re: migrating from 0.18 to 0.20

2010-09-16 Thread David Rosenstrauch
It certainly is! I wasted a few hours on that a couple of weeks back. DR On 09/16/2010 02:58 AM, Lance Norskog wrote: After this, if you add anything to the conf object, it does not get added to the job. This is a source of confusion. Mark Kerzner wrote: Thanks! Mark On Wed, Sep 15, 2010

Re: Deployment of jar files at startup

2010-09-15 Thread David Rosenstrauch
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

Re: Multiple DataNodes on a single machine

2010-09-15 Thread David Rosenstrauch
On 09/15/2010 11:50 AM, Arv Mistry wrote: Hi, Is it possible to run multiple data nodes on a single machine? I currently have a machine with multiple disks and enough disk capacity for replication across them. I don't need redundancy at the machine level but would like to be able to handle a

Re: How to architecture large initialization in map task

2010-09-13 Thread David Rosenstrauch
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

Re: custom task cleanup even when task is killed?

2010-09-13 Thread David Rosenstrauch
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

Re: Writable questions

2010-08-31 Thread David Rosenstrauch
On 08/31/2010 12:58 PM, Mark wrote: I have a question regarding outputting Writable objects. I thought all Writables know how to serialize themselves to output. For example I have an ArrayWritable of strings (or Texts) but when I output it to a file it shows up as

Re: Writable questions

2010-08-31 Thread David Rosenstrauch
On 08/31/2010 02:09 PM, Mark wrote: On 8/31/10 10:07 AM, David Rosenstrauch wrote: On 08/31/2010 12:58 PM, Mark wrote: I have a question regarding outputting Writable objects. I thought all Writables know how to serialize themselves to output. For example I have an ArrayWritable of strings

Re: how to implement post-mapper processing

2010-08-25 Thread David Rosenstrauch
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

Re: how to implement post-mapper processing

2010-08-25 Thread David Rosenstrauch
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

Re: Custom partitioner for hadoop

2010-08-25 Thread David Rosenstrauch
On 08/25/2010 12:40 PM, Mithila Nagendra wrote: In order to avoid this I was thinking of passing the range boundaries to the partitioner. How would I do that? Is there an alternative? Any suggestion would prove useful. We use a custom partitioner, for which we pass in configuration data that

Re: Custom partitioner for hadoop

2010-08-25 Thread David Rosenstrauch
If you define a Hadoop object as implementing Configurable, then its setConf() method will be called once, right after it gets instantiated. So each partitioner that gets instantiated will have its setConf() method called right afterwards. I'm taking advantage of that fact by calling my own

Re: hadoop 0.20 migration for JobClient?

2010-08-23 Thread David Rosenstrauch
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...

Viewing counters in history job

2010-08-23 Thread David Rosenstrauch
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

Null mapper?

2010-08-16 Thread David Rosenstrauch
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

Re: Null mapper?

2010-08-16 Thread David Rosenstrauch
On 08/16/2010 05:48 PM, Ted Yu wrote: No. On Mon, Aug 16, 2010 at 1:25 PM, David Rosenstrauchdar...@darose.netwrote: 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

Re: Preferred way to submit a job?

2010-08-12 Thread David Rosenstrauch
On 08/12/2010 12:10 AM, Harsh J wrote: On Thu, Aug 12, 2010 at 7:57 AM, David Rosenstrauchdar...@darose.net 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

How to work around MAPREDUCE-1700

2010-08-12 Thread David Rosenstrauch
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

Re: How to work around MAPREDUCE-1700

2010-08-12 Thread David Rosenstrauch
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

Re: DiskChecker$DiskErrorException: Could not find any valid local directory -- lots of free disk space

2010-08-12 Thread David Rosenstrauch
On 08/12/2010 01:42 PM, Rares Vernica wrote: I forgot to mention that in my cluster the HDFS replication is set to 1. I know this is not recommended but I only have 5 nodes in the cluster, there are no failures There will be! :-) DR

Re: mrunit question

2010-08-11 Thread David Rosenstrauch
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

Re: Sequence Number Generation With Zookeeper

2010-08-10 Thread David Rosenstrauch
need to tweak in the code to make it release-ready. (e.g., Change package names? Slap an Apache license on it? etc.) Thanks, DR On 08/06/2010 10:39 PM, David Rosenstrauch wrote: I'll run it by my boss next week. DR On 08/06/2010 07:30 PM, Mahadev Konar wrote: Hi David, I think it would

Re: Sequence Number Generation With Zookeeper

2010-08-10 Thread David Rosenstrauch
, David Rosenstrauch wrote: Good news! I got approval to release this code! (Man, I love working for a startup!!!) :-) So anyone know: what's the next step? Do I need to obtain commit privileges? Or do I deliver the code to someone who has commit privs who shepherds this for me? Also, what

  1   2   >