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

Andrew Wang commented on HADOOP-10433:
--------------------------------------

Thanks for the huge amount of work here Tucu. I'm still fresh to this topic, so 
please forgive any silly review comments.

* In the docs, "Hadoop KMS Server" is like saying "ATM machine" :) docs could 
use a pass, but we can do that after the base patch is committed.
* For the same reason, you could rename "KMSServer" to just "KMS" or "KMServer"
* More javadoc, at the minimum class javadoc for everything
* KeyNotFoundException needs a serialVersionUID
* pom.xml, not sure where these vars are being used, but I'm not aware of 
Maven's subtleties.
{code}
    <kms.source.repository>REPO NOT AVAIL</kms.source.repository>
    <kms.source.revision>REVISION NOT AVAIL</kms.source.revision>
    <kms.build.timestamp>${maven.build.timestamp}</kms.build.timestamp>
{code}
* "mvn package -Pdist -Dtar" doesn't work because there's an extra newline in 
the pom.xml for dist-maketar.sh.

Notes from running the KMS:
* Running "./sbin/kms.sh" has a lot of debugging output, not sure if we can 
quash it or not.
* Is there some way of aborting tomcat if the KMSWebApp dies? I didn't put a 
core-site.xml in my conf dir initially, KMSWebApp FATAL'd, and I had a useless 
Tomcat process lying around.
* Running ops against the KMS with "hadoop key" generates big log entries like 
this:
{code}
2014-04-16 16:40:28,160 INFO  WadlGeneratorJAXBGrammarGenerator - Couldn't find 
JAX-B element for class javax.ws.rs.core.Response
2014-04-16 16:40:28,160 ERROR WadlGeneratorJAXBGrammarGenerator - 
java.lang.IllegalAccessException: Class 
com.sun.jersey.server.wadl.generators.WadlGeneratorJAXBGrammarGenerator$8 can 
not access a member of class javax.ws.rs.core.Response with modifiers 
"protected"
<stacktrace>
{code}
* Audit logging doesn't have newlines
* I didn't test anything outside of KeyShell operations. I couldn't figure out 
the right curl incantation to hit REST endpoints directly, whenever I tried I 
got an authentication exception about being an anonymous user.

REST API design comments:
* It's kind of ugly that we're doing manual serde of things into JSON, since it 
throws away types and makes the protocol harder to understand. Jackson has ways 
of automatically doing POJO<->JSON for beans, which seems pretty reasonable to 
me.
* If we use Jackson, we should really aspire to JSON in, JSON out instead of 
using so many query parameters (in particular, create).
* I think we should bake the op into the path when possible, since it makes the 
API more self-descriptive. Maybe something like this:
{code}
/keys
/keys?metadata # this is still ugly, dunno
/keys/<keyName> # GET for metadata, PUT for create, POST for roll, DELETE
/keys/<keyName>/versions # getKeyVersions
/keys/<keyName>/versions/<version> # getKeyVersion
/keys/<keyName>/versions/current
{code}
* Also, shouldn't the REST API be explained in the docs? I assume we chose REST 
for interop with non-Hadoop and non-Java components, so the protocol should be 
documented.

KMSClientProvider:
* Class javadoc refers to JksProviders
* Typo: CHIPER_FIELD and chiper, I assume cipher
* It'd be nice to put all the {{private static final}} before {{public static 
final}}. Some of the public ones also aren't used outside of this class, so we 
could close them up.
* Constructor, can use equalsIgnoreCase rather than toLowerCase
* Wherever we're building a URL, consider using URIBuilder and then toURL() 
instead.
* Based on this page 
(http://docs.oracle.com/javase/6/docs/technotes/guides/net/http-keepalive.html) 
we should try/catch when calling HttpURLConnection#getInputStream and I assume 
#getResponseCode too, and also close the input stream after we're done. The 
error stream logic in #validateResponse also doesn't quite adhere to the 
recommendations.
* Is there a reason flush is a NOP? The JKSP doesn't do this, so a explanatory 
comment would help.
* It doesn't look like rollNewVersionInternal needs to throw 
NoSuchAlgorithmException?

KMSACLs:
* You could write this with a ScheduledExecutorService and a 
ThreadFactoryBuilder, more idiomatic.
* Probably should make start/stopReloader idempotent to future proof it, e.g. 
check a "isRunning" boolean.
* In loadACLs, I assume you do the conf get to init the conf outside of the 
lock? Would help to clarify that in the comment.
* The lastReload time should be set to the modtime of the file, not by calling 
System.currentTimeMillis again. Also, what's the point of waiting for it to be 
at least 100ms newer?
* I'd bump the setACLs logs up to INFO, since this probably isn't a frequent 
operation and is pretty useful.

KMSServer:
* The hasAccess method is a bit gross. We could make an Access enum rather than 
passing the ACL as an argument. Could also get rid of the boolean for throwing 
an exception, and have two helper methods {{checkAccess}} and {{assertAccess}} 
or something like that.
* I think it'd also be clearer if all the audit logging happened in each 
function via a try/finally. The control flow with Exceptions means that right 
now we don't always log something.
* Another q, why is the audit logger trying to log client method names rather 
the API that the server provides, e.g. logging "getKeyVersion()" rather than 
{{KMSClientProvider.GET_VERSION}}? Having these strings sprinkled everywhere 
also seems prone to typos.
* It'd also be better to have the {{throws}} state the specific exceptions, 
rather than just throwing {{Exception}}.
* For createKey, why is everything being passed as a query parameter rather 
than a JSON blob in the payload?
* If we get rid of OP by putting it in the path, we could get rid of a bunch of 
these IllegalArgumentExceptions since it'll just 404 instead. Could also use 
Preconditions.checkArgument instead to be more concise.

Tests:
* Got these errors when running the tests, can attach the full logs if you like:
{noformat}
ailed tests: 
  TestKMSServer.testACLs:505->runServer:142->access$0:257->doAs:270 
java.io.IOException: 
org.apache.hadoop.security.authentication.client.AuthenticationException: 
Authentication failed, status: 401, message: Authentication required
  
TestKMSServer.testServicePrincipalACLs:958->runServer:142->access$0:257->doAs:270
 java.io.IOException: 
org.apache.hadoop.security.authentication.client.AuthenticationException: 
Authentication failed, status: 401, message: Authentication required

Tests in error: 
  
TestKMSServer.testStartStopHttpKerberos:345->testStartStop:304->runServer:142->access$0:257->doAs:270
 » PrivilegedAction
  
TestKMSServer.testStartStopHttpsKerberos:350->testStartStop:304->runServer:142->access$0:257->doAs:270
 » PrivilegedAction
{noformat}

> 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, 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)

Reply via email to