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/03/09 00:20:45 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_r822176863



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/config/provider/UserCache.java
##########
@@ -0,0 +1,176 @@
+/**
+ * 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 {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(UserCache.class);
+    private static final String USER_CONFIG_PARENT_PATH = "/CONFIGS/USER";
+    private static final String USER_CONFIG_PATH_PREFIX = "/CONFIGS/USER/";
+
+    private final ZkHelixPropertyStore<ZNRecord> _propertyStore;
+
+    private final UserConfigChangeListener _userConfigChangeListener = new UserConfigChangeListener();
+
+    private final Map<String, UserConfig> _userConfigMap = new ConcurrentHashMap<>();
+    private final Map<String, UserConfig> _userControllerConfigMap = new ConcurrentHashMap<>();
+    private final Map<String, UserConfig> _userBrokerConfigMap = new ConcurrentHashMap<>();
+    private final Map<String, UserConfig> _userServerConfigMap = new ConcurrentHashMap<>();

Review comment:
       if you intend to configure all components dynamically, you'll need to track `minion` as well.
   
   @xiangfu0 should we track dedicated permissions for the upcoming pinot proxy as well?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/auth/BasicAuthUtils.java
##########
@@ -56,26 +58,29 @@ private BasicAuthUtils() {
    *     my.prefix.access.control.principals.user456.permissions=read,update
    * </pre>
    *
-   * @param configuration pinot configuration
-   * @param prefix configuration namespace
+   * @param userConfigList user configuration list
    * @return list of BasicAuthPrincipals
    */
