mirror of https://github.com/apache/kafka.git
KAFKA-9308: Reworded the ssl part of the security documentation (#8009)
Reworded the ssl part of the security documentation to fix various issues (mainly as noted by this jira, the problem that SAN extension values are not copied to certificates) and add some recommendations. Reviewers: Mickael Maison <mickael.maison@gmail.com>
This commit is contained in:
parent
9d53ad794d
commit
30ab2297f1
|
@ -37,114 +37,291 @@
|
|||
The guides below explain how to configure and use the security features in both clients and brokers.
|
||||
|
||||
<h3><a id="security_ssl" href="#security_ssl">7.2 Encryption and Authentication using SSL</a></h3>
|
||||
Apache Kafka allows clients to connect over SSL. By default, SSL is disabled but can be turned on as needed.
|
||||
Apache Kafka allows clients to use SSL for encryption of traffic as well as authentication. By default, SSL is disabled but can be turned on if needed.
|
||||
The following paragraphs explain in detail how to set up your own PKI infrastructure, use it to create certificates and configure Kafka to use these.
|
||||
|
||||
<ol>
|
||||
<li><h4><a id="security_ssl_key" href="#security_ssl_key">Generate SSL key and certificate for each Kafka broker</a></h4>
|
||||
The first step of deploying one or more brokers with the SSL support is to generate the key and the certificate for each machine in the cluster. You can use Java's keytool utility to accomplish this task.
|
||||
We will generate the key into a temporary keystore initially so that we can export and sign it later with CA.
|
||||
The first step of deploying one or more brokers with SSL support is to generate a public/private keypair for every server.
|
||||
Since Kafka expects all keys and certificates to be stored in keystores we will use Java's keytool command for this task.
|
||||
The tool supports two different keystore formats, the Java specific jks format which has been deprecated by now, as well as PKCS12.
|
||||
PKCS12 is the default format as of Java version 9, to ensure this format is being used regardless of the Java version in use all following
|
||||
commands explicitly specify the PKCS12 format.
|
||||
<pre class="brush: bash;">
|
||||
keytool -keystore server.keystore.jks -alias localhost -validity {validity} -genkey -keyalg RSA</pre>
|
||||
|
||||
keytool -keystore {keystorefile} -alias localhost -validity {validity} -genkey -keyalg RSA -storetype pkcs12
|
||||
</pre>
|
||||
You need to specify two parameters in the above command:
|
||||
<ol>
|
||||
<li>keystore: the keystore file that stores the certificate. The keystore file contains the private key of the certificate; therefore, it needs to be kept safely.</li>
|
||||
<li>validity: the valid time of the certificate in days.</li>
|
||||
</ol>
|
||||
<br>
|
||||
<li>keystorefile: the keystore file that stores the keys (and later the certificate) for this broker. The keystore file contains the private
|
||||
and public keys of this broker, therefore it needs to be kept safe. Ideally this step is run on the Kafka broker that the key will be
|
||||
used on, as this key should never be transmitted/leave the server that it is intended for.</li>
|
||||
<li>validity: the valid time of the key in days. Please note that this differs from the validity period for the certificate, which
|
||||
will be determined in <a href ="#security_ssl_signing">Signing the certificate</a>. You can use the same key to request multiple
|
||||
certificates: if your key has a validity of 10 years, but your CA will only sign certificates that are valid for one year, you
|
||||
can use the same key with 10 certificates over time.</li>
|
||||
</ol><br>
|
||||
To obtain a certificate that can be used with the private key that was just created a certificate signing request needs to be created. This
|
||||
signing request, when signed by a trusted CA results in the actual certificate which can then be installed in the keystore and used for
|
||||
authentication purposes.<br>
|
||||
To generate certificate signing requests run the following command for all server keystores created so far.
|
||||
|
||||
<pre class="brush: bash;">
|
||||
keytool -keystore server.keystore.jks -alias localhost -validity {validity} -genkey -keyalg RSA -destkeystoretype pkcs12 -ext SAN=DNS:{FQDN},IP:{IPADDRESS1}
|
||||
</pre>
|
||||
This command assumes that you want to add hostname information to the certificate, if this is not the case, you can omit the extension parameter <code>-ext SAN=DNS:{FQDN},IP:{IPADDRESS1}</code>. Please see below for more information on this.
|
||||
|
||||
<h5>Host Name Verification</h5>
|
||||
Host name verification, when enabled, is the process of checking attributes from the certificate that is presented by the server you are
|
||||
connecting to against the actual hostname or ip address of that server to ensure that you are indeed connecting to the correct server.<br>
|
||||
The main reason for this check is to prevent man-in-the-middle attacks.
|
||||
|
||||
For Kafka, this check has been disabled by default for a long time, but as of Kafka 2.0.0 host name verification of servers is enabled by default
|
||||
for client connections as well as inter-broker connections.<br>
|
||||
Server host name verification may be disabled by setting <code>ssl.endpoint.identification.algorithm</code> to an empty string.<br>
|
||||
For dynamically configured broker listeners, hostname verification may be disabled using <code>kafka-configs.sh</code>:<br>
|
||||
|
||||
<h5><a id="security_confighostname" href="#security_confighostname">Configuring Host Name Verification</a></h5>
|
||||
From Kafka version 2.0.0 onwards, host name verification of servers is enabled by default for client connections
|
||||
as well as inter-broker connections to prevent man-in-the-middle attacks. Server host name verification may be disabled
|
||||
by setting <code>ssl.endpoint.identification.algorithm</code> to an empty string. For example,
|
||||
<pre class="brush: text;"> ssl.endpoint.identification.algorithm=</pre>
|
||||
For dynamically configured broker listeners, hostname verification may be disabled using <code>kafka-configs.sh</code>.
|
||||
For example,
|
||||
<pre class="brush: text;">
|
||||
bin/kafka-configs.sh --bootstrap-server localhost:9093 --entity-type brokers --entity-name 0 --alter --add-config "listener.name.internal.ssl.endpoint.identification.algorithm="
|
||||
</pre>
|
||||
|
||||
For older versions of Kafka, <code>ssl.endpoint.identification.algorithm</code> is not defined by default, so host name
|
||||
verification is not performed. The property should be set to <code>HTTPS</code> to enable host name verification.
|
||||
<p><b>Note:</b></p>
|
||||
Normally there is no good reason to disable hostname verification apart from being the quickest way to "just get it to work" followed
|
||||
by the promise to "fix it later when there is more time"!<br>
|
||||
Getting hostname verification right is not that hard when done at the right time, but gets much harder once the cluster is up and
|
||||
running - do yourself a favor and do it now!
|
||||
|
||||
<pre class="brush: text;"> ssl.endpoint.identification.algorithm=HTTPS </pre>
|
||||
|
||||
Host name verification must be enabled to prevent man-in-the-middle attacks if server endpoints are not validated
|
||||
externally.
|
||||
|
||||
<h5><a id="security_configcerthostname" href="#security_configcerthstname">Configuring Host Name In Certificates</a></h5>
|
||||
If host name verification is enabled, clients will verify the server's fully qualified domain name (FQDN) against one of
|
||||
the following two fields:
|
||||
<p>If host name verification is enabled, clients will verify the server's fully qualified domain name (FQDN) or ip address against one of the following two fields:
|
||||
<ol>
|
||||
<li>Common Name (CN)
|
||||
<li>Subject Alternative Name (SAN)
|
||||
</ol>
|
||||
<br>
|
||||
Both fields are valid, RFC-2818 recommends the use of SAN however. SAN is also more flexible, allowing for multiple DNS entries to be declared. Another advantage is that the CN can be set to a more meaningful value for authorization purposes. To add a SAN field append the following argument <code> -ext SAN=DNS:{FQDN} </code> to the keytool command:
|
||||
<li>Common Name (CN)</li>
|
||||
<li><a href="https://tools.ietf.org/html/rfc5280#section-4.2.1.6">Subject Alternative Name (SAN)</a></li>
|
||||
</ol><br>
|
||||
While Kafka checks both fields, usage of the common name field for hostname verification has been
|
||||
<a href="https://tools.ietf.org/html/rfc2818#section-3.1">deprecated</a> since 2000 and should be avoided if possible. In addition the
|
||||
SAN field is much more flexible, allowing for multiple DNS and IP entries to be declared in a certificate.<br>
|
||||
Another advantage is that if the SAN field is used for hostname verification the common name can be set to a more meaningful value for
|
||||
authorization purposes. Since we need the SAN field to be contained in the signed certificate, it will be specified when generating the
|
||||
signing request. It can also be specified when generating the keypair, but this will not automatically be copied into the signing request.<br>
|
||||
|
||||
|
||||
To add a SAN field append the following argument <code> -ext SAN=DNS:{FQDN},IP:{IPADDRESS} </code> to the keytool command:
|
||||
<pre class="brush: bash;">
|
||||
keytool -keystore server.keystore.jks -alias localhost -validity {validity} -genkey -keyalg RSA -ext SAN=DNS:{FQDN}
|
||||
</pre>
|
||||
The following command can be run afterwards to verify the contents of the generated certificate:
|
||||
<pre class="brush: bash;">
|
||||
keytool -list -v -keystore server.keystore.jks
|
||||
keytool -keystore server.keystore.jks -alias localhost -validity {validity} -genkey -keyalg RSA -destkeystoretype pkcs12 -ext SAN=DNS:{FQDN},IP:{IPADDRESS1}
|
||||
</pre>
|
||||
</li>
|
||||
<li><h4><a id="security_ssl_ca" href="#security_ssl_ca">Creating your own CA</a></h4>
|
||||
After the first step, each machine in the cluster has a public-private key pair, and a certificate to identify the machine. The certificate, however, is unsigned, which means that an attacker can create such a certificate to pretend to be any machine.<p>
|
||||
Therefore, it is important to prevent forged certificates by signing them for each machine in the cluster. A certificate authority (CA) is responsible for signing certificates. CA works likes a government that issues passports—the government stamps (signs) each passport so that the passport becomes difficult to forge. Other governments verify the stamps to ensure the passport is authentic. Similarly, the CA signs the certificates, and the cryptography guarantees that a signed certificate is computationally difficult to forge. Thus, as long as the CA is a genuine and trusted authority, the clients have high assurance that they are connecting to the authentic machines.
|
||||
<pre class="brush: bash;">
|
||||
openssl req -new -x509 -keyout ca-key -out ca-cert -days 365</pre>
|
||||
|
||||
The generated CA is simply a public-private key pair and certificate, and it is intended to sign other certificates.<br>
|
||||
<li><h4><a id="security_ssl_ca" href="#security_ssl_ca">Creating your own CA</a></h4>
|
||||
After this step each machine in the cluster has a public/private key pair which can already be used to encrypt traffic and a certificate
|
||||
signing request, which is the basis for creating a certificate. To add authentication capabilities this signing request needs to be signed
|
||||
by a trusted authority, which will be created in this step.
|
||||
|
||||
<p>A certificate authority (CA) is responsible for signing certificates. CAs works likes a government that issues passports - the government
|
||||
stamps (signs) each passport so that the passport becomes difficult to forge. Other governments verify the stamps to ensure the passport is
|
||||
authentic. Similarly, the CA signs the certificates, and the cryptography guarantees that a signed certificate is computationally difficult to
|
||||
forge. Thus, as long as the CA is a genuine and trusted authority, the clients have a strong assurance that they are connecting to the authentic
|
||||
machines.
|
||||
|
||||
<p>For this guide we will be our own Certificate Authority. When setting up a production cluster in a corporate environment these certificates would
|
||||
usually be signed by a corporate CA that is trusted throughout the company. Please see <a href="#security_ssl_production">Common Pitfalls in
|
||||
Production</a> for some things to consider for this case.
|
||||
|
||||
<p>Due to a <a href="https://www.openssl.org/docs/man1.1.1/man1/x509.html#BUGS">bug</a> in OpenSSL, the x509 module will not copy requested
|
||||
extension fields from CSRs into the final certificate. Since we want the SAN extension to be present in our certificate to enable hostname
|
||||
verification, we'll use the <i>ca</i> module instead. This requires some additional configuration to be in place before we generate our
|
||||
CA keypair.<br>
|
||||
|
||||
Save the following listing into a file called openssl-ca.cnf and adjust the values for validity and common attributes as necessary.
|
||||
<pre class="brush: bash">
|
||||
HOME = .
|
||||
RANDFILE = $ENV::HOME/.rnd
|
||||
|
||||
####################################################################
|
||||
[ ca ]
|
||||
default_ca = CA_default # The default ca section
|
||||
|
||||
[ CA_default ]
|
||||
|
||||
base_dir = .
|
||||
certificate = $base_dir/cacert.pem # The CA certifcate
|
||||
private_key = $base_dir/cakey.pem # The CA private key
|
||||
new_certs_dir = $base_dir # Location for new certs after signing
|
||||
database = $base_dir/index.txt # Database index file
|
||||
serial = $base_dir/serial.txt # The current serial number
|
||||
|
||||
default_days = 1000 # How long to certify for
|
||||
default_crl_days = 30 # How long before next CRL
|
||||
default_md = sha256 # Use public key default MD
|
||||
preserve = no # Keep passed DN ordering
|
||||
|
||||
x509_extensions = ca_extensions # The extensions to add to the cert
|
||||
|
||||
email_in_dn = no # Don't concat the email in the DN
|
||||
copy_extensions = copy # Required to copy SANs from CSR to cert
|
||||
|
||||
####################################################################
|
||||
[ req ]
|
||||
default_bits = 4096
|
||||
default_keyfile = cakey.pem
|
||||
distinguished_name = ca_distinguished_name
|
||||
x509_extensions = ca_extensions
|
||||
string_mask = utf8only
|
||||
|
||||
####################################################################
|
||||
[ ca_distinguished_name ]
|
||||
countryName = Country Name (2 letter code)
|
||||
countryName_default = DE
|
||||
|
||||
stateOrProvinceName = State or Province Name (full name)
|
||||
stateOrProvinceName_default = Test Province
|
||||
|
||||
localityName = Locality Name (eg, city)
|
||||
localityName_default = Test Town
|
||||
|
||||
organizationName = Organization Name (eg, company)
|
||||
organizationName_default = Test Company
|
||||
|
||||
organizationalUnitName = Organizational Unit (eg, division)
|
||||
organizationalUnitName_default = Test Unit
|
||||
|
||||
commonName = Common Name (e.g. server FQDN or YOUR name)
|
||||
commonName_default = Test Name
|
||||
|
||||
emailAddress = Email Address
|
||||
emailAddress_default = test@test.com
|
||||
|
||||
####################################################################
|
||||
[ ca_extensions ]
|
||||
|
||||
subjectKeyIdentifier = hash
|
||||
authorityKeyIdentifier = keyid:always, issuer
|
||||
basicConstraints = critical, CA:true
|
||||
keyUsage = keyCertSign, cRLSign
|
||||
|
||||
####################################################################
|
||||
[ signing_policy ]
|
||||
countryName = optional
|
||||
stateOrProvinceName = optional
|
||||
localityName = optional
|
||||
organizationName = optional
|
||||
organizationalUnitName = optional
|
||||
commonName = supplied
|
||||
emailAddress = optional
|
||||
|
||||
####################################################################
|
||||
[ signing_req ]
|
||||
subjectKeyIdentifier = hash
|
||||
authorityKeyIdentifier = keyid,issuer
|
||||
basicConstraints = CA:FALSE
|
||||
keyUsage = digitalSignature, keyEncipherment
|
||||
</pre>
|
||||
|
||||
Then create a database and serial number file, these will be used to keep track of which certificates were signed with this CA. Both of
|
||||
these are simply text files that reside in the same directory as your CA keys.
|
||||
|
||||
<pre class="brush: bash;">
|
||||
echo 01 > serial.txt
|
||||
touch index.txt
|
||||
</pre>
|
||||
|
||||
With these steps done you are now ready to generate your CA that will be used to sign certificates later.
|
||||
|
||||
<pre class="brush: bash;">
|
||||
openssl req -x509 -config openssl-ca.cnf -newkey rsa:4096 -sha256 -nodes -out cacert.pem -outform PEM
|
||||
</pre>
|
||||
|
||||
The CA is simply a public/private key pair and certificate that is signed by itself, and is only intended to sign other certificates.<br>
|
||||
This keypair should be kept very safe, if someone gains access to it, they can create and sign certificates that will be trusted by your
|
||||
infrastructure, which means they will be able to impersonate anybody when connecting to any service that trusts this CA.<br>
|
||||
|
||||
The next step is to add the generated CA to the **clients' truststore** so that the clients can trust this CA:
|
||||
<pre class="brush: bash;">
|
||||
keytool -keystore client.truststore.jks -alias CARoot -import -file ca-cert</pre>
|
||||
keytool -keystore client.truststore.jks -alias CARoot -import -file ca-cert
|
||||
</pre>
|
||||
|
||||
<b>Note:</b> If you configure the Kafka brokers to require client authentication by setting ssl.client.auth to be "requested" or "required" on the <a href="#config_broker">Kafka brokers config</a> then you must provide a truststore for the Kafka brokers as well and it should have all the CA certificates that clients' keys were signed by.
|
||||
<b>Note:</b>
|
||||
If you configure the Kafka brokers to require client authentication by setting ssl.client.auth to be "requested" or "required" in the
|
||||
<a href="#config_broker">Kafka brokers config</a> then you must provide a truststore for the Kafka brokers as well and it should have
|
||||
all the CA certificates that clients' keys were signed by.
|
||||
<pre class="brush: bash;">
|
||||
keytool -keystore server.truststore.jks -alias CARoot -import -file ca-cert</pre>
|
||||
|
||||
In contrast to the keystore in step 1 that stores each machine's own identity, the truststore of a client stores all the certificates that the client should trust. Importing a certificate into one's truststore also means trusting all certificates that are signed by that certificate. As the analogy above, trusting the government (CA) also means trusting all passports (certificates) that it has issued. This attribute is called the chain of trust, and it is particularly useful when deploying SSL on a large Kafka cluster. You can sign all certificates in the cluster with a single CA, and have all machines share the same truststore that trusts the CA. That way all machines can authenticate all other machines.</li>
|
||||
keytool -keystore server.truststore.jks -alias CARoot -import -file ca-cert
|
||||
</pre>
|
||||
|
||||
In contrast to the keystore in step 1 that stores each machine's own identity, the truststore of a client stores all the certificates
|
||||
that the client should trust. Importing a certificate into one's truststore also means trusting all certificates that are signed by that
|
||||
certificate. As the analogy above, trusting the government (CA) also means trusting all passports (certificates) that it has issued. This
|
||||
attribute is called the chain of trust, and it is particularly useful when deploying SSL on a large Kafka cluster. You can sign all certificates
|
||||
in the cluster with a single CA, and have all machines share the same truststore that trusts the CA. That way all machines can authenticate all
|
||||
other machines.
|
||||
</li>
|
||||
<li><h4><a id="security_ssl_signing" href="#security_ssl_signing">Signing the certificate</a></h4>
|
||||
The next step is to sign all certificates generated by step 1 with the CA generated in step 2. First, you need to export the certificate from the keystore:
|
||||
<pre class="brush: bash;">
|
||||
keytool -keystore server.keystore.jks -alias localhost -certreq -file cert-file</pre>
|
||||
|
||||
Then sign it with the CA:
|
||||
<pre class="brush: bash;">
|
||||
openssl x509 -req -CA ca-cert -CAkey ca-key -in cert-file -out cert-signed -days {validity} -CAcreateserial -passin pass:{ca-password}</pre>
|
||||
openssl ca -config openssl-ca.cnf -policy signing_policy -extensions signing_req -out {server certificate} -infiles {certificate signing request}
|
||||
</pre>
|
||||
|
||||
Finally, you need to import both the certificate of the CA and the signed certificate into the keystore:
|
||||
<pre class="brush: bash;">
|
||||
keytool -keystore server.keystore.jks -alias CARoot -import -file ca-cert
|
||||
keytool -keystore server.keystore.jks -alias localhost -import -file cert-signed</pre>
|
||||
keytool -keystore {keystore} -alias CARoot -import -file {CA certificate}
|
||||
keytool -keystore {keystore} -alias localhost -import -file cert-signed
|
||||
</pre>
|
||||
|
||||
The definitions of the parameters are the following:
|
||||
<ol>
|
||||
<li>keystore: the location of the keystore</li>
|
||||
<li>ca-cert: the certificate of the CA</li>
|
||||
<li>ca-key: the private key of the CA</li>
|
||||
<li>ca-password: the passphrase of the CA</li>
|
||||
<li>cert-file: the exported, unsigned certificate of the server</li>
|
||||
<li>cert-signed: the signed certificate of the server</li>
|
||||
<li>CA certificate: the certificate of the CA</li>
|
||||
<li>certificate signing request: the csr created with the server key</li>
|
||||
<li>server certificate: the file to write the signed certificate of the server to</li>
|
||||
</ol>
|
||||
|
||||
Here is an example of a bash script with all above steps. Note that one of the commands assumes a password of `test1234`, so either use that password or edit the command before running it.
|
||||
<pre>
|
||||
#!/bin/bash
|
||||
#Step 1
|
||||
keytool -keystore server.keystore.jks -alias localhost -validity 365 -keyalg RSA -genkey
|
||||
#Step 2
|
||||
openssl req -new -x509 -keyout ca-key -out ca-cert -days 365
|
||||
keytool -keystore server.truststore.jks -alias CARoot -import -file ca-cert
|
||||
keytool -keystore client.truststore.jks -alias CARoot -import -file ca-cert
|
||||
#Step 3
|
||||
keytool -keystore server.keystore.jks -alias localhost -certreq -file cert-file
|
||||
openssl x509 -req -CA ca-cert -CAkey ca-key -in cert-file -out cert-signed -days 365 -CAcreateserial -passin pass:test1234
|
||||
keytool -keystore server.keystore.jks -alias CARoot -import -file ca-cert
|
||||
keytool -keystore server.keystore.jks -alias localhost -import -file cert-signed</pre></li>
|
||||
This will leave you with one truststore called <i>truststore.jks</i> - this can be the same for all clients and brokers and does not
|
||||
contain any sensitive information, so there is no need to secure this.<br>
|
||||
Additionally you will have one <i>server.keystore.jks</i> file per node which contains that nodes keys, certificate and your CAs certificate,
|
||||
please refer to <a href="#security_configbroker">Configuring Kafka Brokers</a> and <a href="#security_configclients">Configuring Kafka Clients</a>
|
||||
for information on how to use these files.
|
||||
|
||||
<p>For some tooling assistance on this topic, please check out the <a href="https://github.com/OpenVPN/easy-rsa">easyRSA</a> project which has
|
||||
extensive scripting in place to help with these steps.</p>
|
||||
|
||||
</li>
|
||||
<li><h4><a id="security_ssl_production" href="#security_ssl_production">Common Pitfalls in Production</a></h4>
|
||||
The above paragraphs show the process to create your own CA and use it to sign certificates for your cluster.
|
||||
While very useful for sandbox, dev, test, and similar systems, this is usually not the correct process to create certificates for a production
|
||||
cluster in a corporate environment.
|
||||
Enterprises will normally operate their own CA and users can send in CSRs to be signed with this CA, which has the benefit of users not being
|
||||
responsible to keep the CA secure as well as a central authority that everybody can trust.
|
||||
However it also takes away a lot of control over the process of signing certificates from the user. Quite often the persons operating corporate
|
||||
CAs will apply tight restrictions on certificates that can cause issues when trying to use these certificates with Kafka.
|
||||
|
||||
<ol>
|
||||
<li><b><a href="https://tools.ietf.org/html/rfc5280#section-4.2.1.12">Extended Key Usage</a></b><br>Certificates may contain an extension
|
||||
field that controls the purpose for which the certificate can be used. If this field is empty, there are no restricitions on the usage,
|
||||
but if any usage is specified in here, valid SSL implementations have to enforce these usages.<br>
|
||||
Relevant usages for Kafka are:
|
||||
<ul>
|
||||
<li>Client authentication</li>
|
||||
<li>Server authentication</li>
|
||||
</ul>
|
||||
Kafka brokers need both these usages to be allowed, as for intra-cluster communication every broker will behave as both the client and
|
||||
the server towards other brokers. It is not uncommon for corporate CAs to have a signing profile for webservers and use this for Kafka as
|
||||
well, which will only contain the <i>serverAuth</i> usage value and cause the SSL handshake to fail.
|
||||
</li>
|
||||
<li><b>Intermediate Certificates</b><br>
|
||||
Corporate Root CAs are often kept offline for security reasons. To enable day-to-day usage, so called intermediate CAs are created, which
|
||||
are then used to sign the final certificates. When importing a certificate into the keystore that was signed by an intermediate CA it is
|
||||
necessarry to provide the entire chain of trust up to the root CA. This can be done by simply <i>cat</i>ing the certificate files into one
|
||||
combined certificate file and then importing this with keytool.
|
||||
</li>
|
||||
<li><b>Failure to copy extension fields</b><br>
|
||||
CA operators are often hesitant to copy and requested extension fields from CSRs and prefer to specify these themselves as this makes it
|
||||
harder for a malicious party to obtain certificates with potentially misleading or fraudulent values.
|
||||
It is adviseable to double check signed certificates, whether these contain all requested SAN fields to enable proper hostname verification.
|
||||
The following command can be used to print certificate details to the console, which should be compared with what was originally requested:
|
||||
<pre class="brush: bash;">
|
||||
openssl x509 -in certificate.crt -text -noout
|
||||
</pre>
|
||||
</li>
|
||||
</ol>
|
||||
</li>
|
||||
<li><h4><a id="security_configbroker" href="#security_configbroker">Configuring Kafka Brokers</a></h4>
|
||||
Kafka Brokers support listening for connections on multiple ports.
|
||||
We need to configure the following property in server.properties, which must have one or more comma-separated values:
|
||||
|
|
Loading…
Reference in New Issue