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/03 08:29:08 UTC

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

apucher commented on a change in pull request #8314:
URL: https://github.com/apache/pinot/pull/8314#discussion_r841184410



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/access/AccessControlUtils.java
##########
@@ -93,4 +93,17 @@ public void validatePermission(Optional<String> tableNameOpt, AccessType accessT
           Response.Status.FORBIDDEN);
     }
   }
+
+
+  public void validatePermission(HttpHeaders httpHeaders, String endpointUrl,
+      AccessControl accessControl) {
+    boolean hasPermission;
+    String accessTypeToEndpintMsg =
+        String.format("request to the endpioint '%s'", endpointUrl);

Review comment:
       delete `accessTypeToEndpintMsg` and merge if & declaration of `hasPermission`?

##########
File path: pinot-core/src/main/java/org/apache/pinot/server/access/ZkBasicAuthAccessFactory.java
##########
@@ -0,0 +1,130 @@
+/**
+ * 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.server.access;
+
+import io.netty.channel.ChannelHandlerContext;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.commons.lang.StringUtils;
+import org.apache.helix.HelixManager;
+import org.apache.pinot.common.config.provider.UserCache;
+import org.apache.pinot.common.utils.BcryptUtils;
+import org.apache.pinot.core.auth.BasicAuthUtils;
+import org.apache.pinot.core.auth.ZkBasicAuthPrincipal;
+import org.apache.pinot.spi.env.PinotConfiguration;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+
+/**
+ * Zookeeper Basic Authentication based on either http headers or Controller UI. Configured user properties
+ * via json format.
+ *
+ * <pre>
+ *     Http headers Json Example:
+ *      {
+ *       "username": "user1",
+ *       "password": "user1@passwd",
+ *       "component": "SERVER",
+ *       "role" : "USER",
+ *       "tables": ["jock_pinot_poc2", "jocktest"],
+ *       "permissions": ["READ"]
+ *      }
+ * </pre>
+ */

Review comment:
       same story with the auth header

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/config/UserConfigUtils.java
##########
@@ -0,0 +1,83 @@
+/**
+ * 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;
+
+public class UserConfigUtils {

Review comment:
       would you mind adding a short javadoc that describes what `UserConfig` is actually being used for?

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/access/ZkBasicAuthAccessControlFactory.java
##########
@@ -0,0 +1,132 @@
+/**
+ * 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.access;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import javax.ws.rs.core.HttpHeaders;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.pinot.common.config.provider.UserCache;
+import org.apache.pinot.common.utils.BcryptUtils;
+import org.apache.pinot.core.auth.BasicAuthUtils;
+import org.apache.pinot.core.auth.ZkBasicAuthPrincipal;
+import org.apache.pinot.spi.config.user.ComponentType;
+import org.apache.pinot.spi.config.user.RoleType;
+
+
+/**
+ * Basic Authentication based on http headers. Configured via the "controller.admin.access.control" family of
+ * properties.
+ *
+ * <pre>
+ *     Example:
+ *     controller.admin.access.control.principals=admin123,user456
+ *     controller.admin.access.control.principals.admin123.password=verysecret
+ *     controller.admin.access.control.principals.user456.password=kindasecret
+ *     controller.admin.access.control.principals.user456.tables=stuff,lessImportantStuff
+ *     controller.admin.access.control.principals.user456.permissions=read,update
+ * </pre>
+ */

Review comment:
       same as above. these aren't the authorization headers you're expecting, right?

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/access/BasicAuthAccessControlFactory.java
##########
@@ -50,10 +51,16 @@
 
   private AccessControl _accessControl;
 
+  @Override
   public void init(PinotConfiguration configuration) {
     _accessControl = new BasicAuthAccessControl(BasicAuthUtils.extractBasicAuthPrincipals(configuration, PREFIX));
   }
 
+  @Override
+  public void init(PinotConfiguration pinotConfiguration, PinotHelixResourceManager pinotHelixResourceManager) {
+    init(pinotConfiguration);
+  }
+

Review comment:
       same as above. no changes should happen to this file

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
##########
@@ -1268,6 +1274,32 @@ public Schema getSchemaForTableConfig(TableConfig tableConfig) {
             AccessOption.PERSISTENT);
   }
 
