zookeeper-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (ZOOKEEPER-1525) Plumb ZooKeeperServer object into auth plugins
Date Mon, 07 Nov 2016 20:15:58 GMT

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

ASF GitHub Bot commented on ZOOKEEPER-1525:
-------------------------------------------

Github user lvfangmin commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/84#discussion_r86824750
  
    --- Diff: src/java/main/org/apache/zookeeper/server/auth/KeyAuthenticationProvider.java
---
    @@ -0,0 +1,139 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.zookeeper.server.auth;
    +
    +import java.io.UnsupportedEncodingException;
    +import java.util.List;
    +
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.KeeperException.NoNodeException;
    +import org.apache.zookeeper.data.ACL;
    +import org.apache.zookeeper.data.Id;
    +import org.apache.zookeeper.server.ServerCnxn;
    +import org.apache.zookeeper.server.ZooKeeperServer;
    +import org.apache.zookeeper.server.ZKDatabase;
    +import org.apache.zookeeper.data.Stat;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/*
    + * This class is a sample implementation of being passed the ZooKeeperServer
    + * handle in the constructor, and reading data from zknodes to authenticate.
    + * At a minimum, a real Auth provider would need to override validate() to
    + * e.g. perform certificate validation of auth based a public key.
    + *
    + * See the "Pluggable ZooKeeper authentication" section of the 
    + * "Zookeeper Programmer's Guide" for general details of implementing an
    + * authentication plugin. e.g.
    + * http://zookeeper.apache.org/doc/trunk/zookeeperProgrammers.html#sc_ZooKeeperPluggableAuthentication
    + *
    + * This class looks for a numeric "key" under the /key node.
    + * Authorizaton is granted if the user passes in as authorization a number
    + * which is a multiple of the key value, i.e. 
    + *   (auth % key) == 0
    + * In a real implementation, you might do something like storing a public
    + * key in /key, and using it to verify that auth tokens passed in were signed
    + * by the corresponding private key.
    + *
    + * When the node /key does not exist, any auth token is accepted, so that 
    + * bootstrapping may occur.
    + *
    + */
    +public class KeyAuthenticationProvider extends ServerAuthenticationProvider {
    +    private static final Logger LOG = LoggerFactory.getLogger(KeyAuthenticationProvider.class);
    +
    +    public String getScheme() {
    +        return "key";
    +    }
    +
    +    public byte[] getKey(ZooKeeperServer zks) {
    +        ZKDatabase db = zks.getZKDatabase();
    +        if (db != null) {
    +            try {
    +                Stat stat = new Stat();
    +                return db.getData("/key", stat, null);
    +            } catch (NoNodeException e) {
    +                // ignore
    +            }
    +        }
    +        return null;
    +    }
    +
    +    public boolean validate(byte[] key, byte[] auth) {
    +        // perform arbitrary function (auth is a multiple of key)
    +        try {
    +            String keyStr = new String(key, "UTF-8");
    +            String authStr = new String(auth, "UTF-8");
    +            int keyVal = Integer.parseInt(keyStr);
    +            int authVal = Integer.parseInt(authStr);
    +            if (keyVal!=0 && ((authVal % keyVal) != 0)) {
    +              return false;
    +            }
    +        } catch (NumberFormatException | UnsupportedEncodingException nfe) {
    +          return false;
    +        }
    +        return true;
    +    }
    +
    +    @Override
    +    public KeeperException.Code handleAuthentication(ZooKeeperServer zks, ServerCnxn
cnxn, byte[] authData) {
    +        byte[] key = getKey(zks);
    +        String authStr = "";
    +        String keyStr = "";
    +        try {
    +          authStr = new String(authData, "UTF-8");
    +        } catch (Exception e) {
    +          // empty authData
    --- End diff --
    
    Same here, better to log the exception.


> Plumb ZooKeeperServer object into auth plugins
> ----------------------------------------------
>
>                 Key: ZOOKEEPER-1525
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1525
>             Project: ZooKeeper
>          Issue Type: Improvement
>    Affects Versions: 3.5.0
>            Reporter: Warren Turkal
>            Assignee: Jordan Zimmerman
>             Fix For: 3.5.3, 3.6.0
>
>         Attachments: ZOOKEEPER-1525.patch, ZOOKEEPER-1525.patch, ZOOKEEPER-1525.patch,
ZOOKEEPER-1525.patch, ZOOKEEPER-1525.patch, ZOOKEEPER-1525.patch, ZOOKEEPER-1525.patch, ZOOKEEPER-1525.patch,
ZOOKEEPER-1525.patch, ZOOKEEPER-1525.patch
>
>
> I want to plumb the ZooKeeperServer object into the auth plugins so that I can store
authentication data in zookeeper itself. With access to the ZooKeeperServer object, I also
have access to the ZKDatabase and can look up entries in the local copy of the zookeeper data.
> In order to implement this, I make sure that a ZooKeeperServer instance is passed in
to the ProviderRegistry.initialize() method. Then initialize() will try to find a constructor
for the AuthenticationProvider that takes a ZooKeeperServer instance. If the constructor is
found, it will be used. Otherwise, initialize() will look for a constructor that takes no
arguments and use that instead.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message