Re: Issues with ResourceManager scheduling functions

2013-12-09 Thread Sandy Ryza
to them? Can you give an example? > > Many thanks. > > > Bill > > > On Mon, Dec 9, 2013 at 3:30 AM, Sandy Ryza wrote: > >> YARN currently is unable to handle requests with different resource >> requirements at the same priority (YARN-314). Using different priori

Re: Issues with ResourceManager scheduling functions

2013-12-09 Thread Sandy Ryza
Mon, Dec 9, 2013 at 12:23 AM, Bill Q wrote: > No, all default 0. Should I use different priorities? > > Many thanks. > > > Bill > > > On Mon, Dec 9, 2013 at 2:53 AM, Sandy Ryza wrote: > >> Are you requesting the different container types at different prioritie

Re: Issues with ResourceManager scheduling functions

2013-12-08 Thread Sandy Ryza
Are you requesting the different container types at different priorities? -Sandy On Sun, Dec 8, 2013 at 8:52 PM, Bill Q wrote: > Hi, > I recently implemented an application using YARN. Everything works fine, > except for the illogical scheduling behavior of the ResourceManger, which > might be

Re: real time analytics on hadoop using spark or storm

2013-12-06 Thread Sandy Ryza
As Azurry said, Spark Streaming can process data in small batches as well. An advantage of Spark Streaming over Storm is that the same code can be used both for small and large batches. Both Spark and Storm can be used with Hadoop. -Sandy On Fri, Dec 6, 2013 at 5:27 PM, Azuryy Yu wrote: > sp

Re: MapReduce Job running Problems with queue designation in fair scheduler for Yarn-2.2.0.

2013-12-06 Thread Sandy Ryza
Hi Tang, One concerning thing is that the vcores for amelie's maxResources is set to 1. Not sure how that would lead to the error you're seeing, but it would prevent applications from running in that queue. Is your job able to run if you set this to someting higher? -Sandy On Thu, Dec 5, 2013

Re: Perfect configuration setting

2013-12-03 Thread Sandy Ryza
s allowed. > > > On Tue, Dec 3, 2013 at 4:26 PM, Sandy Ryza wrote: > >> Hi Geelong, >> >> Check out Todd Lipcon's presentation on tuning MapReduce performance: >> http://www.slideshare.net/cloudera/mr-perf >> >> -Sandy >> >> >> On Mon,

Re: Perfect configuration setting

2013-12-03 Thread Sandy Ryza
Hi Geelong, Check out Todd Lipcon's presentation on tuning MapReduce performance: http://www.slideshare.net/cloudera/mr-perf -Sandy On Mon, Dec 2, 2013 at 11:14 PM, Geelong Yao wrote: > Hi Everyone > > I am now testing the best performance of my cluster > Can anyone give me some *formulas or

Re: MRV2 job takes to long to start

2013-11-28 Thread Sandy Ryza
What scheduler are you using? What do you mean by "start"? For the first map task to start? -Sandy On Thu, Nov 28, 2013 at 6:07 AM, Juan Martin Pampliega wrote: > Hi, > > I have a map-reduce job that was developed for MRV1 and is now being run > daily with no modifications in a hadoop 2.2 cl

Re: 答复: problems of FairScheduler in hadoop2.2.0

2013-11-27 Thread Sandy Ryza
; fair > > datadev admins > > xxx1,xxx2 datadev > > > > > > 1mb,10vcores > > 3mb,30vcores > > 10 > > 2.0 > >

Re: problems of FairScheduler in hadoop2.2.0

2013-11-27 Thread Sandy Ryza
Hi, Can you share the contents of your fair-scheduler.xml? If you submit just a single job, does it run? What do you see if you go to /ws/v1/cluster/scheduler? -Sandy On Wed, Nov 27, 2013 at 12:09 AM, 麦树荣 wrote: > Hi, all > > > > When I run jobs in hadoop 2.2.0, I encounter a problem. Sud

Re: Any reference for upgrade hadoop from 1.x to 2.2

2013-11-22 Thread Sandy Ryza
For MapReduce and YARN, we recently published a couple blog posts on migrating: http://blog.cloudera.com/blog/2013/11/migrating-to-mapreduce-2-on-yarn-for-users/ http://blog.cloudera.com/blog/2013/11/migrating-to-mapreduce-2-on-yarn-for-operators/ hope that helps, Sandy On Fri, Nov 22, 2013 at 3

