Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-18371 TopicBasedRemoteLogMetadataManagerConfig exposes sensitive configuration data in logs #18349

Open
wants to merge 1 commit into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package org.apache.kafka.server.log.remote.metadata.storage;

import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.admin.AdminClientConfig;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.config.ConfigDef;
Expand All @@ -33,6 +34,7 @@
import static org.apache.kafka.common.config.ConfigDef.Type.INT;
import static org.apache.kafka.common.config.ConfigDef.Type.LONG;
import static org.apache.kafka.common.config.ConfigDef.Type.SHORT;
import static org.apache.kafka.common.utils.ConfigUtils.configMapToRedactedString;

/**
* This class defines the configuration of topic based {@link org.apache.kafka.server.log.remote.storage.RemoteLogMetadataManager} implementation.
Expand Down Expand Up @@ -227,9 +229,9 @@ public String toString() {
", metadataTopicReplicationFactor=" + metadataTopicReplicationFactor +
", initializationRetryMaxTimeoutMs=" + initializationRetryMaxTimeoutMs +
", initializationRetryIntervalMs=" + initializationRetryIntervalMs +
", commonProps=" + commonProps +
", consumerProps=" + consumerProps +
", producerProps=" + producerProps +
", commonProps=" + configMapToRedactedString(commonProps, AdminClientConfig.configDef()) +
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

commonProps used to create AdminClient only

", consumerProps=" + configMapToRedactedString(consumerProps, ConsumerConfig.configDef()) +
", producerProps=" + configMapToRedactedString(producerProps, ProducerConfig.configDef()) +
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please add a unit test to verify that no sensitive strings are output

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@chia7712 thanks for the review! Updated test to include validations

'}';
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,12 @@
*/
package org.apache.kafka.server.log.remote.metadata.storage;

import java.util.Arrays;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.config.SaslConfigs;
import org.apache.kafka.common.config.SslConfigs;
import org.apache.kafka.test.TestUtils;

import org.junit.jupiter.api.Test;
Expand All @@ -36,6 +39,7 @@
import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_PROP;
import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_RETENTION_MS_PROP;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;

public class TopicBasedRemoteLogMetadataManagerConfigTest {
private static final String BOOTSTRAP_SERVERS = "localhost:2222";
Expand Down Expand Up @@ -107,6 +111,35 @@ public void testCommonProducerConsumerOverridesConfig() {
assertEquals(overriddenConsumerPropValue, rlmmConfig.consumerProperties().get(overrideEntry.getKey()));
}

@Test
void verifyToStringRedactsSensitiveConfigurations() {
Map<String, Object> commonClientConfig = new HashMap<>();
commonClientConfig.put(CommonClientConfigs.RETRIES_CONFIG, 10);
commonClientConfig.put(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG, 1000L);
commonClientConfig.put(CommonClientConfigs.METADATA_MAX_AGE_CONFIG, 60000L);
addPasswordTypeConfigurationProperties(commonClientConfig);

Map<String, Object> producerConfig = new HashMap<>();
producerConfig.put(ProducerConfig.ACKS_CONFIG, "all");
addPasswordTypeConfigurationProperties(producerConfig);

Map<String, Object> consumerConfig = new HashMap<>();
consumerConfig.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false);
addPasswordTypeConfigurationProperties(consumerConfig);

Map<String, Object> props = createValidConfigProps(commonClientConfig, producerConfig, consumerConfig);

// Check for topic properties
TopicBasedRemoteLogMetadataManagerConfig rlmmConfig = new TopicBasedRemoteLogMetadataManagerConfig(props);

String configString = rlmmConfig.toString();
assertMaskedSensitiveConfigurations(configString);
//verify not redacted properties present
assertTrue(configString.contains("retries=10"));
assertTrue(configString.contains("acks=\"all\""));
assertTrue(configString.contains("enable.auto.commit=false"));
}

private Map<String, Object> createValidConfigProps(Map<String, Object> commonClientConfig,
Map<String, Object> producerConfig,
Map<String, Object> consumerConfig) {
Expand All @@ -132,4 +165,31 @@ private Map<String, Object> createValidConfigProps(Map<String, Object> commonCli
}
return props;
}

/**
* Sample properties marked with {@link org.apache.kafka.common.config.ConfigDef.Type#PASSWORD} in the configuration.
*/
private void addPasswordTypeConfigurationProperties(Map<String, Object> config) {
config.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, "keystorePassword");
config.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, "keyPassword");
config.put(SslConfigs.SSL_KEYSTORE_KEY_CONFIG, "keystoreKey");
config.put(SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG, "keystoreCertificate");
config.put(SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG, "truststoreCertificate");
config.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "truststorePassword");
config.put(SaslConfigs.SASL_JAAS_CONFIG, "saslJaas");
}

private void assertMaskedSensitiveConfigurations(String configString) {
String[] sensitiveConfigKeys = {
SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG,
SslConfigs.SSL_KEY_PASSWORD_CONFIG,
SslConfigs.SSL_KEYSTORE_KEY_CONFIG,
SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG,
SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG,
SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG,
SaslConfigs.SASL_JAAS_CONFIG
};
Arrays.stream(sensitiveConfigKeys)
.forEach(config -> assertTrue(configString.contains(config + "=(redacted)")));
}
}
Loading