You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@zookeeper.apache.org by GitBox <gi...@apache.org> on 2020/10/14 20:38:54 UTC

[GitHub] [zookeeper] arshadmohammad opened a new pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

arshadmohammad opened a new pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] symat commented on pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
symat commented on pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504#issuecomment-716161808


   FYI: I'm merging this to master


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] symat commented on pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
symat commented on pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504#issuecomment-716162556


   merge done, thanks @arshadmohammad for your contribution!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] eolivelli commented on a change in pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504#discussion_r505309630



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java
##########
@@ -3070,4 +3071,19 @@ private void validateACL(List<ACL> acl) throws KeeperException.InvalidACLExcepti
         }
     }
 
+    /**
+     * Returns comma separated list of users authenticated in the current session
+     *
+     * @return users
+     * @throws InterruptedException when interrupted
+     */
+    public String whoAmI() throws InterruptedException {

Review comment:
       What about returning a simple datastructure ?
   in the future we will be able to return more data probably.
   
   I would like to see something like:
   class ClientInfo {
       String[] users;
       // auth scheme ?
       // other relevant info ?
   }




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] arshadmohammad commented on pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
arshadmohammad commented on pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504#issuecomment-709525564


   > What about returning a simple datastructure ?
   Done


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] arshadmohammad commented on a change in pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
arshadmohammad commented on a change in pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504#discussion_r505571007



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java
##########
@@ -3070,4 +3071,19 @@ private void validateACL(List<ACL> acl) throws KeeperException.InvalidACLExcepti
         }
     }
 