Re: Limit on total jobs running using fair scheduler

2013-11-19 Thread Sandy Ryza
Unfortunately, this is not possible in the MR1 fair scheduler without setting the jobs for individual pools. In MR2, fair scheduler hierarchical queues will allow setting maxRunningApps at the top of the hierarchy, which would have the effect you're looking for. -Sandy On Tue, Nov 19, 2013 at 2

Re: Allocating Containers on a particular Node in Yarn

2013-11-14 Thread Sandy Ryza
umber of racks in the cluster, this > > feature is disabled by default, set to -1. > > > > > > > > -Gaurav > > > > *From:* Sandy Ryza [mailto:sandy.r...@cloudera.com] > *Sent:* Thursday, November 14, 2013 12:41 PM > > *To:* user@hadoop.apache.

Re: Allocating Containers on a particular Node in Yarn

2013-11-14 Thread Sandy Ryza
ners. > > 3. With scheduler delay on and relax locality set to true without > requesting rack, I don’t get the containers on the required host > What scheduler are you using and what properties are you using to turn the scheduler delay on? > > > Thanks > > -Ga

Re: Allocating Containers on a particular Node in Yarn

2013-11-14 Thread Sandy Ryza
; LOG.info("Released container, id={}", containerId.getId()); > amRmClient.releaseAssignedContainer(containerId); > } > return amRmClient.allocate(0); > > > > -Gaurav > > > On 11/13/2013 07:36 PM, Sandy Ryza wrote: > > In that case, the

Re: Allocating Containers on a particular Node in Yarn

2013-11-13 Thread Sandy Ryza
hanks > -Gaurav > > On 11/13/2013 5:04 PM, gaurav wrote: > > I have hadoop-2.2.0 > > Thanks > -Gaurav > > On 11/13/2013 4:59 PM, Sandy Ryza wrote: > > What version are you using? Setting the relax locality to true for nodes > is always correct. For racks,

Re: Allocating Containers on a particular Node in Yarn

2013-11-13 Thread Sandy Ryza
> > Thanks > -Gaurav > > On 11/13/2013 4:24 PM, Sandy Ryza wrote: > > [moving to user list] > > Right. relaxLocality needs to be set on the next level up. It > determines whether locality can be relaxed to that level. Confusing, I > know. If you are using AMRM

Re: Allocating Containers on a particular Node in Yarn

2013-11-13 Thread Sandy Ryza
req.getCapability(), req, > req.getRelaxLocality());// > /// > > > Thanks > -Gaurav > > > On 11/13/2013 4:02 PM, Sandy Ryza wrote: > >> Yeah, specifying a host name with relaxLocality is meaningful. Schedulers >> use delay scheduling ( >> http://

Re: question about preserving data locality in MapReduce with Yarn

2013-10-31 Thread Sandy Ryza
r that needs to consider HDFS > data-locality. thx. > > r. > > > On Mon, Oct 28, 2013 at 10:21 PM, Sandy Ryza wrote: > >> Hi Ricky, >> >> The input splits contain the locations of the blocks they cover. The AM >> gets the information from the input splits

Re: question about preserving data locality in MapReduce with Yarn

2013-10-28 Thread Sandy Ryza
Hi Ricky, The input splits contain the locations of the blocks they cover. The AM gets the information from the input splits and submits requests for those location. Each container request spans all the replicas that the block is located on. Are you interested in something more specific? -Sand

Re: yarn.nodemanager.vmem-check-enabled

2013-10-23 Thread Sandy Ryza
Hi John, At Cloudera we turn it off by default and recommend doing so because of the way that Java can behave weirdly with high vmems. -Sandy On Sun, Oct 20, 2013 at 7:34 AM, John Lilley wrote: > Is this option typically enabled in practice? Is there a way for a > specific AM to disable it,

Re: Yarn never use TeraSort#TotalOrderPartitioner when run TeraSort job?

2013-10-18 Thread Sandy Ryza
Hi Sam, Have you tried changing the map or reduce classes and seeing if that has any effect? -Sandy On Fri, Oct 18, 2013 at 8:05 AM, Ravi Prakash wrote: > Sam, I would guess that the jar file you think is running, is not actually > the one. I am guessing that in the task classpath, there is a

Re: State of Art in Hadoop Log aggregation

