Default scheduler yarn

2016-04-29 Thread mierdatutis mi
Hi
Could you say me what default scheduler use yarn?
Thanks and sorry for my English


Guideline on setting Namenode RPC Handler count (client and service)

2016-04-29 Thread Chackravarthy Esakkimuthu
Hi,

Is there any recommendation or guideline on setting no of RPC handlers in
Namenode based on cluster size (no of datanodes)?

Cluster details :

No of datanodes - 1200
NN hardware - 74G heap allocated to NN process, 40 core machine
Total blocks - 80M+
Total Files/Directories - 60M+
Total FSObjects - 150M+

We have isolated service and client RPC by enabling service-rpc.

Currently dfs.namenode.handler.count=400 and
dfs.namenode.service.handler.count=200

Is 200 good fit for this cluster or any change recommended. Please help out.

Thanks in advance!

(We have tried increasing service handler count to 600 and have seen delay
in NN startup time and then it looked quite stable. And setting it to 200
decreases the delay in startup time but it has slightly higher rpcQueueTime
and rpcAvgProcessingTime comparing to 600 handler count.)

Thanks,
Chackra


403 when trying to access secure hadoop http UI /logs/ - any workaround? or explanation?

2016-04-29 Thread Jeffrey Rodriguez
Hi Folks,
  I am getting a 403 accessing Kerberized cluster (Hadoop Kerberized).

kinit . valid Kerberos user...

curl -L  --negotiate -u :  http://locathost:50070/logs/

