[jira] [Assigned] (HADOOP-10517) InputStream is not closed in two methods of JarFinder

2014-05-04 Thread Ted Yu (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-10517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ted Yu reassigned HADOOP-10517:
---

Assignee: Ted Yu

 InputStream is not closed in two methods of JarFinder
 -

 Key: HADOOP-10517
 URL: https://issues.apache.org/jira/browse/HADOOP-10517
 Project: Hadoop Common
  Issue Type: Bug
Reporter: Ted Yu
Assignee: Ted Yu
Priority: Minor
 Attachments: HADOOP-10517.1.patch.txt, hadoop-10517-v1.txt, 
 hadoop-10517-v2.txt


 JarFinder#jarDir() and JarFinder#zipDir() have such code:
 {code}
  InputStream is = new FileInputStream(f);
  copyToZipStream(is, anEntry, zos);
 {code}
 The InputStream is closed in copyToZipStream() but should be enclosed in 
 finally block.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (HADOOP-10517) InputStream is not closed in two methods of JarFinder

2014-05-04 Thread Ted Yu (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-10517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ted Yu updated HADOOP-10517:


Attachment: hadoop-10517-v2.txt

Patch v2 addresses Swarnim's comment.

 InputStream is not closed in two methods of JarFinder
 -

 Key: HADOOP-10517
 URL: https://issues.apache.org/jira/browse/HADOOP-10517
 Project: Hadoop Common
  Issue Type: Bug
Reporter: Ted Yu
Priority: Minor
 Attachments: HADOOP-10517.1.patch.txt, hadoop-10517-v1.txt, 
 hadoop-10517-v2.txt


 JarFinder#jarDir() and JarFinder#zipDir() have such code:
 {code}
  InputStream is = new FileInputStream(f);
  copyToZipStream(is, anEntry, zos);
 {code}
 The InputStream is closed in copyToZipStream() but should be enclosed in 
 finally block.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (HADOOP-10448) Support pluggable mechanism to specify proxy user settings

2014-05-04 Thread Benoy Antony (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-10448?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Benoy Antony updated HADOOP-10448:
--

Attachment: (was: HADOOP-10448.patch)

 Support pluggable mechanism to specify proxy user settings
 --

 Key: HADOOP-10448
 URL: https://issues.apache.org/jira/browse/HADOOP-10448
 Project: Hadoop Common
  Issue Type: Sub-task
  Components: security
Affects Versions: 2.3.0
Reporter: Benoy Antony
Assignee: Benoy Antony
 Attachments: HADOOP-10448.patch, HADOOP-10448.patch, 
 HADOOP-10448.patch, HADOOP-10448.patch, HADOOP-10448.patch, 
 HADOOP-10448.patch, HADOOP-10448.patch, HADOOP-10448.patch, HADOOP-10448.patch


 We have a requirement to support large number of superusers. (users who 
 impersonate as another user) 
 (http://hadoop.apache.org/docs/r1.2.1/Secure_Impersonation.html) 
 Currently each  superuser needs to be defined in the core-site.xml via 
 proxyuser settings. This will be cumbersome when there are 1000 entries.
 It seems useful to have a pluggable mechanism to specify  proxy user settings 
 with the current approach as the default. 



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (HADOOP-10448) Support pluggable mechanism to specify proxy user settings

2014-05-04 Thread Benoy Antony (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-10448?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Benoy Antony updated HADOOP-10448:
--

Attachment: HADOOP-10448.patch

Resubmitting the patch to trigger a build

 Support pluggable mechanism to specify proxy user settings
 --

 Key: HADOOP-10448
 URL: https://issues.apache.org/jira/browse/HADOOP-10448
 Project: Hadoop Common
  Issue Type: Sub-task
  Components: security
Affects Versions: 2.3.0
Reporter: Benoy Antony
Assignee: Benoy Antony
 Attachments: HADOOP-10448.patch, HADOOP-10448.patch, 
 HADOOP-10448.patch, HADOOP-10448.patch, HADOOP-10448.patch, 
 HADOOP-10448.patch, HADOOP-10448.patch, HADOOP-10448.patch, HADOOP-10448.patch


 We have a requirement to support large number of superusers. (users who 
 impersonate as another user) 
 (http://hadoop.apache.org/docs/r1.2.1/Secure_Impersonation.html) 
 Currently each  superuser needs to be defined in the core-site.xml via 
 proxyuser settings. This will be cumbersome when there are 1000 entries.
 It seems useful to have a pluggable mechanism to specify  proxy user settings 
 with the current approach as the default. 



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (HADOOP-10433) Key Management Server based on KeyProvider API

2014-05-04 Thread Alejandro Abdelnur (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10433?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13989086#comment-13989086
 ] 

Alejandro Abdelnur commented on HADOOP-10433:
-

planning to commit on monday

 Key Management Server based on KeyProvider API
 --

 Key: HADOOP-10433
 URL: https://issues.apache.org/jira/browse/HADOOP-10433
 Project: Hadoop Common
  Issue Type: Improvement
  Components: security
Affects Versions: 3.0.0
Reporter: Alejandro Abdelnur
Assignee: Alejandro Abdelnur
 Attachments: HADOOP-10433.patch, HADOOP-10433.patch, 
 HADOOP-10433.patch, HADOOP-10433.patch, HADOOP-10433.patch, 
 HADOOP-10433.patch, HADOOP-10433.patch, HADOOP-10433.patch, 
 HADOOP-10433.patch, HADOOP-10433.patch, HADOOP-10433.patch, 
 HADOOP-10433.patch, HadoopKMSDocsv2.pdf, KMS-doc.pdf


 (from HDFS-6134 proposal)
 Hadoop KMS is the gateway, for Hadoop and Hadoop clients, to the underlying 
 KMS. It provides an interface that works with existing Hadoop security 
 components (authenticatication, confidentiality).
 Hadoop KMS will be implemented leveraging the work being done in HADOOP-10141 
 and HADOOP-10177.
 Hadoop KMS will provide an additional implementation of the Hadoop 
 KeyProvider class. This implementation will be a client-server implementation.
 The client-server protocol will be secure:
 * Kerberos HTTP SPNEGO (authentication)
 * HTTPS for transport (confidentiality and integrity)
 * Hadoop ACLs (authorization)
 The Hadoop KMS implementation will not provide additional ACL to access 
 encrypted files. For sophisticated access control requirements, HDFS ACLs 
 (HDFS-4685) should be used.
 Basic key administration will be supported by the Hadoop KMS via the, already 
 available, Hadoop KeyShell command line tool
 There are minor changes that must be done in Hadoop KeyProvider functionality:
 The KeyProvider contract, and the existing implementations, must be 
 thread-safe
 KeyProvider API should have an API to generate the key material internally
 JavaKeyStoreProvider should use, if present, a password provided via 
 configuration
 KeyProvider Option and Metadata should include a label (for easier 
 cross-referencing)
 To avoid overloading the underlying KeyProvider implementation, the Hadoop 
 KMS will cache keys using a TTL policy.
 Scalability and High Availability of the Hadoop KMS can achieved by running 
 multiple instances behind a VIP/Load-Balancer. For High Availability, the 
 underlying KeyProvider implementation used by the Hadoop KMS must be High 
 Available.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (HADOOP-10571) Use Log.*(Object, Throwable) overload to log exceptions

2014-05-04 Thread Steve Loughran (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10571?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13989110#comment-13989110
 ] 

Steve Loughran commented on HADOOP-10571:
-

Arpit, we had a discussion on common-dev about actually switching to SLF4J -its 
on the classpath. If we are going to rework exceptions it lets us

# varags
# skip the {{is*Enabled}} checks -at least on info err and warn
# rely on dynamic string expansion with handling for nulls and first element of 
arrays
# let you include the text of an exception in the short message, as well as its 
stack (it looks for a throwable as the last arg and prints the stack trace.
# still compatible at source level with commons-logging

together this would let you do
{code}
log.info(exception connecting to {} with timeout :{},host, time, e). 
{code}

 Use Log.*(Object, Throwable) overload to log exceptions
 ---

 Key: HADOOP-10571
 URL: https://issues.apache.org/jira/browse/HADOOP-10571
 Project: Hadoop Common
  Issue Type: Bug
Affects Versions: 2.4.0
Reporter: Arpit Agarwal
Assignee: Arpit Agarwal
 Attachments: HADOOP-10571.01.patch


 When logging an exception, we often convert the exception to string or call 
 {{.getMessage}}. Instead we can use the log method overloads which take 
 {{Throwable}} as a parameter.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (HADOOP-10562) Namenode exits on exception without printing stack trace in AbstractDelegationTokenSecretManager

2014-05-04 Thread Hudson (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10562?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13989115#comment-13989115
 ] 

Hudson commented on HADOOP-10562:
-

SUCCESS: Integrated in Hadoop-trunk-Commit #5591 (See 
[https://builds.apache.org/job/Hadoop-trunk-Commit/5591/])
HADOOP-10562. Namenode exits on exception without printing stack trace in 
AbstractDelegationTokenSecretManager. (Contributed by Suresh Srinivas) (arp: 
http://svn.apache.org/viewcvs.cgi/?root=Apache-SVNview=revrev=1592002)
* /hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
* 
/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java


 Namenode exits on exception without printing stack trace in 
 AbstractDelegationTokenSecretManager
 

 Key: HADOOP-10562
 URL: https://issues.apache.org/jira/browse/HADOOP-10562
 Project: Hadoop Common
  Issue Type: Bug
Affects Versions: 1.2.1, 2.4.0
Reporter: Suresh Srinivas
Assignee: Suresh Srinivas
Priority: Critical
 Fix For: 3.0.0, 2.5.0

 Attachments: HADOOP-10562.1.patch, HADOOP-10562.branch-1.1.patch, 
 HADOOP-10562.patch


 Not printing the stack trace makes debugging harder.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (HADOOP-10549) MAX_SUBST and varPat should be final in Configuration.java

2014-05-04 Thread Hudson (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13989117#comment-13989117
 ] 

Hudson commented on HADOOP-10549:
-

SUCCESS: Integrated in Hadoop-trunk-Commit #5591 (See 
[https://builds.apache.org/job/Hadoop-trunk-Commit/5591/])
HADOOP-10549. MAX_SUBST and varPat should be final in Configuration.java. 
Contributed by Gera Shegalov. (cnauroth: 
http://svn.apache.org/viewcvs.cgi/?root=Apache-SVNview=revrev=1592011)
* /hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
* 
/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java


 MAX_SUBST and varPat should be final in Configuration.java
 --

 Key: HADOOP-10549
 URL: https://issues.apache.org/jira/browse/HADOOP-10549
 Project: Hadoop Common
  Issue Type: Improvement
  Components: conf
Affects Versions: 2.4.0
Reporter: Gera Shegalov
Assignee: Gera Shegalov
 Fix For: 3.0.0, 2.5.0

 Attachments: HADOOP-10549.v01.patch


 In Configuration, expansion of variables is handled using the following 
 constants that are not declared final:
 {code}
   private static Pattern varPat = 
 Pattern.compile(\\$\\{[^\\}\\$\u0020]+\\});
   private static int MAX_SUBST = 20;
 {code}



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (HADOOP-10568) Add s3 server-side encryption

2014-05-04 Thread Hudson (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10568?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13989120#comment-13989120
 ] 

Hudson commented on HADOOP-10568:
-

SUCCESS: Integrated in Hadoop-trunk-Commit #5591 (See 
[https://builds.apache.org/job/Hadoop-trunk-Commit/5591/])
HADOOP-10568. Add s3 server-side encryption. Contributed by David S. Wang. 
(atm: http://svn.apache.org/viewcvs.cgi/?root=Apache-SVNview=revrev=1592133)
* /hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
* 
/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/s3native/Jets3tNativeFileSystemStore.java
* 
/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml


 Add s3 server-side encryption
 -

 Key: HADOOP-10568
 URL: https://issues.apache.org/jira/browse/HADOOP-10568
 Project: Hadoop Common
  Issue Type: Bug
  Components: fs/s3
Affects Versions: 2.4.0
Reporter: David S. Wang
Assignee: David S. Wang
  Labels: s3
 Fix For: 2.5.0

 Attachments: HADOOP-10568-2.patch, HADOOP-10568.patch


 Add s3 server-side encryption as described here:
 http://docs.aws.amazon.com/AmazonS3/latest/dev/SSEUsingJavaSDK.html



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (HADOOP-10563) Remove the dependency of jsp in trunk

2014-05-04 Thread Hudson (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10563?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13989116#comment-13989116
 ] 

Hudson commented on HADOOP-10563:
-

SUCCESS: Integrated in Hadoop-trunk-Commit #5591 (See 
[https://builds.apache.org/job/Hadoop-trunk-Commit/5591/])
HADOOP-10563. Remove the dependency of jsp in trunk. Contributed by Haohui Mai. 
(wheat9: 
http://svn.apache.org/viewcvs.cgi/?root=Apache-SVNview=revrev=1592069)
* /hadoop/common/trunk/hadoop-client/pom.xml
* /hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
* /hadoop/common/trunk/hadoop-common-project/hadoop-common/pom.xml
* 
/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
* 
/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/resources/webapps/test/testjsp.jsp
* /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
* /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-nfs/pom.xml
* /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/pom.xml
* 
/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
* 
/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/proto-datanode-web.xml
* 
/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/proto-hdfs-web.xml
* 
/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/proto-journal-web.xml
* 
/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/proto-secondary-web.xml
* 
/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/proto-web.xml
* 
/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestJspHelper.java
* /hadoop/common/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
* /hadoop/common/trunk/hadoop-mapreduce-project/pom.xml
* /hadoop/common/trunk/hadoop-project/pom.xml
* /hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/pom.xml
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/pom.xml
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml
* 
/hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/pom.xml


 Remove the dependency of jsp in trunk
 -

 Key: HADOOP-10563
 URL: https://issues.apache.org/jira/browse/HADOOP-10563
 Project: Hadoop Common
  Issue Type: Improvement
Reporter: Haohui Mai
Assignee: Haohui Mai
 Fix For: 3.0.0

 Attachments: HADOOP-10563.000.patch, HADOOP-10563.001.patch, 
 HADOOP-10563.002.patch


 After HDFS-6252 neither hdfs nor yarn uses jsp, thus the dependency of the 
 jsp can be removed from the pom.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (HADOOP-10541) InputStream in MiniKdc#initKDCServer for minikdc.ldiff is not closed

2014-05-04 Thread Hadoop QA (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10541?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13989130#comment-13989130
 ] 

Hadoop QA commented on HADOOP-10541:


{color:red}-1 overall{color}.  Here are the results of testing the latest 
attachment 
  
http://issues.apache.org/jira/secure/attachment/12643251/HADOOP-10541.2.patch.txt
  against trunk revision .

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:red}-1 tests included{color}.  The patch doesn't appear to include 
any new or modified tests.
Please justify why no new tests are needed for this 
patch.
Also please list what manual steps were performed to 
verify this patch.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:green}+1 javadoc{color}.  There were no new javadoc warning messages.

{color:green}+1 eclipse:eclipse{color}.  The patch built with 
eclipse:eclipse.

{color:green}+1 findbugs{color}.  The patch does not introduce any new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:green}+1 core tests{color}.  The patch passed unit tests in 
hadoop-common-project/hadoop-minikdc.

{color:green}+1 contrib tests{color}.  The patch passed contrib unit tests.

Test results: 
https://builds.apache.org/job/PreCommit-HADOOP-Build/3907//testReport/
Console output: 
https://builds.apache.org/job/PreCommit-HADOOP-Build/3907//console

This message is automatically generated.

 InputStream in MiniKdc#initKDCServer for minikdc.ldiff is not closed
 

 Key: HADOOP-10541
 URL: https://issues.apache.org/jira/browse/HADOOP-10541
 Project: Hadoop Common
  Issue Type: Bug
  Components: test
Affects Versions: 3.0.0, 2.4.0
Reporter: Ted Yu
Assignee: Swarnim Kulkarni
Priority: Minor
 Attachments: HADOOP-10541.1.patch.txt, HADOOP-10541.2.patch.txt


 The same InputStream variable is used for minikdc.ldiff and minikdc-krb5.conf 
 :
 {code}
 InputStream is = cl.getResourceAsStream(minikdc.ldiff);
 ...
 is = cl.getResourceAsStream(minikdc-krb5.conf);
 {code}
 Before the second assignment, is should be closed.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (HADOOP-10572) Example NFS mount command must pass noacl as it isn't supported by the server yet

2014-05-04 Thread Hadoop QA (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10572?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13989199#comment-13989199
 ] 

Hadoop QA commented on HADOOP-10572:


{color:red}-1 overall{color}.  Here are the results of testing the latest 
attachment 
  http://issues.apache.org/jira/secure/attachment/12643188/HADOOP-10572.patch
  against trunk revision .

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:red}-1 tests included{color}.  The patch doesn't appear to include 
any new or modified tests.
Please justify why no new tests are needed for this 
patch.
Also please list what manual steps were performed to 
verify this patch.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:green}+1 javadoc{color}.  There were no new javadoc warning messages.

{color:green}+1 eclipse:eclipse{color}.  The patch built with 
eclipse:eclipse.

{color:green}+1 findbugs{color}.  The patch does not introduce any new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:green}+1 core tests{color}.  The patch passed unit tests in 
hadoop-hdfs-project/hadoop-hdfs.

{color:green}+1 contrib tests{color}.  The patch passed contrib unit tests.

Test results: 
https://builds.apache.org/job/PreCommit-HADOOP-Build/3909//testReport/
Console output: 
https://builds.apache.org/job/PreCommit-HADOOP-Build/3909//console

This message is automatically generated.

 Example NFS mount command must pass noacl as it isn't supported by the server 
 yet
 -

 Key: HADOOP-10572
 URL: https://issues.apache.org/jira/browse/HADOOP-10572
 Project: Hadoop Common
  Issue Type: Improvement
  Components: nfs
Affects Versions: 2.4.0
Reporter: Harsh J
Assignee: Harsh J
Priority: Trivial
 Attachments: HADOOP-10572.patch


 Use of the documented default mount command results in the below server side 
 log WARN event, cause the client tries to locate the ACL program (#100227):
 {code}
 12:26:11.975 AM   TRACE   org.apache.hadoop.oncrpc.RpcCall
 Xid:-1114380537, messageType:RPC_CALL, rpcVersion:2, program:100227, 
 version:3, procedure:0, credential:(AuthFlavor:AUTH_NONE), 
 verifier:(AuthFlavor:AUTH_NONE)
 12:26:11.976 AM   TRACE   org.apache.hadoop.oncrpc.RpcProgram 
 NFS3 procedure #0
 12:26:11.976 AM   WARNorg.apache.hadoop.oncrpc.RpcProgram 
 Invalid RPC call program 100227
 {code}
 The client mount command must pass {{noacl}} to avoid this.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (HADOOP-10448) Support pluggable mechanism to specify proxy user settings

2014-05-04 Thread Hadoop QA (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10448?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13989200#comment-13989200
 ] 

Hadoop QA commented on HADOOP-10448:


{color:red}-1 overall{color}.  Here are the results of testing the latest 
attachment 
  http://issues.apache.org/jira/secure/attachment/12643282/HADOOP-10448.patch
  against trunk revision .

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:green}+1 tests included{color}.  The patch appears to include 11 new 
or modified test files.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:green}+1 javadoc{color}.  There were no new javadoc warning messages.

{color:green}+1 eclipse:eclipse{color}.  The patch built with 
eclipse:eclipse.

{color:green}+1 findbugs{color}.  The patch does not introduce any new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:red}-1 core tests{color}.  The patch failed these unit tests in 
hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs 
hadoop-hdfs-project/hadoop-hdfs-nfs 
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager:

  
org.apache.hadoop.yarn.server.resourcemanager.TestClientRMService
  
org.apache.hadoop.hdfs.server.balancer.TestBalancerWithNodeGroup

{color:green}+1 contrib tests{color}.  The patch passed contrib unit tests.

Test results: 
https://builds.apache.org/job/PreCommit-HADOOP-Build/3908//testReport/
Console output: 
https://builds.apache.org/job/PreCommit-HADOOP-Build/3908//console

This message is automatically generated.

 Support pluggable mechanism to specify proxy user settings
 --

 Key: HADOOP-10448
 URL: https://issues.apache.org/jira/browse/HADOOP-10448
 Project: Hadoop Common
  Issue Type: Sub-task
  Components: security
Affects Versions: 2.3.0
Reporter: Benoy Antony
Assignee: Benoy Antony
 Attachments: HADOOP-10448.patch, HADOOP-10448.patch, 
 HADOOP-10448.patch, HADOOP-10448.patch, HADOOP-10448.patch, 
 HADOOP-10448.patch, HADOOP-10448.patch, HADOOP-10448.patch, HADOOP-10448.patch


 We have a requirement to support large number of superusers. (users who 
 impersonate as another user) 
 (http://hadoop.apache.org/docs/r1.2.1/Secure_Impersonation.html) 
 Currently each  superuser needs to be defined in the core-site.xml via 
 proxyuser settings. This will be cumbersome when there are 1000 entries.
 It seems useful to have a pluggable mechanism to specify  proxy user settings 
 with the current approach as the default. 



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (HADOOP-10517) InputStream is not closed in two methods of JarFinder

2014-05-04 Thread Hadoop QA (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10517?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13989208#comment-13989208
 ] 

Hadoop QA commented on HADOOP-10517:


{color:red}-1 overall{color}.  Here are the results of testing the latest 
attachment 
  http://issues.apache.org/jira/secure/attachment/12643270/hadoop-10517-v2.txt
  against trunk revision .

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:green}+1 tests included{color}.  The patch appears to include 1 new 
or modified test files.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:green}+1 javadoc{color}.  There were no new javadoc warning messages.

{color:green}+1 eclipse:eclipse{color}.  The patch built with 
eclipse:eclipse.

{color:green}+1 findbugs{color}.  The patch does not introduce any new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:red}-1 core tests{color}.  The patch failed these unit tests in 
hadoop-common-project/hadoop-common:

  org.apache.hadoop.util.TestJarFinder

{color:green}+1 contrib tests{color}.  The patch passed contrib unit tests.

Test results: 
https://builds.apache.org/job/PreCommit-HADOOP-Build/3910//testReport/
Console output: 
https://builds.apache.org/job/PreCommit-HADOOP-Build/3910//console

This message is automatically generated.

 InputStream is not closed in two methods of JarFinder
 -

 Key: HADOOP-10517
 URL: https://issues.apache.org/jira/browse/HADOOP-10517
 Project: Hadoop Common
  Issue Type: Bug
Reporter: Ted Yu
Assignee: Ted Yu
Priority: Minor
 Attachments: HADOOP-10517.1.patch.txt, hadoop-10517-v1.txt, 
 hadoop-10517-v2.txt


 JarFinder#jarDir() and JarFinder#zipDir() have such code:
 {code}
  InputStream is = new FileInputStream(f);
  copyToZipStream(is, anEntry, zos);
 {code}
 The InputStream is closed in copyToZipStream() but should be enclosed in 
 finally block.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (HADOOP-10541) InputStream in MiniKdc#initKDCServer for minikdc.ldiff is not closed

2014-05-04 Thread Swarnim Kulkarni (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-10541?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Swarnim Kulkarni updated HADOOP-10541:
--

Attachment: HADOOP-10541.3.patch.txt

New patch attached with minor cleanup.

 InputStream in MiniKdc#initKDCServer for minikdc.ldiff is not closed
 

 Key: HADOOP-10541
 URL: https://issues.apache.org/jira/browse/HADOOP-10541
 Project: Hadoop Common
  Issue Type: Bug
  Components: test
Affects Versions: 3.0.0, 2.4.0
Reporter: Ted Yu
Assignee: Swarnim Kulkarni
Priority: Minor
 Attachments: HADOOP-10541.1.patch.txt, HADOOP-10541.2.patch.txt, 
 HADOOP-10541.3.patch.txt


 The same InputStream variable is used for minikdc.ldiff and minikdc-krb5.conf 
 :
 {code}
 InputStream is = cl.getResourceAsStream(minikdc.ldiff);
 ...
 is = cl.getResourceAsStream(minikdc-krb5.conf);
 {code}
 Before the second assignment, is should be closed.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (HADOOP-10517) InputStream is not closed in two methods of JarFinder

2014-05-04 Thread Ted Yu (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10517?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13989265#comment-13989265
 ] 

Ted Yu commented on HADOOP-10517:
-

Patch v2 doesn't work while v1 works.

 InputStream is not closed in two methods of JarFinder
 -

 Key: HADOOP-10517
 URL: https://issues.apache.org/jira/browse/HADOOP-10517
 Project: Hadoop Common
  Issue Type: Bug
Reporter: Ted Yu
Assignee: Ted Yu
Priority: Minor
 Attachments: HADOOP-10517.1.patch.txt, hadoop-10517-v1.txt, 
 hadoop-10517-v2.txt


 JarFinder#jarDir() and JarFinder#zipDir() have such code:
 {code}
  InputStream is = new FileInputStream(f);
  copyToZipStream(is, anEntry, zos);
 {code}
 The InputStream is closed in copyToZipStream() but should be enclosed in 
 finally block.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (HADOOP-10541) InputStream in MiniKdc#initKDCServer for minikdc.ldiff is not closed

2014-05-04 Thread Hadoop QA (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10541?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13989270#comment-13989270
 ] 

Hadoop QA commented on HADOOP-10541:


{color:red}-1 overall{color}.  Here are the results of testing the latest 
attachment 
  
http://issues.apache.org/jira/secure/attachment/12643318/HADOOP-10541.3.patch.txt
  against trunk revision .

{color:green}+1 @author{color}.  The patch does not contain any @author 
tags.

{color:red}-1 tests included{color}.  The patch doesn't appear to include 
any new or modified tests.
Please justify why no new tests are needed for this 
patch.
Also please list what manual steps were performed to 
verify this patch.

{color:green}+1 javac{color}.  The applied patch does not increase the 
total number of javac compiler warnings.

{color:green}+1 javadoc{color}.  There were no new javadoc warning messages.

{color:green}+1 eclipse:eclipse{color}.  The patch built with 
eclipse:eclipse.

{color:green}+1 findbugs{color}.  The patch does not introduce any new 
Findbugs (version 1.3.9) warnings.

{color:green}+1 release audit{color}.  The applied patch does not increase 
the total number of release audit warnings.

{color:green}+1 core tests{color}.  The patch passed unit tests in 
hadoop-common-project/hadoop-minikdc.

{color:green}+1 contrib tests{color}.  The patch passed contrib unit tests.

Test results: 
https://builds.apache.org/job/PreCommit-HADOOP-Build/3911//testReport/
Console output: 
https://builds.apache.org/job/PreCommit-HADOOP-Build/3911//console

This message is automatically generated.

 InputStream in MiniKdc#initKDCServer for minikdc.ldiff is not closed
 

 Key: HADOOP-10541
 URL: https://issues.apache.org/jira/browse/HADOOP-10541
 Project: Hadoop Common
  Issue Type: Bug
  Components: test
Affects Versions: 3.0.0, 2.4.0
Reporter: Ted Yu
Assignee: Swarnim Kulkarni
Priority: Minor
 Attachments: HADOOP-10541.1.patch.txt, HADOOP-10541.2.patch.txt, 
 HADOOP-10541.3.patch.txt


 The same InputStream variable is used for minikdc.ldiff and minikdc-krb5.conf 
 :
 {code}
 InputStream is = cl.getResourceAsStream(minikdc.ldiff);
 ...
 is = cl.getResourceAsStream(minikdc-krb5.conf);
 {code}
 Before the second assignment, is should be closed.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (HADOOP-10517) InputStream is not closed in two methods of JarFinder

2014-05-04 Thread Swarnim Kulkarni (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10517?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13989274#comment-13989274
 ] 

Swarnim Kulkarni commented on HADOOP-10517:
---

That's very interesting because the only thing extra that closeQuietly does is 
check for null which anyways should throw an NPE for a null value

 InputStream is not closed in two methods of JarFinder
 -

 Key: HADOOP-10517
 URL: https://issues.apache.org/jira/browse/HADOOP-10517
 Project: Hadoop Common
  Issue Type: Bug
Reporter: Ted Yu
Assignee: Ted Yu
Priority: Minor
 Attachments: HADOOP-10517.1.patch.txt, hadoop-10517-v1.txt, 
 hadoop-10517-v2.txt


 JarFinder#jarDir() and JarFinder#zipDir() have such code:
 {code}
  InputStream is = new FileInputStream(f);
  copyToZipStream(is, anEntry, zos);
 {code}
 The InputStream is closed in copyToZipStream() but should be enclosed in 
 finally block.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (HADOOP-10481) Fix new findbugs warnings in hadoop-auth

2014-05-04 Thread Swarnim Kulkarni (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13989278#comment-13989278
 ] 

Swarnim Kulkarni commented on HADOOP-10481:
---

This is ready for review.

 Fix new findbugs warnings in hadoop-auth
 

 Key: HADOOP-10481
 URL: https://issues.apache.org/jira/browse/HADOOP-10481
 Project: Hadoop Common
  Issue Type: Sub-task
Reporter: Haohui Mai
Assignee: Swarnim Kulkarni
  Labels: newbie
 Attachments: HADOOP-10481.1.patch.txt


 The following findbugs warnings need to be fixed:
 {noformat}
 [INFO] --- findbugs-maven-plugin:2.5.3:check (default-cli) @ hadoop-auth ---
 [INFO] BugInstance size is 2
 [INFO] Error size is 0
 [INFO] Total bugs: 2
 [INFO] Found reliance on default encoding in 
 org.apache.hadoop.security.authentication.server.AuthenticationFilter.init(FilterConfig):
  String.getBytes() 
 [org.apache.hadoop.security.authentication.server.AuthenticationFilter] At 
 AuthenticationFilter.java:[lines 76-455]
 [INFO] Found reliance on default encoding in 
 org.apache.hadoop.security.authentication.util.Signer.computeSignature(String):
  String.getBytes() [org.apache.hadoop.security.authentication.util.Signer] 
 At Signer.java:[lines 34-96]
 {noformat}



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (HADOOP-10481) Fix new findbugs warnings in hadoop-auth

2014-05-04 Thread Swarnim Kulkarni (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-10481?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Swarnim Kulkarni updated HADOOP-10481:
--

Attachment: HADOOP-10481.1.patch.txt

Patch attached.

Output with the patch

{noformat}
[INFO] --- findbugs-maven-plugin:2.5.3:findbugs (default-cli) @ hadoop-auth ---
[INFO] Fork Value is true
[INFO] Done FindBugs Analysis
[INFO] 
[INFO] BUILD SUCCESS
[INFO] 
[INFO] Total time: 25.571s
[INFO] Finished at: Sun May 04 23:38:14 CDT 2014
[INFO] Final Memory: 18M/81M
[INFO] 
{noformat}

 Fix new findbugs warnings in hadoop-auth
 

 Key: HADOOP-10481
 URL: https://issues.apache.org/jira/browse/HADOOP-10481
 Project: Hadoop Common
  Issue Type: Sub-task
Reporter: Haohui Mai
Assignee: Swarnim Kulkarni
  Labels: newbie
 Attachments: HADOOP-10481.1.patch.txt


 The following findbugs warnings need to be fixed:
 {noformat}
 [INFO] --- findbugs-maven-plugin:2.5.3:check (default-cli) @ hadoop-auth ---
 [INFO] BugInstance size is 2
 [INFO] Error size is 0
 [INFO] Total bugs: 2
 [INFO] Found reliance on default encoding in 
 org.apache.hadoop.security.authentication.server.AuthenticationFilter.init(FilterConfig):
  String.getBytes() 
 [org.apache.hadoop.security.authentication.server.AuthenticationFilter] At 
 AuthenticationFilter.java:[lines 76-455]
 [INFO] Found reliance on default encoding in 
 org.apache.hadoop.security.authentication.util.Signer.computeSignature(String):
  String.getBytes() [org.apache.hadoop.security.authentication.util.Signer] 
 At Signer.java:[lines 34-96]
 {noformat}



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (HADOOP-10517) InputStream is not closed in two methods of JarFinder

2014-05-04 Thread Swarnim Kulkarni (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10517?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13989281#comment-13989281
 ] 

Swarnim Kulkarni commented on HADOOP-10517:
---

It looks like there is a bug which we can fix with this JIRA. The closeEntry is 
called twice, Once here[1] and second here[2]. So if the if portion gets 
executed, we try to close the entry twice resulting in the error in the JUnit. 
I think we can remove the one inside if to fix the issue.

[1] 
https://github.com/apache/hadoop-common/blob/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/JarFinder.java#L67
[2] 
https://github.com/apache/hadoop-common/blob/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/JarFinder.java#L72

 InputStream is not closed in two methods of JarFinder
 -

 Key: HADOOP-10517
 URL: https://issues.apache.org/jira/browse/HADOOP-10517
 Project: Hadoop Common
  Issue Type: Bug
Reporter: Ted Yu
Assignee: Ted Yu
Priority: Minor
 Attachments: HADOOP-10517.1.patch.txt, hadoop-10517-v1.txt, 
 hadoop-10517-v2.txt


 JarFinder#jarDir() and JarFinder#zipDir() have such code:
 {code}
  InputStream is = new FileInputStream(f);
  copyToZipStream(is, anEntry, zos);
 {code}
 The InputStream is closed in copyToZipStream() but should be enclosed in 
 finally block.



--
This message was sent by Atlassian JIRA
(v6.2#6252)