2013-10-11 Thread Sandy Ryza
Just a clarification: Cloudera Manager is now free for any number of nodes. Ref: http://www.cloudera.com/content/cloudera/en/products/cloudera-manager.html -Sandy On Fri, Oct 11, 2013 at 7:05 AM, DSuiter RDX wrote: > Sagar, > > It sounds like you want a management console. We are using Clouder

Re: Non data-local scheduling

2013-10-03 Thread Sandy Ryza
ind a good description of what this feature means (what is a > scheduling opportunity, how many are there?). It does not seem to be in the > current documentation > http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/CapacityScheduler.html > > > 2013/10/3 Sandy Ryz

Re: Non data-local scheduling

2013-10-03 Thread Sandy Ryza
Hi Andre, Try setting yarn.scheduler.capacity.node-locality-delay to a number between 0 and 1. This will turn on delay scheduling - here's the doc on how this works: For applications that request containers on particular nodes, the number of scheduling opportunities since the last container assi

Re: Cluster config: Mapper:Reducer Task Capapcity

2013-09-30 Thread Sandy Ryza
Hi Himanshu, Changing the ratio is definitely a reasonable thing to do. The capacities come from the mapred.tasktracker.map.tasks.maximum and mapred.tasktracker.reduce.tasks.maximum tasktracker configurations. You can tweak these on your nodes to get your desired ratio. -Sandy On Mon, Sep 30,

Re: Which Subphases Do Times on JobHistory Web UI Cover

2013-09-24 Thread Sandy Ryza
Average map time includes everything the map task is doing, i.e. all the things you mentioned. Reduce time does not cover shuffle time. Reduce time is the time spent calling the reducer function and writing its output to HDFS. Merge time is related to reduce, not map. -Sandy On Tue, Sep 24, 2

Re: Semantics of ApplicationResourceUsageReport

2013-09-21 Thread Sandy Ryza
Hi Albert, You're correct about used. Reserved is a little bit more arcane - it refers to a mechanism that schedulers use to prevent applications with larger container sizes from starving. Applications place container "reservations" on nodes, and no other containers can be placed on the node unt

Re: YARN MapReduce 2 concepts

2013-09-19 Thread Sandy Ryza
Hi Mohit, answers inline On Fri, Sep 20, 2013 at 1:33 AM, Mohit Anchlia wrote: > I am going through the concepts of resource manager, application master > and node manager. As I undersand resource manager receives the job > submission and launches application master. It also launches node manage

Re: Scheduler question

2013-09-09 Thread Sandy Ryza
Hi John, YARN schedulers handle this with the concept of "reservations". Scheduling decisions occur on node heartbeats. When a node that is full heartbeats, the next application that should be able to place a container on it gets to place a "reservation" on it. Each node has space for a single

Re: question about fair scheduler

2013-08-23 Thread Sandy Ryza
That's right that the other 2 apps will end up getting 10 resources each, but as more resources become released, eventually the cluster will converge to a fair state. I.e. if the first app requested additional resources after releasing resources, it would not receive any more until either another

Re: Is fair scheduler still experimental?

2013-08-22 Thread sandy . ryza
Moving to cdh-user, Hi, The Fair Scheduler in 4.3 is stable and is recommended by Cloudera. -Sandy On Aug 22, 2013, at 6:20 PM, ch huang wrote: > hi,all: > i use cdh4.3 yarn , it's default scheduler is capacity scheduler ,i > want to switch to fair scheduler,but i see doc says "NOT

Re: Is there any way to use a hdfs file as a Circular buffer?

2013-08-15 Thread Sandy Ryza
Hi Lin, It might be worth checking out Apache Flume, which was built for highly parallel ingest into HDFS. -Sandy On Thu, Aug 15, 2013 at 11:16 AM, Adam Faris wrote: > If every device can send it's information as a 'event', you could use a > publish-subscribe messaging system like Apache Kafk

Re: Calling a MATLAB library in map reduce program

2013-08-14 Thread Sandy Ryza
To add to that, if you want to take advantage of MapReduce, e.g. you need to do a distributed grouping or sort, pipes or streaming would be the way to go. If you're mainly interested in running your code in parallel on a cluster, distributed shell, a YARN application outside of MapReduce, could be

Re: Maven Cloudera Configuration problem

