[ https://issues.apache.org/jira/browse/KAFKA-8774?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Randall Hauch resolved KAFKA-8774. ---------------------------------- Resolution: Fixed Reviewer: Randall Hauch Fix Version/s: 2.3.1 2.4.0 2.2.2 2.1.2 2.0.2 [~wicknicks] did a great job identifying the root cause, which involved how the AbstractHerder to correctly identifies task configs that contain variables for externalized secrets. The original method incorrectly used `matcher.matches()` instead of `matcher.find()`. The former method expects the entire string to match the regex, whereas the second one can find a pattern anywhere within the input string (which fits this use case more correctly). This is why the problem is only in the tasks endpoint (no other endpoints) when connector configs contain externalized secret variables _plus additional characters_. If a config value contains only the variable, the secret is not exposed on this task endpoint. Arjun added unit tests to cover various cases of a config with externalized secrets, and updated system tests to cover case where config value contains additional characters besides secret that requires regex pattern to be found anywhere in the string (as opposed to complete match). Merged back to the `2.0` branch, which was when [KIP-297 and externalized secrets](https://cwiki.apache.org/confluence/display/KAFKA/KIP-297%3A+Externalizing+Secrets+for+Connect+Configurations) were introduced. > Connect REST API exposes plaintext secrets in tasks endpoint if config value > contains additional characters > ----------------------------------------------------------------------------------------------------------- > > Key: KAFKA-8774 > URL: https://issues.apache.org/jira/browse/KAFKA-8774 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect > Affects Versions: 2.3.0 > Reporter: Oleksandr Diachenko > Assignee: Arjun Satish > Priority: Critical > Fix For: 2.0.2, 2.1.2, 2.2.2, 2.4.0, 2.3.1 > > > I have configured a Connector to use externalized secrets, and the following > endpoint returns secrets in the externalized form: > {code:java} > curl localhost:8083/connectors/foobar|jq > {code} > {code:java} > { > "name": "foobar", > "config": { > "connector.class": "io.confluent.connect.s3.S3SinkConnector", > ... > "consumer.override.sasl.jaas.config": > "org.apache.kafka.common.security.plain.PlainLoginModule required > username=\"${file:/some/secret/path/secrets.properties:kafka.api.key}\" > password=\"${file:/some/secret/path/secrets.properties:kafka.api.secret}\";", > "admin.override.sasl.jaas.config": > "org.apache.kafka.common.security.plain.PlainLoginModule required > username=\"${file:/some/secret/path/secrets.properties:kafka.api.key}\" > password=\"${file:/some/secret/path/secrets.properties:kafka.api.secret}\";", > "consumer.sasl.jaas.config": > "org.apache.kafka.common.security.plain.PlainLoginModule required > username=\"${file:/some/secret/path/secrets.properties:kafka.api.key}\" > password=\"${file:/some/secret/path/secrets.properties:kafka.api.secret}\";", > "producer.override.sasl.jaas.config": > "org.apache.kafka.common.security.plain.PlainLoginModule required > username=\"${file:/some/secret/path/secrets.properties:kafka.api.key}\" > password=\"${file:/some/secret/path/secrets.properties:kafka.api.secret}\";", > "producer.sasl.jaas.config": > "org.apache.kafka.common.security.plain.PlainLoginModule required > username=\"${file:/some/secret/path/secrets.properties:kafka.api.key}\" > password=\"${file:/some/secret/path/secrets.properties:kafka.api.secret}\";", > ... > }, > "tasks": [ > { "connector": "foobar", "task": 0 } > ], > "type": "sink" > }{code} > But another endpoint returns secrets in plain text: > {code:java} > curl localhost:8083/connectors/foobar/tasks|jq > {code} > {code:java} > [ > { > "id": { > "connector": "lcc-kgkpm", > "task": 0 > }, > "config": { > "connector.class": "io.confluent.connect.s3.S3SinkConnector", > ... > "errors.log.include.messages": "true", > "flush.size": "1000", > "consumer.override.sasl.jaas.config": > "org.apache.kafka.common.security.plain.PlainLoginModule required > username=\"OOPS\" password=\"SURPRISE\";", > "admin.override.sasl.jaas.config": > "org.apache.kafka.common.security.plain.PlainLoginModule required > username=\"OOPS\" password=\"SURPRISE\";", > "consumer.sasl.jaas.config": > "org.apache.kafka.common.security.plain.PlainLoginModule required > username=\"OOPS\" password=\"SURPRISE\";", > "producer.override.sasl.jaas.config": > "org.apache.kafka.common.security.plain.PlainLoginModule required > username=\"OOPS\" password=\"SURPRISE\";", > "producer.sasl.jaas.config": > "org.apache.kafka.common.security.plain.PlainLoginModule required > username=\"OOPS\" password=\"SURPRISE\";", > ... > } > } > ] > {code} > > EDIT: This bug only shows up if the secrets are a substring in the config > value. If they form the entirety of the config value, then the secrets are > hidden at the /tasks endpoints. -- This message was sent by Atlassian JIRA (v7.6.14#76016)