-  public static List<BasicAuthPrincipal> extractBasicAuthPrincipals(PinotConfiguration configuration, String prefix) {
-    String principalNames = configuration.getProperty(prefix);
-    Preconditions.checkArgument(StringUtils.isNotBlank(principalNames), "must provide principals");
+  public static List<BasicAuthPrincipal> extractBasicAuthPrincipals(List<UserConfig> userConfigList) {
+    return userConfigList.stream()
+        .map(user -> {
+          String name = user.getUserName().trim();
+          Preconditions.checkArgument(StringUtils.isNoneBlank(name), "%s is not a valid username", name);
+          String password = user.getPassword().trim();
+          Preconditions.checkArgument(StringUtils.isNoneBlank(password), "must provide a password for %s", name);

Review comment:
       should this be `isNotBlank`?

##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/config/user/UserConfig.java
##########
@@ -0,0 +1,143 @@
+/**
+ * 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.spi.config.user;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyDescription;
+import com.google.common.base.Preconditions;
+import java.util.List;
+import java.util.Objects;
+import javax.annotation.Nullable;
+import org.apache.pinot.spi.config.BaseJsonConfig;
+
+
+public class UserConfig extends BaseJsonConfig {
+    public static final String USERNAME_KEY = "username";
+    public static final String PASSWORD_KEY = "password";
+    public static final String COMPONET_KEY = "component";
+    public static final String ROLE_KEY = "role";
+    public static final String AUTH_TOKEN_KEY = "authToken";
+    public static final String TABLES_KEY = "tables";
+    public static final String PERMISSIONS_KEY = "permissions";
+
+    @JsonPropertyDescription("The name of User")
+    private String _username;
+
+    @JsonPropertyDescription("The password of User")
+    private String _password;
+
+    @JsonPropertyDescription("The name of Component")
+    private ComponentType _componentType;
+
+    @JsonPropertyDescription("The role of user")
+    private RoleType _roleType;
+
+    @JsonPropertyDescription("The tables owned of User")
+    private List<String> _tables;
+
+    @JsonPropertyDescription("The table permission of User")
+    private List<AccessType> _permissions;
+
+    @JsonCreator
+    public UserConfig(@JsonProperty(value = USERNAME_KEY, required = true) String username,
+        @JsonProperty(value = PASSWORD_KEY, required = true) String password,
+        @JsonProperty(value = COMPONET_KEY, required = true) String component,
+        @JsonProperty(value = ROLE_KEY, required = true) String role,
+        @JsonProperty(value = TABLES_KEY) @Nullable List<String> tableList,
+        @JsonProperty(value = PERMISSIONS_KEY) @Nullable List<AccessType> permissionList
+    ) {
+        Preconditions.checkArgument(username != null, "'username' must be configured");
+        Preconditions.checkArgument(password != null, "'password' must be configured");
+
+        // NOTE: Handle lower case table type and raw table name for backward-compatibility
+        _username = username;
+        _password = password;
+        _componentType = ComponentType.valueOf(component.toUpperCase());
+        _roleType = RoleType.valueOf(role.toUpperCase());
+        _tables = tableList;
+        _permissions = permissionList;
+    }
+
+    @JsonProperty(USERNAME_KEY)
+    public String getUserName() {
+        return _username;
+    }
+
+    public String getUsernameWithComponent() {
+        return getUserName() + "_" + getComponentType().toString();
+    }
+
+    public boolean isExist(String username, ComponentType component) {
+        return _username.equals(username) && _componentType.equals(component);
+    }
+
+    @JsonProperty(PASSWORD_KEY)
+    public String getPassword() {
+        return _password;
+    }
+
+    @JsonProperty(TABLES_KEY)
+    public List<String> getTables() {
+        return _tables;
+    }
+
+    @JsonProperty(PERMISSIONS_KEY)
+    public List<AccessType> getPermissios() {
+        return _permissions;
+    }
+
+    @JsonProperty(COMPONET_KEY)
+    public ComponentType getComponentType() {
+        return _componentType;
+    }
+
+    @JsonProperty(ROLE_KEY)
+    public RoleType getRoleType() {
+        return _roleType;
+    }
+
+    public void setRole(String roleTypeStr) {
+        _roleType = RoleType.valueOf(roleTypeStr);
+    }
+
+    public void setPassword(String password) {
+        _password = password;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        if (!super.equals(o)) {
+            return false;
+        }
+        UserConfig that = (UserConfig) o;
+        return _username.equals(that._username) && _componentType == that._componentType;
+    }

Review comment:
       this might lead to unexpected behavior. what's the rationale for using `username` and `component` only?

##########
File path: pinot-broker/src/main/java/org/apache/pinot/broker/broker/BasicAuthAccessControlFactory.java
##########
@@ -55,8 +58,13 @@ public BasicAuthAccessControlFactory() {
     // left blank
   }

Review comment:
       This whole class should be factored out into a separate implementation. We have to ensure backwards-compatibility for the existing BasicAuth implementation. You could move your implementation to a new `ZkBasicAuth` class or similar.
   
   This has the further benefit that your whole PR may become fully backwards compatible.

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/access/AccessControlFactory.java
##########
@@ -30,5 +32,8 @@ default void init(PinotConfiguration pinotConfiguration) {
     // left blank
   }
 
+  default void init(ZkHelixPropertyStore<ZNRecord> zkHelixPropertyStore) {
+  }
+

Review comment:
       similar to broker, backwards-compatibility is key. I suggest a `init(ControllerConf, ZkHelixPropertyStore<ZNRecord>)` here that calls `init(ControllerConf)` by default

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/auth/BasicAuthTest.java
##########
@@ -28,24 +30,30 @@
   @Test

Review comment:
       move implementation to new class, retain backward-compatibility.

##########
File path: pinot-core/src/main/java/org/apache/pinot/server/access/BasicAuthAccessFactory.java
##########
@@ -37,10 +40,16 @@
 
   private AccessControl _accessControl;
 
+  @Override
   public void init(PinotConfiguration configuration) {
-    _accessControl = new BasicAuthAccessControl(BasicAuthUtils.extractBasicAuthPrincipals(configuration, PREFIX));
+//    _accessControl = new BasicAuthAccessControl(BasicAuthUtils.extractBasicAuthPrincipals(configuration, PREFIX));
   }
 
+  public void init(ZkHelixPropertyStore<ZNRecord> propertyStore) {
+    _accessControl = new BasicAuthAccessControl(new UserCache(propertyStore));
+  }

Review comment:
       same as broker and controller. extend the interface and default to legacy behavior. also, move the implementation to its own class.

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/EncryptUtils.java
##########
@@ -0,0 +1,180 @@
+/**
+ * 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;
+
+import java.util.Base64;
+import javax.crypto.Cipher;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * AES Encryption Algorithm
+ *
+ */
+public class EncryptUtils {
+
+    /**
+     * encryption and decryption keys, even external
+     */
+    public static final String AES_DATA_SECURITY_KEY = "4%YkW!@g5LGcf9Ut";

Review comment:
       +1 for storing passwords in encrypted form. I suggest having a look at `bcrypt` as an alternative since a hard-coded password is easily reverse-engineered. Chances are, you'll be able to replace this entire util with a native implementation.

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/config/provider/UserCache.java
##########
@@ -0,0 +1,176 @@
+/**
+ * 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 {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(UserCache.class);
+    private static final String USER_CONFIG_PARENT_PATH = "/CONFIGS/USER";
+    private static final String USER_CONFIG_PATH_PREFIX = "/CONFIGS/USER/";
+
+    private final ZkHelixPropertyStore<ZNRecord> _propertyStore;
+
+    private final UserConfigChangeListener _userConfigChangeListener = new UserConfigChangeListener();
+
+    private final Map<String, UserConfig> _userConfigMap = new ConcurrentHashMap<>();
+    private final Map<String, UserConfig> _userControllerConfigMap = new ConcurrentHashMap<>();
+    private final Map<String, UserConfig> _userBrokerConfigMap = new ConcurrentHashMap<>();
+    private final Map<String, UserConfig> _userServerConfigMap = new ConcurrentHashMap<>();
+
+    public UserCache(ZkHelixPropertyStore<ZNRecord> propertyStore) {
+        _propertyStore = propertyStore;
+        synchronized (_userConfigChangeListener) {

Review comment:
       what's the rationale behind `synchronized` here?

##########
File path: pinot-controller/src/main/resources/app/App.tsx
##########
@@ -52,6 +52,29 @@ const App = () => {
   const [authorizationEndpoint, setAuthorizationEndpoint] = React.useState(
     null
   );
+  const [role, setRole] = React.useState('');

Review comment:
       @joshigaurava could you have a look here?

##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/config/user/ComponentType.java
##########
@@ -0,0 +1,23 @@
+/**
+ * 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.spi.config.user;
+
+public enum ComponentType {
+    CONTROLLER, BROKER, SERVER

Review comment:
       + minion, proxy

##########
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:
       We can't break backwards-compatibility with existing auth implementations - however, we can add an interface here that receives both `PinotConfiguration` and `ZkHelixPropertyStore` and has a default implementation that simply calls the legacy `init(PinotConfiguration)`
   
   The BrokerStarter can then call the enhance interface. Existing AuthFactories will simply default to the existing behavior, while your new implementation can override and use the enhanced one.

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/config/provider/UserCache.java
##########
@@ -0,0 +1,176 @@
+/**
+ * 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 {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(UserCache.class);
+    private static final String USER_CONFIG_PARENT_PATH = "/CONFIGS/USER";
+    private static final String USER_CONFIG_PATH_PREFIX = "/CONFIGS/USER/";

Review comment:
       I like the overall approach here. @Jackie-Jiang @xiangfu0 may have some thoughts on ZK paths here and below

##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/config/user/AccessType.java
##########
@@ -0,0 +1,23 @@
+/**
+ * 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.spi.config.user;
+
+public enum AccessType {
+    CREATE, READ, UPDATE, DELETE
+}

Review comment:
       @Jackie-Jiang @xiangfu0 should the SPI changes here go into a different package?

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/access/BasicAuthAccessControlFactory.java
##########
@@ -50,8 +53,9 @@
 

Review comment:
       Similar to Broker auth, this should be in its own class for backwards-compatibility.

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/access/AccessControlUtils.java
##########
@@ -93,4 +93,21 @@ public void validatePermission(Optional<String> tableNameOpt, AccessType accessT
           Response.Status.FORBIDDEN);
     }
   }
+
+  public void validatePermission(HttpHeaders httpHeaders, String endpointUrl,
+      AccessControl accessControl) {
+    validatePermission("you", httpHeaders, endpointUrl, accessControl);

Review comment:
       what's the rationale behind a hard-coded `you` user here?

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/Constants.java
##########
@@ -59,6 +61,18 @@ public static TableType validateTableType(String tableTypeStr) {
     }
   }
 
+  public static ComponentType validateComponentType(String componentTypeStr) {
+    if (componentTypeStr == null || componentTypeStr.isEmpty()) {
+      return null;
+    }

Review comment:
       you could us `StringUtils.isNotNull` here

##########
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:
       I suggest factoring-out the bootstrapping to plain old properties injection. That way, anyone using pinot can bootstrap the cluster with a set of defined yet deployment-specific credentials and then mutate in zookeeper as needed




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