2013-08-13 Thread sandy . ryza
Nothing in your pom.xml should affect the configurations your job runs with. Are you running your job from a node on the cluster? When you say localhost configurations, do you mean it's using the LocalJobRunner? -sandy (iphnoe tpying) On Aug 13, 2013, at 9:07 AM, Pavan Sudheendra wrote: > Wh

Re: Maven Cloudera Configuration problem

2013-08-13 Thread sandy . ryza
Hi Pavan, Configuration properties generally aren't included in the jar itself unless you explicitly set them in your java code. Rather they're picked up from the mapred-site.xml file located in the Hadoop configuration directory on the host you're running your job from. Is there an issue you'

Re: why FairScheduler prefer to schedule MR jobs into the same node?

2013-08-08 Thread Sandy Ryza
Hi devdoer, What version are you using? -Sandy On Thu, Aug 8, 2013 at 4:25 AM, devdoer bird wrote: > HI: > > I configure the FairScheduler with default settings and my job has 19 > reduce tasks. I found that all the reduce tasks are schedule to run in one > node. > > While with default FIFO

Re: whitelist feature of YARN

2013-08-07 Thread Sandy Ryza
ect that you submit a ContainerRequest for each container you want. > -Kishore > > > > > On Wed, Aug 7, 2013 at 11:37 AM, Sandy Ryza wrote: > >> YARN-521, which brings whitelisting to the AMRMClient APIs, is now >> included in 2.1.0-beta. Check out the doc for

Re: whitelist feature of YARN

2013-08-06 Thread Sandy Ryza
-Kishore > > > On Mon, Jul 8, 2013 at 10:53 PM, Sandy Ryza wrote: > >> Hi Krishna, >> >> From your previous email, it looks like you are using the AMRMClient >> APIs. Support for whitelisting is not yet supported through them. I am >> working on this in Y

Re: Parameter 'yarn.nodemanager.resource.cpu-cores' does not work

2013-07-22 Thread Sandy Ryza
Hi Sam, LinuxResourceCalculatorPlugin and DominantResourceCalculator control separate things. The former is for a NodeManager to calculate the resource usage of a container process so that it can kill it if it gets too large. The latter is used by the Capacity Scheduler to allocate containers, a

Re: API difference between hadoop branch1 and branch2?

2013-07-18 Thread Sandy Ryza
Hi Li Yu, I don't think it has been published yet, but a document on the MapReduce changes was recently completed at https://issues.apache.org/jira/browse/MAPREDUCE-5184. -Sandy On Thu, Jul 11, 2013 at 4:18 AM, Yu Li wrote: > Dear all, > > I have some applications used to run on hadoop-1.1.1,

Re: Running hadoop for processing sources in full sky maps

2013-07-13 Thread Sandy Ryza
Hi Andrea, For copying the full sky map to each node, look up the distributed cache. It works by placing the sky map file on HDFS and each task will pull it down when needed. For feeding the input data into Hadoop, what format is it in currently? One simple way would be to have a text file with

Re: Fair Scheduler pools regardless of users

2013-07-08 Thread Sandy Ryza
Hi Amit, You can set the mapred.job.queue.name to the name of the queue you want to submit it to. If the queue does not exist, it will be created. If you want to configure a queue with particular properties, you can do so in the Fair Scheduler allocations file. See http://hadoop.apache.org/docs

Re: whitelist feature of YARN

2013-07-08 Thread Sandy Ryza
Hi Krishna, >From your previous email, it looks like you are using the AMRMClient APIs. Support for whitelisting is not yet supported through them. I am working on this in YARN-521, which should be included in the next release after 2.1.0-beta. If you are submitting ResourceRequests directly to

Re: Containers and CPU

2013-07-02 Thread Sandy Ryza
ontainers per 8-core node? > > John > > ** ** > > ** ** > > *From:* Sandy Ryza [mailto:sandy.r...@cloudera.com] > *Sent:* Tuesday, July 02, 2013 1:26 PM > > *To:* user@hadoop.apache.org > *Subject:* Re: Containers and CPU > > ** ** > > Use

Re: Containers and CPU

2013-07-02 Thread Sandy Ryza
have access to all CPU cores and simply fight it > out in the OS thread scheduler. > > Thanks,**** > > john > > ** ** > > *From:* Sandy Ryza [mailto:sandy.r...@cloudera.com] > *Sent:* Tuesday, July 02, 2013 11:56 AM > *To:* user@hadoop.apache.org > *Subjec