+    /**
+     * Returns comma separated list of users authenticated in the current session
+     *
+     * @return users
+     * @throws InterruptedException when interrupted
+     */
+    public String whoAmI() throws InterruptedException {

Review comment:
       Good suggestion. 
   Added  below structure
   ```
   class ClientInfo {
           ustring authScheme; // Authentication scheme
           ustring user;       // user name or any other id(for example ip)
       }
   ```
   Response changed to 
   ```
   class WhoAmIResponse {
           vector<org.apache.zookeeper.data.ClientInfo> clientInfo;
       }
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] ztzg commented on a change in pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504#discussion_r505258006



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java
##########
@@ -565,6 +566,12 @@ public void processRequest(Request request) {
                 rsp = new GetEphemeralsResponse(ephemerals);
                 break;
             }
+            case OpCode.whoAmI: {
+                lastOp = "HOMI";
+                rsp = new WhoAmIResponse(request.getUsers(true));
+                break;
+            }
+

Review comment:
       This is where the builds break; it wants a `default:` in this switch:
   
   > `[ERROR] Switch statement found in org.apache.zookeeper.server.FinalRequestProcessor.processRequest(Request) where default case is missing [org.apache.zookeeper.server.FinalRequestProcessor] At FinalRequestProcessor.java:[lines 212-571] SF_SWITCH_NO_DEFAULT`
   
   (Not something you introduced, AFAICT, but "you touch it, you own it" seems to be the policy :)
   
   

##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java
##########
@@ -3070,4 +3071,19 @@ private void validateACL(List<ACL> acl) throws KeeperException.InvalidACLExcepti
         }
     }
 
+    /**
+     * Returns comma separated list of users authenticated in the current session
+     *
+     * @return users
+     * @throws InterruptedException when interrupted
+     */
+    public String whoAmI() throws InterruptedException {

Review comment:
       I note the absence of an asynchronous version of the operation, but I suppose we can live without it :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] arshadmohammad commented on pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
arshadmohammad commented on pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504#issuecomment-715511119


   @eolivelli  all the requested changes are done. Pleas have a look..


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] muse-dev[bot] commented on a change in pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
muse-dev[bot] commented on a change in pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504#discussion_r504963034



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java
##########
@@ -3070,4 +3071,19 @@ private void validateACL(List<ACL> acl) throws KeeperException.InvalidACLExcepti
         }
     }
 
+    /**
+     * Returns comma separated list of users authenticated in the current session
+     *
+     * @return users
+     * @throws InterruptedException when interrupted
+     */
+    public String whoAmI() throws InterruptedException {
+        RequestHeader h = new RequestHeader();
+        h.setType(ZooDefs.OpCode.whoAmI);
+        WhoAmIResponse response = new WhoAmIResponse();
+        cnxn.submitRequest(h, null, response, null);

Review comment:
       *THREAD_SAFETY_VIOLATION:*  Unprotected write. Non-private method `ZooKeeper.whoAmI()` indirectly writes to field `this.cnxn.sendThread.clientCnxnSocket.lastHeard` outside of synchronization.
    Reporting because another access to the same memory occurs on a background thread, although this access may not.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] arshadmohammad commented on a change in pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
arshadmohammad commented on a change in pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504#discussion_r505768183



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java
##########
@@ -565,6 +566,12 @@ public void processRequest(Request request) {
                 rsp = new GetEphemeralsResponse(ephemerals);
                 break;
             }
+            case OpCode.whoAmI: {
+                lastOp = "HOMI";
+                rsp = new WhoAmIResponse(request.getUsers(true));
+                break;
+            }
+

Review comment:
       Handled




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] arshadmohammad commented on a change in pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
arshadmohammad commented on a change in pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504#discussion_r506688563



##########
File path: zookeeper-server/src/test/java/org/apache/zookeeper/ZooKeeperTest.java
##########
@@ -698,4 +700,55 @@ public void testInsufficientPermission() throws Exception {
         assertEquals("Insufficient permission : " + zNodeToBeCreated, errorMessage);
     }
 
+    @Test
+    public void testWhoAmIAPI() throws Exception {
+        final ZooKeeper zk = createClient();
+
+        // Check who ami without authentication/without any user into the session
+        List<ClientInfo> clientInfos = zk.whoAmI();
+        // By default server adds ip as the authentication info
+        assertEquals(1, clientInfos.size());
+        assertEquals("ip", clientInfos.get(0).getAuthScheme());
+
+        // Add one user into the session
+        zk.addAuthInfo("digest", "user1:abcXYZ".getBytes());
+        clientInfos = zk.whoAmI();
+        assertEquals(2, clientInfos.size());
+        ClientInfo user1 = getClientInfos(clientInfos, "user1");
+        assertEquals("digest", user1.getAuthScheme());
+
+        // Add one more user into the session
+        zk.addAuthInfo("digest", "user2:xyzABC".getBytes());
+        clientInfos = zk.whoAmI();
+        assertEquals(3, clientInfos.size());
+        user1 = getClientInfos(clientInfos, "user1");
+        assertEquals("digest", user1.getAuthScheme());
+        ClientInfo user2 = getClientInfos(clientInfos, "user2");
+        assertEquals("digest", user2.getAuthScheme());
+    }
+
+    private ClientInfo getClientInfos(List<ClientInfo> clientInfos, String user) {
+        for (ClientInfo clientInfo : clientInfos) {
+            if (clientInfo.getUser().equals(user)) {
+                return clientInfo;
+            }
+        }
+        throw new AssertionError("User +" + user + " not found");
+    }
+
+    @Test
+    public void testWhoAmICLICommand() throws Exception {
+        final ZooKeeper zk = createClient();
+        WhoAmICommand cmd = new WhoAmICommand();
+        cmd.setZk(zk);
+
+        // Check who ami without authentication/without any user into the session
+        cmd.parse(new String[] { "whoami" });
+        runCommandExpect(cmd, Arrays.asList("Auth scheme: User", "ip: 127.0.0.1"));
+
+        // Add one user into the session
+        zk.addAuthInfo("digest", "user1:abcXYZ".getBytes());
+        runCommandExpect(cmd, Arrays.asList("Auth scheme: User", "digest: user1", "ip: 127.0.0.1"));

Review comment:
       Very much valid point.
   Corrected it




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] asfgit closed pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] arshadmohammad commented on pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
arshadmohammad commented on pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504#issuecomment-710509479


   Handled it.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] muse-dev[bot] commented on a change in pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
muse-dev[bot] commented on a change in pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504#discussion_r505697115



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/cli/WhoAmICommand.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.cli;
+
+import java.util.List;
+import org.apache.zookeeper.data.ClientInfo;
+
+/**
+ * WhoAmI command for cli
+ */
+public class WhoAmICommand extends CliCommand {
+
+    public WhoAmICommand() {
+        super("whoami", "");
+    }
+
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws CliParseException {
+        return this;
+    }
+
+    @Override
+    public boolean exec() throws CliException {
+        try {
+            List<ClientInfo> clientInfos = zk.whoAmI();
+            out.println("Auth scheme: User");
+            clientInfos.forEach(clientInfo -> {

Review comment:
       *NULL_DEREFERENCE:*  object `clientInfos` last assigned on line 41 could be null and is dereferenced at line 43.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] ztzg commented on a change in pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
ztzg commented on a change in pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504#discussion_r505777242



##########
File path: zookeeper-server/src/test/java/org/apache/zookeeper/ZooKeeperTest.java
##########
@@ -698,4 +700,55 @@ public void testInsufficientPermission() throws Exception {
         assertEquals("Insufficient permission : " + zNodeToBeCreated, errorMessage);
     }
 
+    @Test
+    public void testWhoAmIAPI() throws Exception {
+        final ZooKeeper zk = createClient();
+
+        // Check who ami without authentication/without any user into the session
+        List<ClientInfo> clientInfos = zk.whoAmI();
+        // By default server adds ip as the authentication info
+        assertEquals(1, clientInfos.size());
+        assertEquals("ip", clientInfos.get(0).getAuthScheme());
+
+        // Add one user into the session
+        zk.addAuthInfo("digest", "user1:abcXYZ".getBytes());
+        clientInfos = zk.whoAmI();
+        assertEquals(2, clientInfos.size());
+        ClientInfo user1 = getClientInfos(clientInfos, "user1");
+        assertEquals("digest", user1.getAuthScheme());
+
+        // Add one more user into the session
+        zk.addAuthInfo("digest", "user2:xyzABC".getBytes());
+        clientInfos = zk.whoAmI();
+        assertEquals(3, clientInfos.size());
+        user1 = getClientInfos(clientInfos, "user1");
+        assertEquals("digest", user1.getAuthScheme());
+        ClientInfo user2 = getClientInfos(clientInfos, "user2");
+        assertEquals("digest", user2.getAuthScheme());
+    }
+
+    private ClientInfo getClientInfos(List<ClientInfo> clientInfos, String user) {
+        for (ClientInfo clientInfo : clientInfos) {
+            if (clientInfo.getUser().equals(user)) {
+                return clientInfo;
+            }
+        }
+        throw new AssertionError("User +" + user + " not found");
+    }
+
+    @Test
+    public void testWhoAmICLICommand() throws Exception {
+        final ZooKeeper zk = createClient();
+        WhoAmICommand cmd = new WhoAmICommand();
+        cmd.setZk(zk);
+
+        // Check who ami without authentication/without any user into the session
+        cmd.parse(new String[] { "whoami" });
+        runCommandExpect(cmd, Arrays.asList("Auth scheme: User", "ip: 127.0.0.1"));
+
+        // Add one user into the session
+        zk.addAuthInfo("digest", "user1:abcXYZ".getBytes());
+        runCommandExpect(cmd, Arrays.asList("Auth scheme: User", "digest: user1", "ip: 127.0.0.1"));

Review comment:
       I'm not sure one can expect this output to be stable across JVMs: the `List<Id>` used to build the response is [initialized from `ServerCnxn.authInfo`](https://github.com/apache/zookeeper/blob/c47ef905e077/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxn.java#L277), which is a `ConcurrentHashMap`-backed `Set`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] arshadmohammad commented on a change in pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
arshadmohammad commented on a change in pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504#discussion_r505571861



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java
##########
@@ -3070,4 +3071,19 @@ private void validateACL(List<ACL> acl) throws KeeperException.InvalidACLExcepti
         }
     }
 
+    /**
+     * Returns comma separated list of users authenticated in the current session
+     *
+     * @return users
+     * @throws InterruptedException when interrupted
+     */
+    public String whoAmI() throws InterruptedException {

Review comment:
       > but I suppose we can live without it :)
   yes, not required as of now, Can be added whenever there is use case.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [zookeeper] arshadmohammad commented on a change in pull request #1504: ZOOKEEPER-3969: Add whoami API and Cli command

Posted by GitBox <gi...@apache.org>.
arshadmohammad commented on a change in pull request #1504:
URL: https://github.com/apache/zookeeper/pull/1504#discussion_r505767782



##########
File path: zookeeper-server/src/main/java/org/apache/zookeeper/cli/WhoAmICommand.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.cli;
+
+import java.util.List;
+import org.apache.zookeeper.data.ClientInfo;
+
+/**
+ * WhoAmI command for cli
+ */
+public class WhoAmICommand extends CliCommand {
+
+    public WhoAmICommand() {
+        super("whoami", "");
+    }
+
+    @Override
+    public CliCommand parse(String[] cmdArgs) throws CliParseException {
+        return this;
+    }
+
+    @Override
+    public boolean exec() throws CliException {
+        try {
+            List<ClientInfo> clientInfos = zk.whoAmI();
+            out.println("Auth scheme: User");
+            clientInfos.forEach(clientInfo -> {

Review comment:
       Handled




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org