..
> GET /logs/ HTTP/1.1
> Authorization: Negotiate
YIICVwYJKoZIhvcSAQICAQBuggJGMIICQqADAgEFoQMCAQ6iBwMFAACjggFjYYIBXzCCAVugAwIBBaEJGwdJQk0uQ09NoicwJaADAgEDoR4wHBsESFRUUBsUYmRhdm00ODQuc3ZsLmlibS5jb22jggEeMIIBGqADAgERoQMCAQaiggEMBIIBCGTmcjb1WNFRYaTCzAxgCC9ZMaKdHHyt+7qHV/Q4mRFyuhhouo0hFccjNH7TTC1eUXTf31+zo5Zfg3dNPV/NJ1WH53YdMYWHuHDAkWvd7amBPQB/j5q2pOqn+3X8DEW8hcPYo1vRrzLWht8BKmorxCNuRIDETw0Qn7Q9cETLPgPHbEqTCjeEKNqux/26CaJ8/Ixu6qBbj1DtsJzJZJCKbIVoYbj6hGajv4ACIXTXeIIUa9dqDXeI9R97OZXSVlq/M3foyltPQfjRL3DEWiDdavpmr/3LJbJ6rr3UYeZKona8Wz4SlGWKJwkqSTdBTdpHatVZVRXkTfkeuAi03HNVvZwsJ1v1hPpCaqSBxTCBwqADAgERooG6BIG3jNhBU4niOi+a32hsF5qCAVDne7815PrvvGhweF14u+1nJ2Nk+54eQWUNNIF87AomF0vEoUFjzKtKJ6pAcTer9L9ab782acAhEH0H+O3kW88qc45LGhRtquimF2Xrguq1RrjPIlS1sAoTLtj/b0ctvcFQBH1Vuuryyn5AKyWBvW0IFVzBcJQcLlVjlFoaeA9RpF39BktO3RutCONA4/B/RzbeucEvIhyODss7XBs83o49KemsQT7x
> User-Agent: curl/7.19.7 (x86_64-redhat-linux-gnu) libcurl/7.19.7 NSS/
3.16.2.3 Basic ECC zlib/1.2.3 libidn/1.18 libssh2/1.4.2
> Host: localhost:50070
> Accept: */*
>
< HTTP/1.1 403 User ambari-qa is unauthorized to access this page.
< Content-Type: text/html; charset=iso-8859-1
< Set-Cookie: 
hadoop.auth="u=ambari-qa&p=ambari-qa-tes...@ibm.com&t=kerberos&e=1461979860144&s=oXW3iQyX0/SAWxup9pngeyNSGO4=";
Path=/; Domain=svl.ibm.com; Expires=Sat, 30-Apr-2016 01:31:00 GMT; HttpOnly



id ambari-qa

id ambari-qa
uid=1006(ambari-qa) gid=502(hadoop) groups=502(hadoop),100(users)


All super user/proxy set to *

Any reason why /logs/ are not accessible? Can that be set in configuration?

BTW is I run the request as hdfs user it succeeds so hdfs service user has
authorization.

This is confusing some users since they expect access for hadoop UI /logs/


Re: 403 when trying to access secure hadoop http UI /logs/ - any workaround? or explanation?

2016-04-29 Thread Musty Rehmani
Do you have principal and keytab created for Ambari QA user.  You may want to 
mimic kerberos configuration from hduser and give it a try. 

Sent from Yahoo Mail on Android 
 
  On Fri, Apr 29, 2016 at 11:34 AM, Jeffrey Rodriguez 
wrote:   Hi Folks,
  I am getting a 403 accessing Kerberized cluster (Hadoop Kerberized). 

kinit . valid Kerberos user...

curl -L  --negotiate -u :  http://locathost:50070/logs/

..
> GET /logs/ HTTP/1.1
> Authorization: Negotiate 
> YIICVwYJKoZIhvcSAQICAQBuggJGMIICQqADAgEFoQMCAQ6iBwMFAACjggFjYYIBXzCCAVugAwIBBaEJGwdJQk0uQ09NoicwJaADAgEDoR4wHBsESFRUUBsUYmRhdm00ODQuc3ZsLmlibS5jb22jggEeMIIBGqADAgERoQMCAQaiggEMBIIBCGTmcjb1WNFRYaTCzAxgCC9ZMaKdHHyt+7qHV/Q4mRFyuhhouo0hFccjNH7TTC1eUXTf31+zo5Zfg3dNPV/NJ1WH53YdMYWHuHDAkWvd7amBPQB/j5q2pOqn+3X8DEW8hcPYo1vRrzLWht8BKmorxCNuRIDETw0Qn7Q9cETLPgPHbEqTCjeEKNqux/26CaJ8/Ixu6qBbj1DtsJzJZJCKbIVoYbj6hGajv4ACIXTXeIIUa9dqDXeI9R97OZXSVlq/M3foyltPQfjRL3DEWiDdavpmr/3LJbJ6rr3UYeZKona8Wz4SlGWKJwkqSTdBTdpHatVZVRXkTfkeuAi03HNVvZwsJ1v1hPpCaqSBxTCBwqADAgERooG6BIG3jNhBU4niOi+a32hsF5qCAVDne7815PrvvGhweF14u+1nJ2Nk+54eQWUNNIF87AomF0vEoUFjzKtKJ6pAcTer9L9ab782acAhEH0H+O3kW88qc45LGhRtquimF2Xrguq1RrjPIlS1sAoTLtj/b0ctvcFQBH1Vuuryyn5AKyWBvW0IFVzBcJQcLlVjlFoaeA9RpF39BktO3RutCONA4/B/RzbeucEvIhyODss7XBs83o49KemsQT7x
> User-Agent: curl/7.19.7 (x86_64-redhat-linux-gnu) libcurl/7.19.7 NSS/3.16.2.3 
> Basic ECC zlib/1.2.3 libidn/1.18 libssh2/1.4.2
> Host: localhost:50070
> Accept: */*
> 
< HTTP/1.1 403 User ambari-qa is unauthorized to access this page.
< Content-Type: text/html; charset=iso-8859-1
< Set-Cookie: 
hadoop.auth="u=ambari-qa&p=ambari-qa-tes...@ibm.com&t=kerberos&e=1461979860144&s=oXW3iQyX0/SAWxup9pngeyNSGO4=";
 Path=/; Domain=svl.ibm.com; Expires=Sat, 30-Apr-2016 01:31:00 GMT; HttpOnly



id ambari-qa 

id ambari-qa
uid=1006(ambari-qa) gid=502(hadoop) groups=502(hadoop),100(users)


All super user/proxy set to *

Any reason why /logs/ are not accessible? Can that be set in configuration?

BTW is I run the request as hdfs user it succeeds so hdfs service user has 
authorization.

This is confusing some users since they expect access for hadoop UI /logs/
  


Re: HDFS Federation

2016-04-29 Thread Akira AJISAKA