Re: Containers and CPU

2013-07-02 Thread Sandy Ryza
CPU limits are only enforced if cgroups is turned on. With cgroups on, they are only limited when there is contention, in which case tasks are given CPU time in proportion to the number of cores requested for/allocated to them. Does that make sense? -Sandy On Tue, Jul 2, 2013 at 9:50 AM, Chuan

Re: temporary folders for YARN tasks

2013-07-01 Thread Sandy Ryza
LocalDirAllocator should help with this. You can look through MapReduce code to see how it's used. -Sandy On Mon, Jul 1, 2013 at 11:01 PM, Devaraj k wrote: > You can make use of this configuration to do the same. > > ** ** > > > > List of directories to store *localized* files

Re: Yarn job stuck with no application master being assigned

2013-06-21 Thread Sandy Ryza
t; > > On Fri, Jun 21, 2013 at 4:28 PM, Sandy Ryza wrote: > >> Hi Siddhi, >> >> Moving this question to the CDH list. >> >> Does setting yarn.scheduler.capacity.maximum-am-resource-percent to .5 >> help? >> >> Have you tried using the Fair Sched

Re: Yarn job stuck with no application master being assigned

2013-06-21 Thread Sandy Ryza
Hi Siddhi, Moving this question to the CDH list. Does setting yarn.scheduler.capacity.maximum-am-resource-percent to .5 help? Have you tried using the Fair Scheduler? -Sandy On Fri, Jun 21, 2013 at 4:21 PM, Siddhi Mehta wrote: > Hey All, > > I am running a Hadoop 2.0(cdh4.2.1) cluster on a

Re: container allocation

2013-06-14 Thread Sandy Ryza
Hi John, At this time, releasing containers is the preferred way to be strict about your locality requirements. This is not included in a release yet, but https://issues.apache.org/jira/browse/YARN-392 allows expressing hard locality constraints on requests, so you can tell the scheduler to never

Re: Cloudera Mgmt Services - Status unknown health

2013-06-13 Thread Sandy Ryza
Hi Michael, The Cloudera's scm-user list would be a better place for this question. -Sandy On Thu, Jun 13, 2013 at 11:20 AM, Michael Namaiandeh < mnamaian...@healthcit.com> wrote: > I am running a cloudera hadoop cluster and I have noticed that some of my > services are showing a status of “Un

Re: Container size configuration

2013-06-13 Thread Sandy Ryza
Anyway, if I want to manually configure memory size for a particular > reducer, how can I do it? In ApplicationMaster? Or this would not work at > all? > > Thank you. > > > > > > On 6/13/2013 12:47 PM, Sandy Ryza wrote: > > Hi Yuzhang, > > Moving this questio

Re: Container size configuration

2013-06-13 Thread Sandy Ryza
Hi Yuzhang, Moving this question to the Hadoop user list. Are you using MapReduce or writing your own YARN application? In MapReduce, all maps must request the same amount of memory and all reduces must request the same amount of memory. It would be trivial to do this in your own YARN applicati

Re: History server - Yarn

2013-06-07 Thread Sandy Ryza
Hi Rahul, The job history server is currently specific to MapReduce. -Sandy On Fri, Jun 7, 2013 at 8:56 AM, Rahul Bhattacharjee wrote: > Hello, > > I was doing some sort of prototyping on top of YARN. I was able to launch > AM and then AM in turn was able to spawn a few containers and do cert

Re: Why my tests shows Yarn is worse than MRv1 for terasort?

2013-06-06 Thread Sandy Ryza
Hey Sam, Thanks for sharing your results. I'm definitely curious about what's causing the difference. A couple observations: It looks like you've got yarn.nodemanager.resource.memory-mb in there twice with two different values. Your max JVM memory of 1000 MB is (dangerously?) close to the defau

Re: What is the best way to build and debug MR application?