+  public void initUserACLConfig() throws IOException {
+    if (Optional.ofNullable(ZKMetadataProvider.getAllUserName(_propertyStore)).isEmpty()) {
+      addUser(new UserConfig("admin", "Pda@wap%admin", ComponentType.CONTROLLER.name(),
+          RoleType.ADMIN.name(), null, null));
+      addUser(new UserConfig("admin", "Pda@wap%admin", ComponentType.BROKER.name(),
+          RoleType.ADMIN.name(), null, null));
+      addUser(new UserConfig("admin", "Pda@wap%admin", ComponentType.SERVER.name(),
+          RoleType.ADMIN.name(), null, null));
+    }

Review comment:
       awesome

##########
File path: pinot-broker/src/main/java/org/apache/pinot/broker/broker/AccessControlFactory.java
##########
@@ -29,10 +31,13 @@
   public static final String ACCESS_CONTROL_CLASS_CONFIG = "class";
 
   public abstract void init(PinotConfiguration confguration);
+  public abstract void init(ZkHelixPropertyStore<ZNRecord> propertyStore);
+
 
   public abstract AccessControl create();
 
-  public static AccessControlFactory loadFactory(PinotConfiguration configuration) {
+  public static AccessControlFactory loadFactory(PinotConfiguration configuration,
+      ZkHelixPropertyStore<ZNRecord> propertyStore) {

Review comment:
       see comments below. I'd go as far as marking both init methods non-abstract, and by default implement the new one to call the old one.

##########
File path: pinot-broker/src/main/java/org/apache/pinot/broker/broker/AllowAllAccessControlFactory.java
##########
@@ -34,6 +36,10 @@ public AllowAllAccessControlFactory() {
   public void init(PinotConfiguration configuration) {
   }
 
+  public void init(ZkHelixPropertyStore<ZNRecord> propertyStore) {
+  }
+
+  @Override

Review comment:
       you're still breaking backwards-compatibility. if you have to make changes to this file, you equally affect 3rd party implementations which aren't in the open source repository.
   
   you can simply move the implementation of `init(PinotConfiguration, ZkHelixPropertyStore)` to the AccessControlFactory parent class and declare it non-abstract. then, override the method in you new implementation while this class (and similar ones) can remain unchanged

##########
File path: pinot-broker/src/main/java/org/apache/pinot/broker/broker/BasicAuthAccessControlFactory.java
##########
@@ -55,10 +57,16 @@ public BasicAuthAccessControlFactory() {
     // left blank
   }
 
+  @Override
   public void init(PinotConfiguration configuration) {
     _accessControl = new BasicAuthAccessControl(BasicAuthUtils.extractBasicAuthPrincipals(configuration, PREFIX));
   }
 
+  @Override
+  public void init(ZkHelixPropertyStore<ZNRecord> propertyStore) {
+  }
+
+  @Override

Review comment:
       same

##########
File path: 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:
       @mayankshriv is bcrypt available already? otherwise is the license compatible? https://github.com/jeremyh/jBCrypt/blob/master/LICENSE

##########
File path: pinot-broker/src/main/java/org/apache/pinot/broker/broker/ZkBasicAuthAccessControlFactory.java
##########
@@ -0,0 +1,130 @@
+/**
+ * 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;
+
+
+/**
+ * Basic Authentication based on http headers. Configured via the "pinot.broker.access.control" family of properties.
+ *
+ * <pre>
+ *     Example:
+ *     pinot.broker.access.control.principals=admin123,user456
+ *     pinot.broker.access.control.principals.admin123.password=verysecret
+ *     pinot.broker.access.control.principals.user456.password=kindasecret
+ *     pinot.broker.access.control.principals.user456.tables=stuff,lessImportantStuff
+ * </pre>
+ */

Review comment:
       could you check this? are these the JSON header you're expecting for authorization?

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotUserRestletResource.java
##########
@@ -0,0 +1,244 @@
+/**
+ * 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 java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+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.commons.httpclient.HttpConnectionManager;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.common.metrics.ControllerMetrics;
+import org.apache.pinot.common.utils.BcryptUtils;
+import org.apache.pinot.controller.ControllerConf;
+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.controller.helix.core.minion.PinotHelixTaskResourceManager;
+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:
       move the header javadoc here?




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