> Do you know where is the 50070 come form or where to set it?

50070 comes from "dfs.namenode.http-address".
You can set this in hdfs-site.xml.

> If I configure the Federation in one node, I probably will change 
this, right?
No. If you configure NameNode federation, you need to set 
"dfs.nameservices" and "dfs.namenode.http-address.".
Please see 
https://hadoop.apache.org/docs/r2.7.2/hadoop-project-dist/hadoop-hdfs/Federation.html 
for the detail.


-Akira

On 4/28/16 22:48, Kun Ren wrote:

Thanks a lot, Akira.

When the Federation is disabled, I can view the namenode from
http://localhost:50070

Do you know where is the 50070 come form or where to set it? If I
configure the Federation in one node, I probably will change this, right?

Thanks a lot.

On Thu, Apr 28, 2016 at 1:55 AM, Akira AJISAKA
mailto:ajisa...@oss.nttdata.co.jp>> wrote:

Hi Kun,

(1) ViewFileSystem is related.

https://hadoop.apache.org/docs/r2.7.2/hadoop-project-dist/hadoop-hdfs/ViewFs.html

(2) Yes, NameNode Federation is disabled by default.
I suppose it is possible to configure NameNode Federation in one
node by setting different HTTP/RPC port and fsimage/edits directory
for each NameNode. I haven't tried this, so perhaps it is not possible.

Regards,
Akira

On 4/28/16 09:30, Kun Ren wrote:

Hi Genius,

I have two questions about the HDFS Federation:
(1) Since there are multiple namenodes,  there should be some
code that
analysis the client request and transfer  the request to the
appropriate
namenode, could you please point to me where I can find the
related code?

(2) .Also just confirm that the Hadoop 2.7.2 support HDFS
Federation,
but in default there is only 1 namenode, is this correct?
Meanwhile, do
you think it is possible to configure the HDFS Fderation in the
pseudo
distributed mode in one node?

Thanks so much in advance.

Best,
Kun Ren



-
To unsubscribe, e-mail: user-unsubscr...@hadoop.apache.org

For additional commands, e-mail: user-h...@hadoop.apache.org






-
To unsubscribe, e-mail: user-unsubscr...@hadoop.apache.org
For additional commands, e-mail: user-h...@hadoop.apache.org



Re: Default scheduler yarn

2016-04-29 Thread Ray Chiang
The YARN configuration file can be seen in yarn-default.xml
.
The property you're looking for is:

yarn.resourcemanager.scheduler.class

which has the property:

org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler

-Ray


On Fri, Apr 29, 2016 at 12:49 AM, mierdatutis mi  wrote:

> Hi
> Could you say me what default scheduler use yarn?
> Thanks and sorry for my English
>


Re: YARN queues become unusable and jobs are stuck in ACCEPTED state

2016-04-29 Thread Ray Chiang
Just because you have sufficient resources doesn't mean another job should
launch an AM.  You might want to check maxAMShare
and queueMaxAMShareDefault.

Given that you have sufficient resources, you could be running into
YARN-3491.

I don't know whether you have the option, but CDH 5.3.3 is pretty old at
this point.  CDH 5.3.10/5.4.10/5.5.2 have the latest bug fixes.

-Ray

On Thu, Apr 28, 2016 at 12:03 PM, Matt Cheah  wrote:

> Hi,
>
> I¹ve been sporadically seeing an issue when using Hadoop YARN. I¹m using 
> Hadoop 2.5.0, CDH5.3.3.
>
> When I¹ve configured the stack to use the fair scheduler protocol, after some 
> period of time of the cluster being alive and running jobs, I¹m noticing that 
> when I submit a job, the job will be stuck in the ACCEPTED state even though 
> the cluster has sufficient resources to spawn an application master container 
> as well as the queue I¹m submitting to having sufficient resources available. 
> Furthermore, all jobs submitted to that queue will be stuck in the ACCEPTED 
> state. I can unblock job submission by going into the allocation XML file, 
> renaming the queue, and submitting jobs to that renamed queue instead. 
> However the queue has only changed name, and all of its other settings have 
> been preserved.
>
> It is clearly untenable for me to have to change the queues that I¹m using 
> sometimes. This appears to happen irrespective of the settings of the queue, 
> e.g. Its weight or its minimum resource share. The events leading up to this 
> occurrence are strictly unpredictable and I have no concrete way to reproduce 
> the issue. The logs don¹t show anything interesting either; the resource 
> manager just states that it schedules an attempt for the application 
> submitted to the bad queue, but the attempt¹s application master is never 
> allocated to a container anywhere.
>
> I have looked around the YARN bug base and couldn¹t find any similar issues. 
> I¹ve also used jstack to inspect the Resource Manager process, but nothing is 
> obviously wrong there. I was wondering if anyone has encountered a similar 
> issue before. I apologize that the description is vague, but it¹s the best 
> way I can describe it.
>
> Thanks,
>
> -Matt Cheah
>
>
>


