You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2022/04/18 19:13:29 UTC

[GitHub] [pinot] mayankshriv commented on a diff in pull request #8314: Access Control management Development

mayankshriv commented on code in PR #8314:
URL: https://github.com/apache/pinot/pull/8314#discussion_r852340894


##########
pinot-broker/src/main/java/org/apache/pinot/broker/broker/ZkBasicAuthAccessControlFactory.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * 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.pinot.broker.broker;
+
+import com.google.common.base.Preconditions;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.pinot.broker.api.AccessControl;
+import org.apache.pinot.broker.api.HttpRequesterIdentity;
+import org.apache.pinot.broker.api.RequesterIdentity;
+import org.apache.pinot.common.config.provider.UserCache;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.common.utils.BcryptUtils;
+import org.apache.pinot.core.auth.BasicAuthPrincipal;
+import org.apache.pinot.core.auth.BasicAuthUtils;
+import org.apache.pinot.core.auth.ZkBasicAuthPrincipal;
+import org.apache.pinot.spi.env.PinotConfiguration;
+
+
+/**
+ * Zookeeper Basic Authentication based on Pinot Controller UI.
+ * The user role has been distinguished by user and admin. Only admin can have access to the
+ * user console page in Pinot controller UI. And admin can change user info (table permission/
+ * number of tables/password etc.) or add/delete user without restarting your Pinot clusters,
+ * and these changes happen immediately.
+ * Users Configuration store in Helix Zookeeper and encrypted user password via Bcrypt Encryption Algorithm.
+ *
+ */
+public class ZkBasicAuthAccessControlFactory extends AccessControlFactory {
+    private static final String HEADER_AUTHORIZATION = "authorization";
+
+    private AccessControl _accessControl;
+
+    public ZkBasicAuthAccessControlFactory() {
+        // left blank
+    }
+
+    @Override
+    public void init(PinotConfiguration configuration, ZkHelixPropertyStore<ZNRecord> propertyStore) {
+        _accessControl = new BasicAuthAccessControl(new UserCache(propertyStore));
+    }
+
+    @Override
+    public AccessControl create() {
+        return _accessControl;
+    }
+
+    /**
+     * Access Control using header-based basic http authentication
+     */
+    private static class BasicAuthAccessControl implements AccessControl {
+        private Map<String, ZkBasicAuthPrincipal> _name2principal;
+        private final UserCache _userCache;
+
+        public BasicAuthAccessControl(UserCache userCache) {
+            _userCache = userCache;
+        }
+
+        @Override
+        public boolean hasAccess(RequesterIdentity requesterIdentity) {
+            return hasAccess(requesterIdentity, null);
+        }
+
+        @Override
+        public boolean hasAccess(RequesterIdentity requesterIdentity, BrokerRequest brokerRequest) {

Review Comment:
   Is this going to be called per read query? If so, not really sure if ZK is a good store for reads, esp at high throughputs.



##########
pinot-broker/src/main/java/org/apache/pinot/broker/broker/AccessControlFactory.java:
##########
@@ -28,11 +30,24 @@ public abstract class AccessControlFactory {
   public static final Logger LOGGER = LoggerFactory.getLogger(AccessControlFactory.class);
   public static final String ACCESS_CONTROL_CLASS_CONFIG = "class";
 
-  public abstract void init(PinotConfiguration confguration);
+  public void init(PinotConfiguration configuration) {
+  };
+
+  /**
+   * Extend original init method inorder to support Zookeeper BasicAuthAccessControlFactory

Review Comment:
   One issue I see with this approach is that we will end up adding flavors of `init()` for each of the auth factories depending on what they needed. One alternative might be to have something like `PinotHelixResourceManager` that can give access to cluster level properties, to make it more shareable across other future factory inits.



##########
pinot-broker/src/main/java/org/apache/pinot/broker/broker/ZkBasicAuthAccessControlFactory.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * 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.pinot.broker.broker;
+
+import com.google.common.base.Preconditions;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.pinot.broker.api.AccessControl;
+import org.apache.pinot.broker.api.HttpRequesterIdentity;
+import org.apache.pinot.broker.api.RequesterIdentity;
+import org.apache.pinot.common.config.provider.UserCache;
+import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.common.utils.BcryptUtils;
+import org.apache.pinot.core.auth.BasicAuthPrincipal;
+import org.apache.pinot.core.auth.BasicAuthUtils;
+import org.apache.pinot.core.auth.ZkBasicAuthPrincipal;
+import org.apache.pinot.spi.env.PinotConfiguration;
+
+
+/**
+ * Zookeeper Basic Authentication based on Pinot Controller UI.
+ * The user role has been distinguished by user and admin. Only admin can have access to the
+ * user console page in Pinot controller UI. And admin can change user info (table permission/
+ * number of tables/password etc.) or add/delete user without restarting your Pinot clusters,
+ * and these changes happen immediately.
+ * Users Configuration store in Helix Zookeeper and encrypted user password via Bcrypt Encryption Algorithm.
+ *
+ */
+public class ZkBasicAuthAccessControlFactory extends AccessControlFactory {
+    private static final String HEADER_AUTHORIZATION = "authorization";
+
+    private AccessControl _accessControl;
+
+    public ZkBasicAuthAccessControlFactory() {
+        // left blank
+    }
+
+    @Override
+    public void init(PinotConfiguration configuration, ZkHelixPropertyStore<ZNRecord> propertyStore) {
+        _accessControl = new BasicAuthAccessControl(new UserCache(propertyStore));
+    }
+
+    @Override
+    public AccessControl create() {
+        return _accessControl;
+    }
+
+    /**
+     * Access Control using header-based basic http authentication
+     */
+    private static class BasicAuthAccessControl implements AccessControl {
+        private Map<String, ZkBasicAuthPrincipal> _name2principal;
+        private final UserCache _userCache;
+
+        public BasicAuthAccessControl(UserCache userCache) {
+            _userCache = userCache;
+        }
+
+        @Override
+        public boolean hasAccess(RequesterIdentity requesterIdentity) {
+            return hasAccess(requesterIdentity, null);
+        }
+
+        @Override
+        public boolean hasAccess(RequesterIdentity requesterIdentity, BrokerRequest brokerRequest) {

Review Comment:
   Oh perhaps this is going to be used only to populate/update cache? If so, then the above comment is a non-issue.



##########
pinot-common/pom.xml:
##########
@@ -301,6 +301,11 @@
       <groupId>org.apache.yetus</groupId>
       <artifactId>audience-annotations</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.mindrot</groupId>
+      <artifactId>jbcrypt</artifactId>
+      <version>0.4</version>
+    </dependency>

Review Comment:
   Thanks @apucher. If this is a new dependency being added, licensing might not be compatible with Apache License. Are there alternatives to be used here?



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotUserRestletResource.java:
##########
@@ -0,0 +1,243 @@
+/**
+ * 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.pinot.controller.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import javax.inject.Inject;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.common.utils.BcryptUtils;
+import org.apache.pinot.controller.api.access.AccessControlFactory;
+import org.apache.pinot.controller.api.access.AccessControlUtils;
+import org.apache.pinot.controller.api.access.AccessType;
+import org.apache.pinot.controller.api.access.Authenticate;
+import org.apache.pinot.controller.api.exception.ControllerApplicationException;
+import org.apache.pinot.controller.api.exception.UserAlreadyExistsException;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.spi.config.user.ComponentType;
+import org.apache.pinot.spi.config.user.UserConfig;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.glassfish.grizzly.http.server.Request;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Api(tags = Constants.USER_TAG)
+@Path("/")
+public class PinotUserRestletResource {

Review Comment:
   Further qualify class name.



##########
pinot-common/src/main/java/org/apache/pinot/common/config/provider/UserCache.java:
##########
@@ -0,0 +1,174 @@
+/**
+ * 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.pinot.common.config.provider;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.I0Itec.zkclient.IZkChildListener;
+import org.I0Itec.zkclient.IZkDataListener;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.helix.AccessOption;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.pinot.common.utils.config.UserConfigUtils;
+import org.apache.pinot.spi.config.user.ComponentType;
+import org.apache.pinot.spi.config.user.UserConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class UserCache {

Review Comment:
   Please add JavaDoc for the class. Also, qualify the class name a bit more, for example, `AccessControlUserCache` or similar.



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/config/UserConfigUtils.java:
##########
@@ -0,0 +1,88 @@
+/**
+ * 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.pinot.common.utils.config;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.helix.ZNRecord;
+import org.apache.pinot.spi.config.user.AccessType;
+import org.apache.pinot.spi.config.user.UserConfig;
+
+/**
+ * UserConfigUtils is responsible for two things:
+ * 1. Used to acquire user config by parsing znRecord that stored in Zookeeper
+ * 2. Used to construct znRecord by packaging user config
+ */
+public class UserConfigUtils {

Review Comment:
   Further qualify class name.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org