2013-06-06 Thread Sandy Ryza
(and running them from Eclipse) On Thu, Jun 6, 2013 at 7:21 PM, Sandy Ryza wrote: > Hi Lin, > > This is by no means a comprehensive answer to your question, but I've > found that I'm able to iterate fastest by writing unit tests using MRUnit ( > http://mrunit.apache.

Re: What is the best way to build and debug MR application?

2013-06-06 Thread Sandy Ryza
Hi Lin, This is by no means a comprehensive answer to your question, but I've found that I'm able to iterate fastest by writing unit tests using MRUnit ( http://mrunit.apache.org/) -Sandy On Thu, Jun 6, 2013 at 7:02 PM, Lin Yang wrote: > Hi,dear friends, > > I have setup a Hadoop cluster with

Re: built hadoop! please help with next steps?

2013-06-06 Thread Sandy Ryza
-Sandy On Tue, Jun 4, 2013 at 3:22 PM, John Lilley wrote: > Sandy, > > Thanks for the help! I am trying this now. Could you post your complete > yarn-site.xml so I can use it as an example? > > John > > ** ** > > ** ** > > *From:* Sandy

Re: built hadoop! please help with next steps?

2013-05-31 Thread Sandy Ryza
23/developing-testing-and-debugging-hadoop-map-reduce-jobs-with-eclipse > > > looks promising as a Hadoop-in-Eclipse strategy, but it is over a year old > and I’m not sure if it applies to Hadoop 2.0 and YARN. > > John > > ** ** > > *From:* Sandy Ryza [m

Re: built hadoop! please help with next steps?

2013-05-31 Thread Sandy Ryza
Hi John, Here's how I deploy/debug Hadoop locally: To build and tar Hadoop: mvn clean package -Pdist -Dtar -DskipTests=true The tar will be located in the project directory under hadoop-dist/target/. I untar it into my deploy directory. I then copy these scripts into the same directory: had

Re: Shuffle phase replication factor

2013-05-23 Thread Sandy Ryza
In MR1, the tasktracker serves the mapper files (so that tasks don't have to stick around taking up resources). In MR2, the shuffle service, which lives inside the nodemanager, serves them. -Sandy On Thu, May 23, 2013 at 10:22 AM, John Lilley wrote: > Ling, > > Thanks for the response! I

Re: Is there a way to limit # of hadoop tasks per user at runtime?

2013-05-21 Thread Sandy Ryza
Hi Mehmet, Are you using MR1 or MR2? The fair scheduler, present in both versions, but configured slightly differently, allows you to limit the number of map and reduce tasks in a queue. The configuration can be updated at runtime by modifying the scheduler's allocations file. It also has a fea

Re: Installed Hadoop on Linux server - not able to see web UI

2013-05-16 Thread Sandy Ryza
Hi Raj, The web UIs are located on different ports than the RPC ports you specified. If you are using MR1, the HDFS UI is typically located on port 50070, and the MapReduce UI is typically located on port 50030. -Sandy On Thu, May 16, 2013 at 2:58 PM, Raj Hadoop wrote: > Hi, > > I have insta

Re: YARN in 0.23 vs 2.0

2013-05-16 Thread Sandy Ryza
Hi John, You are correct that both 0.23 and 2.0 contain YARN, and that 1.x does not. The (confusing) reason for this is that the 1.x line descends from the 0.20 line, while the 2.0 line descends from the 0.23 line. -Sandy On Thu, May 16, 2013 at 11:46 AM, John Lilley wrote: > We will be p

Re: Map Tasks do not obey data locality principle........

2013-05-15 Thread Sandy Ryza
Hi Nikhil, Which scheduler are you using? -Sandy On Tue, May 14, 2013 at 3:55 AM, Agarwal, Nikhil wrote: > Hi, > > ** ** > > I have a 3-node cluster, with JobTracker running on one machine and > TaskTrackers on other two (say, slave1 and slave2). Instead of using HDFS, > I have written m

Re: Hadoop schedulers!

2013-05-13 Thread Sandy Ryza
Hi Rahul, You're right that the schedulers have evolved to support many of the same features. To your second question, I haven't looked in detail at the FIFO or capacity schedulers, but for the FIFO mode in the fair scheduler, jobs next in the queue will get slots if the first one isn't using the

Re: Why could not find finished jobs in yarn.resourcemanager.webapp.address?

2013-05-02 Thread Sandy Ryza
p- > mapreduce-examples-2.0.3-alpha.jar pi 2 30' > > > 2013/5/3 Sandy Ryza > >> This shouldn't be asked on the dev lists, so putting mapreduce-dev and >> hdfs-dev in the bcc. Have you made sure you're not using the local job >> runner? Did you r

Re: Why could not find finished jobs in yarn.resourcemanager.webapp.address?

2013-05-02 Thread Sandy Ryza
This shouldn't be asked on the dev lists, so putting mapreduce-dev and hdfs-dev in the bcc. Have you made sure you're not using the local job runner? Did you restart the resourcemanager after running the job? -Sandy On Thu, May 2, 2013 at 6:31 PM, sam liu wrote: > Can anyone help this issue?

Re: Automatically mapping a job submitted by a particular user to a specific hadoop map-reduce queue

2013-04-26 Thread Sandy Ryza
eduler >properties]. We want this to be completely transparent to the user of our >cluster. > > The jira above would be a great first step towards such automatic mapping!! > > Cheers, > Sagar > > > On Wed, Apr 24, 2013 at 11:41 PM, Sandy Ryza wrote: > >>

Re: Comparison between JobClient/JobConf and Job/Configuration

2013-04-25 Thread Sandy Ryza
Hi Kevin, JobClient/JobConf is the "old" mapred API and Job/Configuration is the improved "new" API, so in general the latter is preferred. -Sandy On Thu, Apr 25, 2013 at 5:27 AM, Kevin Burton wrote: > I notice that in some beginning texts on starting a Hadoop MapReduce job > sometimes JobClie

Re: Automatically mapping a job submitted by a particular user to a specific hadoop map-reduce queue

2013-04-24 Thread Sandy Ryza
Hi Sagar, This capability currently does not exist in the fair scheduler (or other schedulers, as far as I know), but a JIRA has been filed recently that addresses a similar need. Would https://issues.apache.org/jira/browse/MAPREDUCE-5132 work for what you're trying to do? If not, would you min

Re: YARN - container networking and ports

2013-04-22 Thread Sandy Ryza
The yarn-default.xml file in the Hadoop repository contains the default ports for all of the YARN protocols. -Sandy On Mon, Apr 22, 2013 at 8:27 AM, Marcos Luis Ortiz Valmaseda < marcosluis2...@gmail.com> wrote: > A great overview of MR2, you can find it in the Cloudera´s blog: > http://blog.cl

Re: Article: 'How to Deploy Hadoop 2 (Yarn) on EC2'

2013-04-17 Thread Sandy Ryza
This is great, Keith. On Wed, Apr 17, 2013 at 12:58 PM, Keith Wiley wrote: > I've posted an article on my website that details precisely how to deploy > Hadoop 2.0 with Yarn on AWS (or least how I did it, whether or not such an > approach will translate to others' circumstances). I had been di

Re: YARN - specify hosts in ContainerRequest

2013-04-12 Thread Sandy Ryza
. Huffman < bhuff...@etinternational.com> wrote: > I get a container, but not on the node I'm asking for. > > Thanks, > Brian > > > On 04/12/2013 03:01 PM, Sandy Ryza wrote: > > What do you mean when you say it doesn't seem to use the code? That > y

Re: YARN - specify hosts in ContainerRequest

2013-04-12 Thread Sandy Ryza
What do you mean when you say it doesn't seem to use the code? That you're not getting containers back? -Sandy On Fri, Apr 12, 2013 at 7:44 AM, Brian C. Huffman < bhuff...@etinternational.com> wrote: > Yes, this is exactly the DistributedShell example in the 2.0.3 source > (except for the added

Re: Streaming value of (200MB) from a SequenceFile

2013-03-31 Thread Sandy Ryza
Hi Rahul, I don't think saving the stream for later use would work - I was just suggesting that if only some aggregate statistics needed to be calculated, they could be calculated at read time instead of in the mapper. Nothing requires a Writable to contain all the data that it reads. That's a g

Re: Streaming value of (200MB) from a SequenceFile

2013-03-31 Thread Sandy Ryza
Hi Jerry, I assume you're providing your own Writable implementation? The Writable readFields method is given a stream. Are you able to perform you able to perform your processing while reading the it there? -Sandy On Sat, Mar 30, 2013 at 10:52 AM, Jerry Lam wrote: > Hi everyone, > > I'm havi

Re: Yarn communication between containers

2013-03-27 Thread Sandy Ryza
Hi tmp, YARN doesn't provide an explicit protocol for doing this. Applications are expected to have their own mechanism for communication between task containers, other task containers, and app masters. If you want to see how this is done in MapReduce, I would suggest looking at the TaskUmbilica

Re: Any answer ? Candidate application for map reduce

2013-03-25 Thread Sandy Ryza
Hi Bala, A standard benchmark program for mapreduce is terasort, which is included in the hadoop examples jar. You can generate data for it using teragen, which runs a map-only job: hadoop jar path-to-examples-jar.jar and then sort the data using terasort: hadoop jar path-to-examples-jar.jar

Re: Too many open files error with YARN

2013-03-20 Thread Sandy Ryza
Hi Kishore, 50010 is the datanode port. Does your lsof indicate that the sockets are in CLOSE_WAIT? I had come across an issue like this where that was a symptom. -Sandy On Wed, Mar 20, 2013 at 4:24 AM, Krishna Kishore Bonagiri < write2kish...@gmail.com> wrote: > Hi, > > I am running a date c

Re: Question regarding hadoop jar command usage

2013-03-13 Thread Sandy Ryza
ain what is the difference between running an MR job using the > jar command vs mapred job command (looks like the hadoop job command is > deprecated). > > Thanks > Kay > > > On Wed, Mar 13, 2013 at 10:14 AM, Sandy Ryza wrote: > >> Hi Kay, >> >> The jar is ju

Re: Question regarding hadoop jar command usage

2013-03-13 Thread Sandy Ryza
Hi Kay, The jar is just executed locally. If the jar fires up a mapreduce job and sets itself as the job jar, then mapreduce will handle copying it to the nodes that will use it. -Sandy On Wed, Mar 13, 2013 at 9:01 AM, KayVajj wrote: > I have a question regarding the hadoop jar command. In a

Re: Map reduce technique

2013-03-05 Thread Sandy Ryza
Hi Balachandar, In MapReduce, interpreting input files as key value pairs is accomplished through InputFormats. Some common InputFormats are TextInputFormat, which uses lines in a text file as values and their byte offset into the file as keys, KeyValueTextInputFormat, which interprets the first

Re: Transpose

2013-03-05 Thread Sandy Ryza
Hi, Essentially what you want to do is group your data points by their position in the column, and have each reduce call construct the data for each row into a row. To have each record that the mapper processes be one of the columns, you can use TextInputFormat with conf.set("textinputformat.reco

Re: Accumulo and Mapreduce

2013-03-04 Thread Sandy Ryza
Hi Aji, Oozie is a mature project for managing MapReduce workflows. http://oozie.apache.org/ -Sandy On Mon, Mar 4, 2013 at 8:17 AM, Justin Woody wrote: > Aji, > > Why don't you just chain the jobs together? > http://developer.yahoo.com/hadoop/tutorial/module4.html#chaining > > Justin > > On M

Re: Custom output value for map function

2013-02-27 Thread Sandy Ryza
gt; > out.writeUTF("blabla") > out.writeInt(12) > > the following would be correct > > text = in.readUTF(); > number = in.readInt(); > > and this would fail: > > number = in.readInt(); > text = in.readUTF(); > > ? > > 2013/2/27 Sandy Ryza :

Re: Custom output value for map function

2013-02-27 Thread Sandy Ryza
Hi Paul, To do this, you need to make your Dog class implement Hadoop's Writable interface, so that it can be serialized to and deserialized from bytes. http://hadoop.apache.org/docs/r1.1.1/api/org/apache/hadoop/io/Writable.html The methods you implement would look something like this: public vo

Re: Can I perfrom a MR on my local filesystem

2013-02-18 Thread Sandy Ryza
Hi Nikhil, The jobtracker doesn't do any deployment of other daemons. They are expected to be installed and started on other nodes separately. If I understand your question more broadly, MR doesn't necessarily run its map and reduce tasks on the nodes that contain the data. All data is read fro

Re: Sorting huge text files in Hadoop

2013-02-15 Thread Sandy Ryza
A map-only job does not result in the standard shuffle-sort. Map outputs are written directly to HDFS. -Sandy On Fri, Feb 15, 2013 at 12:23 PM, Jay Vyas wrote: > Maybe im mistaken about what is meant by map-only. Does a map-only job > still result in standard shuffle-sort ? Or does that get

Re: Generic output key class

2013-02-10 Thread Sandy Ryza
Hi Amit, One way to accomplish this would be to create a custom writable implementation, TextOrIntWritable, that has fields for both. It could look something like: class TextOrIntWritable implements Writable { private boolean isText; private Text text; private IntWritable integer; void