[ https://issues.apache.org/jira/browse/HADOOP-19639?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18009892#comment-18009892 ]
ASF GitHub Bot commented on HADOOP-19639: ----------------------------------------- abstractdog commented on code in PR #7827: URL: https://github.com/apache/hadoop/pull/7827#discussion_r2230573481 ########## hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/SecretManagerConfig.java: ########## @@ -0,0 +1,133 @@ +/** + * 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 + * <p> + * http://www.apache.org/licenses/LICENSE-2.0 + * <p> + * 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.hadoop.security.token; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.crypto.KeyGenerator; +import javax.crypto.Mac; +import java.security.NoSuchAlgorithmException; + +/** + * Provides configuration and utility methods for managing cryptographic key generation + * and message authentication code (MAC) generation using specified algorithms and key lengths. + * <p> + * This class supports static access to the selected cryptographic algorithm and key length, + * and provides methods to create configured {@link javax.crypto.KeyGenerator} + * and {@link javax.crypto.Mac} instances. + * The configuration is initialized statically from a provided {@link Configuration} object. + * <p> + * The {@link SecretManager} has some static method, so static configuration is required + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class SecretManagerConfig { + private static final Logger LOG = LoggerFactory.getLogger(SecretManagerConfig.class); + private static String selectedAlgorithm; + private static int selectedLength; + private static boolean initialized; + + static { + update(new Configuration()); + } + + private SecretManagerConfig() { + } + + /** + * Updates the selected cryptographic algorithm and key length using the provided + * Hadoop {@link Configuration}. This method reads the values for + * {@code HADOOP_SECURITY_SECRET_MANAGER_KEY_GENERATOR_ALGORITHM_KEY} and + * {@code HADOOP_SECURITY_SECRET_MANAGER_KEY_LENGTH_KEY}, or uses default values if not set. + * + * @param conf the configuration object containing cryptographic settings + */ + public static synchronized void update(Configuration conf) { + if (initialized) { + LOG.warn( + "Keygen or Mac was already initialized with older config, those will not be updated"); + } + selectedAlgorithm = conf.get( + CommonConfigurationKeysPublic.HADOOP_SECURITY_SECRET_MANAGER_KEY_GENERATOR_ALGORITHM_KEY, + CommonConfigurationKeysPublic.HADOOP_SECURITY_SECRET_MANAGER_KEY_GENERATOR_ALGORITHM_DEFAULT); + LOG.debug("Selected hash algorithm: {}", selectedAlgorithm); + selectedLength = conf.getInt( + CommonConfigurationKeysPublic.HADOOP_SECURITY_SECRET_MANAGER_KEY_LENGTH_KEY, + CommonConfigurationKeysPublic.HADOOP_SECURITY_SECRET_MANAGER_KEY_LENGTH_DEFAULT); + LOG.debug("Selected hash key length: {}", selectedLength); + } + + /** + * Returns the currently selected cryptographic algorithm. + * + * @return the name of the selected algorithm + */ + public static synchronized String getSelectedAlgorithm() { + return selectedAlgorithm; + } + + /** + * Returns the currently selected key length in bits. + * + * @return the selected key length + */ + public static synchronized int getSelectedLength() { + return selectedLength; + } + + /** + * Creates a new {@link KeyGenerator} instance configured with the currently selected + * algorithm and key length. + * + * @return a new {@code KeyGenerator} instance + * @throws IllegalArgumentException if the specified algorithm is not available + */ + public static synchronized KeyGenerator createKeyGenerator() { + LOG.debug("Creating key generator instance {}, {}", selectedAlgorithm, selectedLength); + initialized = true; + try { + KeyGenerator keyGen = KeyGenerator.getInstance(selectedAlgorithm); + keyGen.init(selectedLength); + return keyGen; + } catch (NoSuchAlgorithmException nsa) { + throw new IllegalArgumentException("Can't find " + selectedAlgorithm, nsa); + } + } + + /** + * Creates a new {@link Mac} instance using the currently selected algorithm. + * + * @return a new {@code Mac} instance + * @throws IllegalArgumentException if the specified algorithm is not available + */ + public static synchronized Mac createMac() { + LOG.debug("Creating mac instance {}", selectedAlgorithm); Review Comment: include thread id in this message (regardless of what the logging library does, we might want to have this information) > SecretManager configuration at runtime > -------------------------------------- > > Key: HADOOP-19639 > URL: https://issues.apache.org/jira/browse/HADOOP-19639 > Project: Hadoop Common > Issue Type: Improvement > Components: hadoop-common > Affects Versions: 3.5.0 > Reporter: Bence Kosztolnik > Assignee: Bence Kosztolnik > Priority: Major > Labels: pull-request-available > > In case of TEZ *DAGAppMaster* the Hadoop *SecretManager* code can not read > yarn config xml file, therefore the SELECTED_ALGORITHM and SELECTED_LENGTH > variables in SecretManager can not be set at runtime. > This can results with the following exception in FIPS environment: > {code:java} > java.security.InvalidParameterException: Key size for HMAC must be at least > 112 bits in approved mode: SHA-1/HMAC > at > com.safelogic.cryptocomply.fips.core/com.safelogic.cryptocomply.jcajce.provider.BaseKeyGenerator.engineInit(Unknown > Source) > at java.base/javax.crypto.KeyGenerator.init(KeyGenerator.java:540) > at java.base/javax.crypto.KeyGenerator.init(KeyGenerator.java:517) > at > org.apache.hadoop.security.token.SecretManager.<init>(SecretManager.java:157) > at > org.apache.hadoop.yarn.security.client.BaseClientToAMTokenSecretManager.<init>(BaseClientToAMTokenSecretManager.java:38) > at > org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager.<init>(ClientToAMTokenSecretManager.java:46) > at > org.apache.tez.common.security.TezClientToAMTokenSecretManager.<init>(TezClientToAMTokenSecretManager.java:33) > at > org.apache.tez.dag.app.DAGAppMaster.serviceInit(DAGAppMaster.java:493) > at > org.apache.hadoop.service.AbstractService.init(AbstractService.java:164) > at org.apache.tez.dag.app.DAGAppMaster$9.run(DAGAppMaster.java:2649) > at java.base/java.security.AccessController.doPrivileged(Native Method) > at java.base/javax.security.auth.Subject.doAs(Subject.java:423) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1910) > at > org.apache.tez.dag.app.DAGAppMaster.initAndStartAppMaster(DAGAppMaster.java:2646) > at org.apache.tez.dag.app.DAGAppMaster.main(DAGAppMaster.java:2440) > {code} > To mitigate the problem we should provide some ability for the component to > be able to modify the configuration without corresponding config files on > class path. -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org