Re: S3 Hadoop FileSystems

2016-04-29 Thread Chris Nauroth
Hello Elliot,

The current state of support for the various S3 file system implementations 
within the Apache Hadoop community can be summed up as follows:

s3: Soon to be deprecated, not actively maintained, appears to not work 
reliably at all in recent versions.
s3n: Not yet on its way to deprecation, but also not actively maintained.
s3a: This is seen as the direction forward for S3 integration, so this is where 
Hadoop contributors are currently focusing their energy.

Regarding interoperability with EMR, I can't speak from any of my own 
experience on how to achieve this.  We know that EMR runs custom code different 
from what you'll see in the Apache repos.  I think that creates a risk for 
interop.  My only suggestion would be to experiment and make sure to test any 
of your interop scenarios end-to-end very thoroughly.

As you noticed, s3n no longer has a 5 GB limitation.  Issue HADOOP-9454 
introduced support for files larger than 5 GB by using multi-part upload.  This 
patch was released in Apache Hadoop 2.4.0.

Regarding lack of MD5 verification in s3a, I believe that is just an oversight, 
not an intentional design choice.  I filed HADOOP-13076 to track adding this 
feature in s3a.

--Chris Nauroth

From: Elliot West mailto:tea...@gmail.com>>
Date: Thursday, April 28, 2016 at 5:01 AM
To: "user@hadoop.apache.org" 
mailto:user@hadoop.apache.org>>
Subject: S3 Hadoop FileSystems

Hello,

I'm working on a project that moves data from HDFS file systems into S3 for 
analysis with Hive on EMR. Recently I've become quite confused with the state 
of play regarding the different FileSystems: s3, s3n, and s3a. For my use case 
I require the following:

  *   Support for the transfer of very large files.
  *   MD5 checks on copy operations to provide data verification.
  *   Excellent compatibility within an EMR/Hive environment.

To move data between clusters it would seem that current versions of the 
NativeS3FileSystem are my best bet; It appears that only s3n provides MD5 
checking.
 It is often cited that s3n does not support files over 5GB but I can find no 
indication of such a limitation in the source code, in fact I see that it 
switches over to multi-part upload for larger 
files.
 So, has this limitation been removed in s3n?

Within EMR Amazon appear to recommend s3, support s3n, and advise against 
s3a.
 So yet again s3n would appear to win out here too? I assume that the s3n 
implementation available in EMR is different to that in Apache Hadoop? I find 
it hard to imagine that AWS would use JetS3t instead of their own AWS Java 
client, but perhaps they do?

Finally, could I use NativeS3FileSystem to perform the actual transfer on my 
Apache Hadoop cluster but then rewrite the table locations in my EMR Hive 
metastore to use the s3:// protocol prefix? Could that work?

I'd appreciate any light that can be shed on these questions, and any advice 
regarding my reasoning behind my proposal to use s3n for this particular use 
case.

Thanks,

Elliot.




Re: [HDFS-inotify] "IOException: The client is stopped" after reading file

2016-04-29 Thread Chris Nauroth
Hello Cazen,

This looks to me like this is suffering from an unintended side effect of 
closing the FileSystem object.  Hadoop internally caches instances of the 
FileSystem class, and the same instance can be returned to multiple call sites. 
 Even after one call site closes it, it's possible that other call sites still 
hold a reference to that same FileSystem instance.  Closing the FileSystem 
instance makes it unusable.

HdfsAdmin#getInotifyEventStream is likely using the same FileSystem instance 
that your own FileSystem.get call returns.  By closing it (using 
try-with-resources), that FileSystem instance is made invalid for the 
subsequent calls to retrieve inotify events.

The FileSystem cache is a fairly common source of confusion.  However, its 
current behavior is considered by design.  For reasons of 
backwards-incompatibility, we can't easily change its behavior to help with 
confusing situations like this.  (Sorry!)

A few suggestions to try:

1. Just don't close the FileSystem.  Even if you don't close it explicitly, it 
will be closed at process teardown via a shutdown hook.  This definitely looks 
wrong from a resource management perspective, but a lot of applications work 
this way.

2. Call FileSystem#newInstance instead of FileSystem#get.  The newInstance 
method is guaranteed to return an instance unique to that call site, not a 
shared instance potentially in use by other call sites.  If you use 
newInstance, then you must guarantee it gets closed to avoid a leak with a 
long-term impact.

3. You can disable the FileSystem cache for specific file system types by 
editing core-site.xml and setting property fs..impl.disable.cache to true, e.g. fs.hdfs.impl.disable.cache.  In general, 
disabling the cache is not desirable, because the performance benefits of the 
cache are noticeable.  Sometimes this is a helpful workaround for specific 
applications though.

--Chris Nauroth

From: Cazen Lee mailto:cazen@gmail.com>>
Date: Thursday, April 28, 2016 at 5:53 PM
To: "user@hadoop.apache.org" 
mailto:user@hadoop.apache.org>>
Subject: [HDFS-inotify] "IOException: The client is stopped" after reading file


Good day this is Cazen
Could I kindly ask about something weird situation when reading file in hdfs 
with inotify polling

- Env : MacOS, EMR, Linux(standalone) - same problem
- Version : Hadoop 2.7.2

1. I would like to write down a code that read file under particular location 
when it created(with using inotify)
So I modify sample code based on "hdfs-inotify-example" in github
https://github.com/onefoursix/hdfs-inotify-example/blob/master/src/main/java/com/onefoursix/HdfsINotifyExample.java

2. I've changed code with read and print line to console when it renamed
https://github.com/onefoursix/hdfs-inotify-example/commit/82485881c5da85a46dd1741c2d8420c7c4e81f93

case RENAME:
Event.RenameEvent renameEvent = (Event.RenameEvent) event;
Configuration conf = new Configuration();
conf.set("fs.defaultFS", defaultFS);
System.out.println(renameEvent.getDstPath() + " " + inputPath.getPath());
if (renameEvent.getDstPath().startsWith(inputPath.getPath())) {
//Try to read file
try (FileSystem fs = FileSystem.get(conf)) {
Path filePath = new Path(defaultFS + renameEvent.getDstPath());
BufferedReader br = new BufferedReader(new 
InputStreamReader(fs.open(filePath)));
String line;
line = br.readLine();
while (line != null) {
System.out.println(line);
line = br.readLine();
}
br.close();
}
}

3. It works. But I encountered IOException in next eventStream.take() after 
file read. It doesn't happen if I do not read file on hdfs.
-CODE-
DFSInotifyEventInputStream eventStream = admin.getInotifyEventStream();
EventBatch batch = eventStream.take();

-LOG-
Cazens-MacBook-Pro:hdfs-inotify-example Cazen$ java -jar 
target/hdfs-inotify-example-uber.jar hdfs://localhost:8032/cazen/
lastReadTxid = 0
log4j:WARN No appenders could be found for logger 
(org.apache.hadoop.util.Shell).
log4j:WARN Please initialize the log4j system properly.
log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig for more 
info.
TxId = 3134
event type = CREATE
  path = /cazen/test2.txt._COPYING_
  owner = Cazen
  ctime = 1461850245559
TxId = 3138
event type = CLOSE
TxId = 3139
event type = RENAME
/cazen/test2.txt /cazen/
File Start
Input File Text Sample LOL
File END
Exception in thread "main" java.io.IOException: The client is stopped
at org.apache.hadoop.ipc.Client.getConnection(Client.java:1507)
at org.apache.hadoop.ipc.Client.call(Client.java:1451)
at org.apache.hadoop.ipc.Client.call(Client.java:1412)
at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
at com.sun.proxy.$Proxy9.getEditsFromTxid(Unknown Sou