This is an automated email from the ASF dual-hosted git repository.

mimaison pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 74be72a559d MINOR: Various fixes in the docs (#14914)
74be72a559d is described below

commit 74be72a559d10c2bf7e88490e5b53e09fd186e7b
Author: Mickael Maison <mimai...@users.noreply.github.com>
AuthorDate: Mon Dec 4 22:06:49 2023 +0100

    MINOR: Various fixes in the docs (#14914)
    
    - Only use https links
    - Fix broken HTML tags
    - Replace usage of <tt> which is deprecated with <code>
    - Replace hardcoded version numbers
    
    Reviewers: Chris Egerton <fearthecel...@gmail.com>, Greg Harris 
<gharris1...@gmail.com>
---
 docs/configuration.html  |   2 +-
 docs/connect.html        |   4 +-
 docs/design.html         |  36 ++++-----
 docs/implementation.html |   2 +-
 docs/ops.html            |  53 ++++++------
 docs/quickstart.html     |   2 +-
 docs/security.html       | 204 +++++++++++++++++++++++------------------------
 docs/toc.html            |  12 +--
 docs/upgrade.html        |   4 +-
 docs/uses.html           |   6 +-
 10 files changed, 161 insertions(+), 164 deletions(-)

diff --git a/docs/configuration.html b/docs/configuration.html
index 03038223b21..7bcb097b944 100644
--- a/docs/configuration.html
+++ b/docs/configuration.html
@@ -16,7 +16,7 @@
 -->
 
 <script id="configuration-template" type="text/x-handlebars-template">
-  Kafka uses key-value pairs in the <a 
href="http://en.wikipedia.org/wiki/.properties";>property file format</a> for 
configuration. These values can be supplied either from a file or 
programmatically.
+  Kafka uses key-value pairs in the <a 
href="https://en.wikipedia.org/wiki/.properties";>property file format</a> for 
configuration. These values can be supplied either from a file or 
programmatically.
 
   <h3 class="anchor-heading"><a id="brokerconfigs" class="anchor-link"></a><a 
href="#brokerconfigs">3.1 Broker Configs</a></h3>
 
diff --git a/docs/connect.html b/docs/connect.html
index 10b423aa380..aa8cf9e35a0 100644
--- a/docs/connect.html
+++ b/docs/connect.html
@@ -318,7 +318,7 @@ listeners=http://localhost:8080,https://localhost:8443</pre>
             <ul>
                 <li><code>GET /connectors/{name}/offsets</code> - get the 
current offsets for a connector</li>
                 <li><code>DELETE /connectors/{name}/offsets</code> - reset the 
offsets for a connector. The connector must exist and must be in the stopped 
state (see <a href="#connect_stopconnector"><code>PUT 
/connectors/{name}/stop</code></a>)</li>
-                <li><code>PATCH /connectors/{name}/offsets</code> - alter the 
offsets for a connector. The connector must exist and must be in the stopped 
state (see <a href="#connect_stopconnector"><code>PUT 
/connectors/{name}/stop</code></a>). The request body should be a JSON object 
containing a JSON array <code>offsets</code> field, similar to the response 
body of the <code>GET /connectors/{name}/offsets</code> endpoint</li>
+                <li><code>PATCH /connectors/{name}/offsets</code> - alter the 
offsets for a connector. The connector must exist and must be in the stopped 
state (see <a href="#connect_stopconnector"><code>PUT 
/connectors/{name}/stop</code></a>). The request body should be a JSON object 
containing a JSON array <code>offsets</code> field, similar to the response 
body of the <code>GET /connectors/{name}/offsets</code> endpoint.
                 An example request body for the 
<code>FileStreamSourceConnector</code>:
                 <pre class="line-numbers"><code class="json">
 {
@@ -357,7 +357,7 @@ listeners=http://localhost:8080,https://localhost:8443</pre>
   ]
 }
                 </code></pre>
-                The "offset" field may be null to reset the offset for a 
specific partition (applicable to both source and sink connectors). Note that 
the request body format depends on the connector implementation in the case of 
source connectors, whereas there is a common format across all sink connectors.
+                The "offset" field may be null to reset the offset for a 
specific partition (applicable to both source and sink connectors). Note that 
the request body format depends on the connector implementation in the case of 
source connectors, whereas there is a common format across all sink 
connectors.</li>
             </ul>
         </li>
     </ul>
diff --git a/docs/design.html b/docs/design.html
index a2edc552413..18f78d044c4 100644
--- a/docs/design.html
+++ b/docs/design.html
@@ -38,11 +38,11 @@
     Kafka relies heavily on the filesystem for storing and caching messages. 
There is a general perception that "disks are slow" which makes people 
skeptical that a persistent structure can offer competitive performance.
     In fact disks are both much slower and much faster than people expect 
depending on how they are used; and a properly designed disk structure can 
often be as fast as the network.
     <p>
-    The key fact about disk performance is that the throughput of hard drives 
has been diverging from the latency of a disk seek for the last decade. As a 
result the performance of linear writes on a <a 
href="http://en.wikipedia.org/wiki/Non-RAID_drive_architectures";>JBOD</a>
+    The key fact about disk performance is that the throughput of hard drives 
has been diverging from the latency of a disk seek for the last decade. As a 
result the performance of linear writes on a <a 
href="https://en.wikipedia.org/wiki/Non-RAID_drive_architectures";>JBOD</a>
     configuration with six 7200rpm SATA RAID-5 array is about 600MB/sec but 
the performance of random writes is only about 100k/sec&mdash;a difference of 
over 6000X. These linear reads and writes are the most
     predictable of all usage patterns, and are heavily optimized by the 
operating system. A modern operating system provides read-ahead and 
write-behind techniques that prefetch data in large block multiples and
-    group smaller logical writes into large physical writes. A further 
discussion of this issue can be found in this <a 
href="http://queue.acm.org/detail.cfm?id=1563874";>ACM Queue article</a>; they 
actually find that
-    <a 
href="http://deliveryimages.acm.org/10.1145/1570000/1563874/jacobs3.jpg";>sequential
 disk access can in some cases be faster than random memory access!</a>
+    group smaller logical writes into large physical writes. A further 
discussion of this issue can be found in this <a 
href="https://queue.acm.org/detail.cfm?id=1563874";>ACM Queue article</a>; they 
actually find that
+    <a 
href="https://deliveryimages.acm.org/10.1145/1570000/1563874/jacobs3.jpg";>sequential
 disk access can in some cases be faster than random memory access!</a>
     <p>
     To compensate for this performance divergence, modern operating systems 
have become increasingly aggressive in their use of main memory for disk 
caching. A modern OS will happily divert <i>all</i> free memory to
     disk caching with little performance penalty when the memory is reclaimed. 
All disk reads and writes will go through this unified cache. This feature 
cannot easily be turned off without using direct I/O, so even
@@ -64,7 +64,7 @@
     This suggests a design which is very simple: rather than maintain as much 
as possible in-memory and flush it all out to the filesystem in a panic when we 
run out of space, we invert that. All data is immediately
     written to a persistent log on the filesystem without necessarily flushing 
to disk. In effect this just means that it is transferred into the kernel's 
pagecache.
     <p>
-    This style of pagecache-centric design is described in an <a 
href="http://varnish-cache.org/wiki/ArchitectNotes";>article</a> on the design 
of Varnish here (along with a healthy dose of arrogance).
+    This style of pagecache-centric design is described in an <a 
href="https://varnish-cache.org/wiki/ArchitectNotes";>article</a> on the design 
of Varnish here (along with a healthy dose of arrogance).
 
     <h4 class="anchor-heading"><a id="design_constanttime" 
class="anchor-link"></a><a href="#design_constanttime">Constant Time 
Suffices</a></h4>
     <p>
@@ -107,7 +107,7 @@
     <p>
     The message log maintained by the broker is itself just a directory of 
files, each populated by a sequence of message sets that have been written to 
disk in the same format used by the producer and consumer.
     Maintaining this common format allows optimization of the most important 
operation: network transfer of persistent log chunks. Modern unix operating 
systems offer a highly optimized code path for transferring data
-    out of pagecache to a socket; in Linux this is done with the <a 
href="http://man7.org/linux/man-pages/man2/sendfile.2.html";>sendfile system 
call</a>.
+    out of pagecache to a socket; in Linux this is done with the <a 
href="https://man7.org/linux/man-pages/man2/sendfile.2.html";>sendfile system 
call</a>.
     <p>
     To understand the impact of sendfile, it is important to understand the 
common data path for transfer of data from file to socket:
     <ol>
@@ -160,7 +160,7 @@
     to accumulate no more than a fixed number of messages and to wait no 
longer than some fixed latency bound (say 64k or 10 ms). This allows the 
accumulation of more bytes to send, and few larger I/O operations on the
     servers. This buffering is configurable and gives a mechanism to trade off 
a small amount of additional latency for better throughput.
     <p>
-    Details on <a href="#producerconfigs">configuration</a> and the <a 
href="http://kafka.apache.org/082/javadoc/index.html?org/apache/kafka/clients/producer/KafkaProducer.html";>api</a>
 for the producer can be found
+    Details on <a href="#producerconfigs">configuration</a> and the <a 
href="https://kafka.apache.org/082/javadoc/index.html?org/apache/kafka/clients/producer/KafkaProducer.html";>api</a>
 for the producer can be found
     elsewhere in the documentation.
 
     <h3 class="anchor-heading"><a id="theconsumer" class="anchor-link"></a><a 
href="#theconsumer">4.5 The Consumer</a></h3>
@@ -171,8 +171,8 @@
     <h4 class="anchor-heading"><a id="design_pull" class="anchor-link"></a><a 
href="#design_pull">Push vs. pull</a></h4>
     <p>
     An initial question we considered is whether consumers should pull data 
from brokers or brokers should push data to the consumer. In this respect Kafka 
follows a more traditional design, shared by most messaging
-    systems, where data is pushed to the broker from the producer and pulled 
from the broker by the consumer. Some logging-centric systems, such as <a 
href="http://github.com/facebook/scribe";>Scribe</a> and
-    <a href="http://flume.apache.org/";>Apache Flume</a>, follow a very 
different push-based path where data is pushed downstream. There are pros and 
cons to both approaches. However, a push-based system has difficulty
+    systems, where data is pushed to the broker from the producer and pulled 
from the broker by the consumer. Some logging-centric systems, such as <a 
href="https://github.com/facebook/scribe";>Scribe</a> and
+    <a href="https://flume.apache.org/";>Apache Flume</a>, follow a very 
different push-based path where data is pushed downstream. There are pros and 
cons to both approaches. However, a push-based system has difficulty
     dealing with diverse consumers as the broker controls the rate at which 
data is transferred. The goal is generally for the consumer to be able to 
consume at the maximum possible rate; unfortunately, in a push
     system this means the consumer tends to be overwhelmed when its rate of 
consumption falls below the rate of production (a denial of service attack, in 
essence). A pull-based system has the nicer property that
     the consumer simply falls behind and catches up when it can. This can be 
mitigated with some kind of backoff protocol by which the consumer can indicate 
it is overwhelmed, but getting the rate of transfer to
@@ -364,7 +364,7 @@
     <h4><a id="design_replicatedlog" href="#design_replicatedlog">Replicated 
Logs: Quorums, ISRs, and State Machines (Oh my!)</a></h4>
 
     At its heart a Kafka partition is a replicated log. The replicated log is 
one of the most basic primitives in distributed data systems, and there are 
many approaches for implementing one. A replicated log can be
-    used by other systems as a primitive for implementing other distributed 
systems in the <a 
href="http://en.wikipedia.org/wiki/State_machine_replication";>state-machine 
style</a>.
+    used by other systems as a primitive for implementing other distributed 
systems in the <a 
href="https://en.wikipedia.org/wiki/State_machine_replication";>state-machine 
style</a>.
     <p>
     A replicated log models the process of coming into consensus on the order 
of a series of values (generally numbering the log entries 0, 1, 2, ...). There 
are many ways to implement this, but the simplest and fastest
     is with a leader who chooses the ordering of values provided to it. As 
long as the leader remains alive, all followers need to only copy the values 
and ordering the leader chooses.
@@ -384,16 +384,16 @@
     This majority vote approach has a very nice property: the latency is 
dependent on only the fastest servers. That is, if the replication factor is 
three, the latency is determined by the faster follower not the slower one.
     <p>
     There are a rich variety of algorithms in this family including ZooKeeper's
-    <a 
href="http://web.archive.org/web/20140602093727/http://www.stanford.edu/class/cs347/reading/zab.pdf";>Zab</a>,
+    <a 
href="https://web.archive.org/web/20140602093727/https://www.stanford.edu/class/cs347/reading/zab.pdf";>Zab</a>,
     <a 
href="https://www.usenix.org/system/files/conference/atc14/atc14-paper-ongaro.pdf";>Raft</a>,
-    and <a href="http://pmg.csail.mit.edu/papers/vr-revisited.pdf";>Viewstamped 
Replication</a>.
+    and <a 
href="https://pmg.csail.mit.edu/papers/vr-revisited.pdf";>Viewstamped 
Replication</a>.
     The most similar academic publication we are aware of to Kafka's actual 
implementation is
-    <a 
href="http://research.microsoft.com/apps/pubs/default.aspx?id=66814";>PacificA</a>
 from Microsoft.
+    <a 
href="https://research.microsoft.com/apps/pubs/default.aspx?id=66814";>PacificA</a>
 from Microsoft.
     <p>
     The downside of majority vote is that it doesn't take many failures to 
leave you with no electable leaders. To tolerate one failure requires three 
copies of the data, and to tolerate two failures requires five copies
     of the data. In our experience having only enough redundancy to tolerate a 
single failure is not enough for a practical system, but doing every write five 
times, with 5x the disk space requirements and 1/5th the
     throughput, is not very practical for large volume data problems. This is 
likely why quorum algorithms more commonly appear for shared cluster 
configuration such as ZooKeeper but are less common for primary data
-    storage. For example in HDFS the namenode's high-availability feature is 
built on a <a 
href="http://blog.cloudera.com/blog/2012/10/quorum-based-journaling-in-cdh4-1";>majority-vote-based
 journal</a>, but this more
+    storage. For example in HDFS the namenode's high-availability feature is 
built on a <a 
href="https://blog.cloudera.com/blog/2012/10/quorum-based-journaling-in-cdh4-1";>majority-vote-based
 journal</a>, but this more
     expensive approach is not used for the data itself.
     <p>
     Kafka takes a slightly different approach to choosing its quorum set. 
Instead of majority vote, Kafka dynamically maintains a set of in-sync replicas 
(ISR) that are caught-up to the leader. Only members of this set
@@ -494,12 +494,12 @@
     <li><i>Event sourcing</i>. This is a style of application design which 
co-locates query processing with application design and uses a log of changes 
as the primary store for the application.
     <li><i>Journaling for high-availability</i>. A process that does local 
computation can be made fault-tolerant by logging out changes that it makes to 
its local state so another process can reload these changes and
     carry on if it should fail. A concrete example of this is handling counts, 
aggregations, and other "group by"-like processing in a stream query system. 
Samza, a real-time stream-processing framework,
-    <a 
href="http://samza.apache.org/learn/documentation/0.7.0/container/state-management.html";>uses
 this feature</a> for exactly this purpose.
+    <a 
href="https://samza.apache.org/learn/documentation/0.7.0/container/state-management.html";>uses
 this feature</a> for exactly this purpose.
     </ol>
     In each of these cases one needs primarily to handle the real-time feed of 
changes, but occasionally, when a machine crashes or data needs to be re-loaded 
or re-processed, one needs to do a full load.
     Log compaction allows feeding both of these use cases off the same backing 
topic.
 
-    This style of usage of a log is described in more detail in <a 
href="http://engineering.linkedin.com/distributed-systems/log-what-every-software-engineer-should-know-about-real-time-datas-unifying";>this
 blog post</a>.
+    This style of usage of a log is described in more detail in <a 
href="https://engineering.linkedin.com/distributed-systems/log-what-every-software-engineer-should-know-about-real-time-datas-unifying";>this
 blog post</a>.
     <p>
     The general idea is quite simple. If we had infinite log retention, and we 
logged each change in the above cases, then we would have captured the state of 
the system at each time from when it first began.
     Using this complete log, we could restore to any point in time by 
replaying the first N records in the log. This hypothetical complete log is not 
very practical for systems that update a single record many times
@@ -651,9 +651,9 @@
     <h4 class="anchor-heading"><a id="design_quotascpu" 
class="anchor-link"></a><a href="#design_quotascpu">Request Rate Quotas</a></h4>
     <p>
         Request rate quotas are defined as the percentage of time a client can 
utilize on request handler I/O
-        threads and network threads of each broker within a quota window. A 
quota of <tt>n%</tt> represents
-        <tt>n%</tt> of one thread, so the quota is out of a total capacity of 
<tt>((num.io.threads + num.network.threads) * 100)%</tt>.
-        Each group of clients may use a total percentage of upto <tt>n%</tt> 
across all I/O and network threads in a quota
+        threads and network threads of each broker within a quota window. A 
quota of <code>n%</code> represents
+        <code>n%</code> of one thread, so the quota is out of a total capacity 
of <code>((num.io.threads + num.network.threads) * 100)%</code>.
+        Each group of clients may use a total percentage of upto 
<code>n%</code> across all I/O and network threads in a quota
         window before being throttled. Since the number of threads allocated 
for I/O and network threads are typically based
         on the number of cores available on the broker host, request rate 
quotas represent the total percentage of CPU
         that may be used by each group of clients sharing the quota.
diff --git a/docs/implementation.html b/docs/implementation.html
index 9475a747dd9..fb1f52ade5b 100644
--- a/docs/implementation.html
+++ b/docs/implementation.html
@@ -18,7 +18,7 @@
 <script id="implementation-template" type="text/x-handlebars-template">
     <h3 class="anchor-heading"><a id="networklayer" class="anchor-link"></a><a 
href="#networklayer">5.1 Network Layer</a></h3>
     <p>
-    The network layer is a fairly straight-forward NIO server, and will not be 
described in great detail. The sendfile implementation is done by giving the 
<code>TransferableRecords</code> interface a <code>writeTo</code> method. This 
allows the file-backed message set to use the more efficient 
<code>transferTo</code> implementation instead of an in-process buffered write. 
The threading model is a single acceptor thread and <i>N</i> processor threads 
which handle a fixed number of connec [...]
+    The network layer is a fairly straight-forward NIO server, and will not be 
described in great detail. The sendfile implementation is done by giving the 
<code>TransferableRecords</code> interface a <code>writeTo</code> method. This 
allows the file-backed message set to use the more efficient 
<code>transferTo</code> implementation instead of an in-process buffered write. 
The threading model is a single acceptor thread and <i>N</i> processor threads 
which handle a fixed number of connec [...]
     </p>
     <h3 class="anchor-heading"><a id="messages" class="anchor-link"></a><a 
href="#messages">5.2 Messages</a></h3>
     <p>
diff --git a/docs/ops.html b/docs/ops.html
index 3de7d3faae8..85b683be245 100644
--- a/docs/ops.html
+++ b/docs/ops.html
@@ -15,7 +15,9 @@
  limitations under the License.
 -->
 
-  Here is some information on actually running Kafka as a production system 
based on usage and experience at LinkedIn. Please send us any additional tips 
you know of.
+<script id="ops-template" type="text/x-handlebars-template">
+
+  <p>Here is some information on actually running Kafka as a production 
system. Please send us any additional tips you know of.</p>
 
   <h3 class="anchor-heading"><a id="basic_ops" class="anchor-link"></a><a 
href="#basic_ops">6.1 Basic Kafka Operations</a></h3>
 
@@ -368,13 +370,13 @@
   There are two interfaces that can be used to engage a throttle. The 
simplest, and safest, is to apply a throttle when invoking the 
kafka-reassign-partitions.sh, but kafka-configs.sh can also be used to view and 
alter the throttle values directly.
   <p></p>
   So for example, if you were to execute a rebalance, with the below command, 
it would move partitions at no more than 50MB/s.
-  <pre class="language-bash">$ bin/kafka-reassign-partitions.sh 
--bootstrap-server localhost:9092 --execute --reassignment-json-file 
bigger-cluster.json --throttle 50000000</code></pre>
+  <pre class="language-bash">$ bin/kafka-reassign-partitions.sh 
--bootstrap-server localhost:9092 --execute --reassignment-json-file 
bigger-cluster.json --throttle 50000000</pre>
   When you execute this script you will see the throttle engage:
   <pre class="line-numbers"><code class="language-bash">  The inter-broker 
throttle limit was set to 50000000 B/s
   Successfully started partition reassignment for foo1-0</code></pre>
   <p>Should you wish to alter the throttle, during a rebalance, say to 
increase the throughput so it completes quicker, you can do this by re-running 
the execute command with the --additional option passing the same 
reassignment-json-file:</p>
   <pre class="language-bash">$ bin/kafka-reassign-partitions.sh 
--bootstrap-server localhost:9092  --additional --execute 
--reassignment-json-file bigger-cluster.json --throttle 700000000
-  The inter-broker throttle limit was set to 700000000 B/s</code></pre>
+  The inter-broker throttle limit was set to 700000000 B/s</pre>
 
   <p>Once the rebalance completes the administrator can check the status of 
the rebalance using the --verify option.
       If the rebalance has completed, the throttle will be removed via the 
--verify command. It is important that
@@ -446,12 +448,12 @@
   <p><i>(2) Ensuring Progress:</i></p>
   <p>If the throttle is set too low, in comparison to the incoming write rate, 
it is possible for replication to not
       make progress. This occurs when:</p>
-  <pre>max(BytesInPerSec) > throttle</code></pre>
+  <pre>max(BytesInPerSec) > throttle</pre>
   <p>
       Where BytesInPerSec is the metric that monitors the write throughput of 
producers into each broker. </p>
   <p>The administrator can monitor whether replication is making progress, 
during the rebalance, using the metric:</p>
 
-  
<pre>kafka.server:type=FetcherLagMetrics,name=ConsumerLag,clientId=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)</code></pre>
+  
<pre>kafka.server:type=FetcherLagMetrics,name=ConsumerLag,clientId=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)</pre>
 
   <p>The lag should constantly decrease during replication. If the metric does 
not decrease the administrator should
       increase the
@@ -518,7 +520,7 @@
   <p>
   This is not the only possible deployment pattern. It is possible to read 
from or write to a remote Kafka cluster over the WAN, though obviously this 
will add whatever latency is required to get the cluster.
   <p>
-  Kafka naturally batches data in both the producer and consumer so it can 
achieve high-throughput even over a high-latency connection. To allow this 
though it may be necessary to increase the TCP socket buffer sizes for the 
producer, consumer, and broker using the <code>socket.send.buffer.bytes</code> 
and <code>socket.receive.buffer.bytes</code> configurations. The appropriate 
way to set this is documented <a 
href="http://en.wikipedia.org/wiki/Bandwidth-delay_product";>here</a>.
+  Kafka naturally batches data in both the producer and consumer so it can 
achieve high-throughput even over a high-latency connection. To allow this 
though it may be necessary to increase the TCP socket buffer sizes for the 
producer, consumer, and broker using the <code>socket.send.buffer.bytes</code> 
and <code>socket.receive.buffer.bytes</code> configurations. The appropriate 
way to set this is documented <a 
href="https://en.wikipedia.org/wiki/Bandwidth-delay_product";>here</a>.
   <p>
   It is generally <i>not</i> advisable to run a <i>single</i> Kafka cluster 
that spans multiple datacenters over a high-latency link. This will incur very 
high replication latency both for Kafka writes and ZooKeeper writes, and 
neither Kafka nor ZooKeeper will remain available in all locations if the 
network between locations is unavailable.
 
@@ -700,14 +702,14 @@ us-east.admin.bootstrap.servers = broker8-secondary:9092
     Exactly-once semantics are supported for dedicated MirrorMaker clusters as 
of version 3.5.0.</p>
   
   <p>
-    For new MirrorMaker clusters, set the 
<code>exactly.once.source.support</code> property to enabled for all targeted 
Kafka clusters that should be written to with exactly-once semantics. For 
example, to enable exactly-once for writes to cluster </code>us-east</code>, 
the following configuration can be used:
+    For new MirrorMaker clusters, set the 
<code>exactly.once.source.support</code> property to enabled for all targeted 
Kafka clusters that should be written to with exactly-once semantics. For 
example, to enable exactly-once for writes to cluster <code>us-east</code>, the 
following configuration can be used:
   </p>
 
 <pre class="line-numbers"><code 
class="language-text">us-east.exactly.once.source.support = enabled
 </code></pre>
   
   <p>
-    For existing MirrorMaker clusters, a two-step upgrade is necessary. 
Instead of immediately setting the <code>exactly.once.source.support</code> 
property to enabled, first set it to <code>preparing</code> on all nodes in the 
cluster. Once this is complete, it can be set to </code>enabled</code> on all 
nodes in the cluster, in a second round of restarts.
+    For existing MirrorMaker clusters, a two-step upgrade is necessary. 
Instead of immediately setting the <code>exactly.once.source.support</code> 
property to enabled, first set it to <code>preparing</code> on all nodes in the 
cluster. Once this is complete, it can be set to <code>enabled</code> on all 
nodes in the cluster, in a second round of restarts.
   </p>
   
   <p>
@@ -1324,8 +1326,8 @@ $ bin/kafka-acls.sh \
   It is unlikely to require much OS-level tuning, but there are three 
potentially important OS-level configurations:
   <ul>
       <li>File descriptor limits: Kafka uses file descriptors for log segments 
and open connections. If a broker hosts many partitions, consider that the 
broker needs at least (number_of_partitions)*(partition_size/segment_size) to 
track all log segments in addition to the number of connections the broker 
makes. We recommend at least 100000 allowed file descriptors for the broker 
processes as a starting point. Note: The mmap() function adds an extra 
reference to the file associated with  [...]
-      <li>Max socket buffer size: can be increased to enable high-performance 
data transfer between data centers as <a 
href="http://www.psc.edu/index.php/networking/641-tcp-tune";>described here</a>.
-      <li>Maximum number of memory map areas a process may have (aka 
vm.max_map_count). <a 
href="http://kernel.org/doc/Documentation/sysctl/vm.txt";>See the Linux kernel 
documentation</a>. You should keep an eye at this OS-level property when 
considering the maximum number of partitions a broker may have. By default, on 
a number of Linux systems, the value of vm.max_map_count is somewhere around 
65535. Each log segment, allocated per partition, requires a pair of 
index/timeindex files, an [...]
+      <li>Max socket buffer size: can be increased to enable high-performance 
data transfer between data centers as <a 
href="https://www.psc.edu/index.php/networking/641-tcp-tune";>described here</a>.
+      <li>Maximum number of memory map areas a process may have (aka 
vm.max_map_count). <a 
href="https://kernel.org/doc/Documentation/sysctl/vm.txt";>See the Linux kernel 
documentation</a>. You should keep an eye at this OS-level property when 
considering the maximum number of partitions a broker may have. By default, on 
a number of Linux systems, the value of vm.max_map_count is somewhere around 
65535. Each log segment, allocated per partition, requires a pair of 
index/timeindex files, a [...]
   </ul>
   <p>
 
@@ -1353,14 +1355,14 @@ $ bin/kafka-acls.sh \
 
   <h4 class="anchor-heading"><a id="linuxflush" class="anchor-link"></a><a 
href="#linuxflush">Understanding Linux OS Flush Behavior</a></h4>
 
-  In Linux, data written to the filesystem is maintained in <a 
href="http://en.wikipedia.org/wiki/Page_cache";>pagecache</a> until it must be 
written out to disk (due to an application-level fsync or the OS's own flush 
policy). The flushing of data is done by a set of background threads called 
pdflush (or in post 2.6.32 kernels "flusher threads").
+  In Linux, data written to the filesystem is maintained in <a 
href="https://en.wikipedia.org/wiki/Page_cache";>pagecache</a> until it must be 
written out to disk (due to an application-level fsync or the OS's own flush 
policy). The flushing of data is done by a set of background threads called 
pdflush (or in post 2.6.32 kernels "flusher threads").
   <p>
   Pdflush has a configurable policy that controls how much dirty data can be 
maintained in cache and for how long before it must be written back to disk.
-  This policy is described <a 
href="http://web.archive.org/web/20160518040713/http://www.westnet.com/~gsmith/content/linux-pdflush.htm";>here</a>.
+  This policy is described <a 
href="https://web.archive.org/web/20160518040713/http://www.westnet.com/~gsmith/content/linux-pdflush.htm";>here</a>.
   When Pdflush cannot keep up with the rate of data being written it will 
eventually cause the writing process to block incurring latency in the writes 
to slow down the accumulation of data.
   <p>
   You can see the current state of OS memory usage by doing
-  <pre class="language-bash"> &gt; cat /proc/meminfo </code></pre>
+  <pre class="language-bash"> &gt; cat /proc/meminfo</pre>
   The meaning of these values are described in the link above.
   <p>
   Using pagecache has several advantages over an in-process cache for storing 
data that will be written out to disk:
@@ -3570,7 +3572,7 @@ for built-in state stores, currently we have:
   <h3 class="anchor-heading"><a id="zk" class="anchor-link"></a><a 
href="#zk">6.9 ZooKeeper</a></h3>
 
   <h4 class="anchor-heading"><a id="zkversion" class="anchor-link"></a><a 
href="#zkversion">Stable version</a></h4>
-  The current stable branch is 3.5. Kafka is regularly updated to include the 
latest release in the 3.5 series.
+  The current stable branch is 3.8. Kafka is regularly updated to include the 
latest release in the 3.8 series.
   
   <h4 class="anchor-heading"><a id="zk_depr" class="anchor-link"></a><a 
href="#zk_depr">ZooKeeper Deprecation</a></h4>
   <p>With the release of Apache Kafka 3.5, Zookeeper is now marked deprecated. 
Removal of ZooKeeper is planned in the next major release of Apache Kafka 
(version 4.0),
@@ -3579,7 +3581,7 @@ for built-in state stores, currently we have:
      see <a href="#kraft_missing">current missing features</a> for more 
information.</p>
 
     <h5 class="anchor-heading"><a id="zk_depr_migration" 
class="anchor-link"></a><a href="#zk_drep_migration">Migration</a></h5>
-    <p>Migration of an existing ZooKeeper based Kafka cluster to KRaft is 
currently Preview and we expect it to be ready for production usage in version 
3.6. Users are recommended to begin planning for migration to KRaft and also 
begin testing to provide any feedback. Refer to <a 
href="#kraft_zk_migration">ZooKeeper to KRaft Migration</a> for details on how 
to perform a live migration from ZooKeeper to KRaft and current limitations.</p>
+    <p>Users are recommended to begin planning for migration to KRaft and also 
begin testing to provide any feedback. Refer to <a 
href="#kraft_zk_migration">ZooKeeper to KRaft Migration</a> for details on how 
to perform a live migration from ZooKeeper to KRaft and current limitations.</p>
        
     <h5 class="anchor-heading"><a id="zk_depr_3xsupport" 
class="anchor-link"></a><a href="#zk_depr_3xsupport">3.x and ZooKeeper 
Support</a></h5>
     <p>The final 3.x minor release, that supports ZooKeeper mode, will receive 
critical bug fixes and security fixes for 12 months after its release.</p>
@@ -3750,9 +3752,8 @@ foo
 
   <h3>Preparing for migration</h3>
   <p>
-    Before beginning the migration, the Kafka brokers must be upgraded to 
software version 3.5.0 and have the
-    "inter.broker.protocol.version" configuration set to "3.5". See <a 
href="#upgrade_3_5_0">Upgrading to 3.5.0</a> for
-    upgrade instructions.
+    Before beginning the migration, the Kafka brokers must be upgraded to 
software version {{fullDotVersion}} and have the
+    "inter.broker.protocol.version" configuration set to "{{dotVersion}}".
   </p>
 
   <p>
@@ -3833,7 +3834,7 @@ advertised.listeners=PLAINTEXT://localhost:9092
 listener.security.protocol.map=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT
 
 # Set the IBP
-inter.broker.protocol.version=3.5
+inter.broker.protocol.version={{dotVersion}}
 
 # Enable the migration
 zookeeper.metadata.migration.enable=true
@@ -3875,7 +3876,7 @@ advertised.listeners=PLAINTEXT://localhost:9092
 listener.security.protocol.map=PLAINTEXT:PLAINTEXT,CONTROLLER:PLAINTEXT
 
 # Don't set the IBP, KRaft uses "metadata.version" feature flag
-# inter.broker.protocol.version=3.5
+# inter.broker.protocol.version={{dotVersion}}
 
 # Remove the migration enabled flag
 # zookeeper.metadata.migration.enable=true
@@ -3973,9 +3974,8 @@ listeners=CONTROLLER://:9093
   <li><code>retention.bytes</code></li>
 </ul>
 
-  The configuration prefixed with <code>local</code> are to specify the 
time/size the "local" log file can accept before moving to remote storage, and 
then get deleted.
-  If unset, The value in <code>retention.ms</code> and 
<code>retention.bytes</code> will be used.
-</p>
+  <p>The configuration prefixed with <code>local</code> are to specify the 
time/size the "local" log file can accept before moving to remote storage, and 
then get deleted.
+  If unset, The value in <code>retention.ms</code> and 
<code>retention.bytes</code> will be used.</p>
 
 <h4 class="anchor-heading"><a id="tiered_storage_config_ex" 
class="anchor-link"></a><a href="#tiered_storage_config_ex">Quick Start 
Example</a></h4>
 
@@ -3986,7 +3986,7 @@ listeners=CONTROLLER://:9093
 
 <p>To adopt the `LocalTieredStorage`, the test library needs to be built 
locally</p>
 <pre># please checkout to the specific version tag you're using before 
building it
-# ex: `git checkout 3.6.0`
+# ex: `git checkout {{fullDotVersion}}`
 ./gradlew clean :storage:testJar</pre>
 <p>After build successfully, there should be a `kafka-storage-x.x.x-test.jar` 
file under `storage/build/libs`.
 Next, setting configurations in the broker side to enable tiered storage 
feature.</p>
@@ -4002,7 +4002,7 @@ remote.log.metadata.manager.listener.name=PLAINTEXT
 # This is the mandatory configuration for tiered storage.
 # Here, we use the `LocalTieredStorage` built above.
 
remote.log.storage.manager.class.name=org.apache.kafka.server.log.remote.storage.LocalTieredStorage
-remote.log.storage.manager.class.path=/PATH/TO/kafka-storage-x.x.x-test.jar
+remote.log.storage.manager.class.path=/PATH/TO/kafka-storage-{{fullDotVersion}}-test.jar
 
 # These 2 prefix are default values, but customizable
 remote.log.storage.manager.impl.prefix=rsm.config.
@@ -4018,7 +4018,6 @@ rlmm.config.remote.log.metadata.topic.replication.factor=1
 # Try to speed up the log retention check interval for testing
 log.retention.check.interval.ms=1000
 </pre>
-</p>
 
 <p>Following <a href="#quickstart_startserver">quick start guide</a> to start 
up the kafka environment.
   Then, create a topic with tiered storage enabled with configs:
@@ -4035,7 +4034,6 @@ bin/kafka-topics.sh --create --topic tieredTopic 
--bootstrap-server localhost:90
 --config remote.storage.enable=true --config local.retention.ms=1000 --config 
retention.ms=3600000 \
 --config segment.bytes=1048576 --config file.delete.delay.ms=1000
 </pre>
-</p>
 
 <p>Try to send messages to the `tieredTopic` topic to roll the log segment:</p>
 
@@ -4076,7 +4074,6 @@ bin/kafka-producer-perf-test.sh --topic tieredTopic 
--num-records 1200 --record-
   <li>Deleting tiered storage enabled topics is required before disabling 
tiered storage at the broker level</li>
   <li>Admin actions related to tiered storage feature are only supported on 
clients from version 3.0 onwards</li>
 </ul>
-</p>
 
 <p>For more information, please check <a 
href="https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Tiered+Storage+Early+Access+Release+Notes";>Tiered
 Storage Early Access Release Note</a>.
 </p>
diff --git a/docs/quickstart.html b/docs/quickstart.html
index f396be4f5d5..72ff7b81826 100644
--- a/docs/quickstart.html
+++ b/docs/quickstart.html
@@ -355,7 +355,7 @@ wordCounts.toStream().to("output-topic", 
Produced.with(Serdes.String(), Serdes.L
             <a href="#quickstart_kafkacongrats">Congratulations!</a>
         </h4>
 
-        <p>You have successfully finished the Apache Kafka quickstart.<div>
+        <p>You have successfully finished the Apache Kafka quickstart.</p>
 
         <p>To learn more, we suggest the following next steps:</p>
 
diff --git a/docs/security.html b/docs/security.html
index 63ff3bb6201..895f2b0b0a7 100644
--- a/docs/security.html
+++ b/docs/security.html
@@ -54,7 +54,7 @@
            
     <p>The <code>LISTENER_NAME</code> is usually a descriptive name which 
defines the purpose of
       the listener. For example, many configurations use a separate listener 
for client traffic,
-      so they might refer to the corresponding listener as <code>CLIENT</code> 
in the configuration:</p
+      so they might refer to the corresponding listener as <code>CLIENT</code> 
in the configuration:</p>
       
     <pre class="line-numbers"><code 
class="language-text">listeners=CLIENT://localhost:9092</code></pre>
       
@@ -471,7 +471,7 @@ ssl.truststore.password=test1234</code></pre>
             <pre class="line-numbers"><code 
class="language-text">security.inter.broker.protocol=SSL</code></pre>
 
             <p>
-            Due to import regulations in some countries, the Oracle 
implementation limits the strength of cryptographic algorithms available by 
default. If stronger algorithms are needed (for example, AES with 256-bit 
keys), the <a 
href="http://www.oracle.com/technetwork/java/javase/downloads/index.html";>JCE 
Unlimited Strength Jurisdiction Policy Files</a> must be obtained and installed 
in the JDK/JRE. See the
+            Due to import regulations in some countries, the Oracle 
implementation limits the strength of cryptographic algorithms available by 
default. If stronger algorithms are needed (for example, AES with 256-bit 
keys), the <a 
href="https://www.oracle.com/technetwork/java/javase/downloads/index.html";>JCE 
Unlimited Strength Jurisdiction Policy Files</a> must be obtained and installed 
in the JDK/JRE. See the
             <a 
href="https://docs.oracle.com/javase/8/docs/technotes/guides/security/SunProviders.html";>JCA
 Providers Documentation</a> for more information.
             </p>
 
@@ -535,25 +535,25 @@ ssl.key.password=test1234</code></pre>
                 <li><h5><a id="security_jaas_broker"
                     href="#security_jaas_broker">JAAS configuration for Kafka 
brokers</a></h5>
 
-                    <p><tt>KafkaServer</tt> is the section name in the JAAS 
file used by each
+                    <p><code>KafkaServer</code> is the section name in the 
JAAS file used by each
                     KafkaServer/Broker. This section provides SASL 
configuration options
                     for the broker including any SASL client connections made 
by the broker
                     for inter-broker communication. If multiple listeners are 
configured to use
                     SASL, the section name may be prefixed with the listener 
name in lower-case
-                    followed by a period, e.g. 
<tt>sasl_ssl.KafkaServer</tt>.</p>
+                    followed by a period, e.g. 
<code>sasl_ssl.KafkaServer</code>.</p>
 
-                    <p><tt>Client</tt> section is used to authenticate a SASL 
connection with
+                    <p><code>Client</code> section is used to authenticate a 
SASL connection with
                     zookeeper. It also allows the brokers to set SASL ACL on 
zookeeper
                     nodes which locks these nodes down so that only the 
brokers can
                     modify it. It is necessary to have the same principal name 
across all
                     brokers. If you want to use a section name other than 
Client, set the
-                    system property <tt>zookeeper.sasl.clientconfig</tt> to 
the appropriate
-                    name (<i>e.g.</i>, 
<tt>-Dzookeeper.sasl.clientconfig=ZkClient</tt>).</p>
+                    system property <code>zookeeper.sasl.clientconfig</code> 
to the appropriate
+                    name (<i>e.g.</i>, 
<code>-Dzookeeper.sasl.clientconfig=ZkClient</code>).</p>
 
                     <p>ZooKeeper uses "zookeeper" as the service name by 
default. If you
                     want to change this, set the system property
-                    <tt>zookeeper.sasl.client.username</tt> to the appropriate 
name
-                    (<i>e.g.</i>, 
<tt>-Dzookeeper.sasl.client.username=zk</tt>).</p>
+                    <code>zookeeper.sasl.client.username</code> to the 
appropriate name
+                    (<i>e.g.</i>, 
<code>-Dzookeeper.sasl.client.username=zk</code>).</p>
 
                     <p>Brokers may also configure JAAS using the broker 
configuration property <code>sasl.jaas.config</code>.
                     The property name must be prefixed with the listener 
prefix including the SASL mechanism,
@@ -609,8 +609,8 @@ 
listener.name.sasl_ssl.plain.sasl.jaas.config=org.apache.kafka.common.security.p
                         <li><h6 class="anchor-heading"><a 
id="security_client_staticjaas" class="anchor-link"></a><a 
href="#security_client_staticjaas">JAAS configuration using static config 
file</a></h6>
                             To configure SASL authentication on the clients 
using static JAAS config file:
                             <ol>
-                                <li>Add a JAAS config file with a client login 
section named <tt>KafkaClient</tt>. Configure
-                                    a login module in <tt>KafkaClient</tt> for 
the selected mechanism as described in the examples
+                                <li>Add a JAAS config file with a client login 
section named <code>KafkaClient</code>. Configure
+                                    a login module in <code>KafkaClient</code> 
for the selected mechanism as described in the examples
                                     for setting up <a 
href="#security_sasl_kerberos_clientconfig">GSSAPI (Kerberos)</a>,
                                     <a 
href="#security_sasl_plain_clientconfig">PLAIN</a>,
                                     <a 
href="#security_sasl_scram_clientconfig">SCRAM</a> or
@@ -719,7 +719,7 @@ Client {
     principal="kafka/kafka1.hostname....@example.com";
 };</code></pre>
 
-                            <tt>KafkaServer</tt> section in the JAAS file 
tells the broker which principal to use and the location of the keytab where 
this principal is stored. It
+                            <code>KafkaServer</code> section in the JAAS file 
tells the broker which principal to use and the location of the keytab where 
this principal is stored. It
                             allows the broker to login using the keytab 
specified in this section. See <a href="#security_jaas_broker">notes</a> for 
more details on Zookeeper SASL configuration.
                         </li>
                         <li>Pass the JAAS and optionally the krb5 file 
locations as JVM parameters to each Kafka broker (see <a 
href="https://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/tutorials/KerberosReq.html";>here</a>
 for more details):
@@ -760,7 +760,7 @@ sasl.enabled.mechanisms=GSSAPI</code></pre>
 
                             JAAS configuration for clients may alternatively 
be specified as a JVM parameter similar to brokers
                             as described <a 
href="#security_client_staticjaas">here</a>. Clients use the login section named
-                            <tt>KafkaClient</tt>. This option allows only one 
user for all client connections from a JVM.</li>
+                            <code>KafkaClient</code>. This option allows only 
one user for all client connections from a JVM.</li>
                         <li>Make sure the keytabs configured in the JAAS 
configuration are readable by the operating system user who is starting kafka 
client.</li>
                         <li>Optionally pass the krb5 file locations as JVM 
parameters to each client JVM (see <a 
href="https://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/tutorials/KerberosReq.html";>here</a>
 for more details):
                             <pre class="line-numbers"><code 
class="language-bash">-Djava.security.krb5.conf=/etc/kafka/krb5.conf</code></pre></li>
@@ -788,9 +788,9 @@ sasl.kerberos.service.name=kafka</code></pre></li>
     user_admin="admin-secret"
     user_alice="alice-secret";
 };</code></pre>
-                            This configuration defines two users (<i>admin</i> 
and <i>alice</i>). The properties <tt>username</tt> and <tt>password</tt>
-                            in the <tt>KafkaServer</tt> section are used by 
the broker to initiate connections to other brokers. In this example,
-                            <i>admin</i> is the user for inter-broker 
communication. The set of properties <tt>user_<i>userName</i></tt> defines
+                            This configuration defines two users (<i>admin</i> 
and <i>alice</i>). The properties <code>username</code> and 
<code>password</code>
+                            in the <code>KafkaServer</code> section are used 
by the broker to initiate connections to other brokers. In this example,
+                            <i>admin</i> is the user for inter-broker 
communication. The set of properties <code>user_<i>userName</i></code> defines
                             the passwords for all users that connect to the 
broker and the broker validates all client connections including
                             those from other brokers using these 
properties.</li>
                         <li>Pass the JAAS config file location as JVM 
parameter to each Kafka broker:
@@ -812,14 +812,14 @@ sasl.enabled.mechanisms=PLAIN</code></pre></li>
                             <pre class="line-numbers"><code 
class="language-text">sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule
 required \
     username="alice" \
     password="alice-secret";</code></pre>
-                            <p>The options <tt>username</tt> and 
<tt>password</tt> are used by clients to configure
+                            <p>The options <code>username</code> and 
<code>password</code> are used by clients to configure
                                 the user for client connections. In this 
example, clients connect to the broker as user <i>alice</i>.
                                 Different clients within a JVM may connect as 
different users by specifying different user names
                                 and passwords in 
<code>sasl.jaas.config</code>.</p>
 
                             <p>JAAS configuration for clients may 
alternatively be specified as a JVM parameter similar to brokers
                                 as described <a 
href="#security_client_staticjaas">here</a>. Clients use the login section named
-                                <tt>KafkaClient</tt>. This option allows only 
one user for all client connections from a JVM.</p></li>
+                                <code>KafkaClient</code>. This option allows 
only one user for all client connections from a JVM.</p></li>
                         <li>Configure the following properties in 
producer.properties or consumer.properties:
                             <pre class="line-numbers"><code 
class="language-text">security.protocol=SASL_SSL
 sasl.mechanism=PLAIN</code></pre></li>
@@ -853,7 +853,7 @@ sasl.mechanism=PLAIN</code></pre></li>
             <ol>
                 <li><h5 class="anchor-heading"><a 
id="security_sasl_scram_credentials" class="anchor-link"></a><a 
href="#security_sasl_scram_credentials">Creating SCRAM Credentials</a></h5>
                     <p>The SCRAM implementation in Kafka uses Zookeeper as 
credential store. Credentials can be created in
-                        Zookeeper using <tt>kafka-configs.sh</tt>. For each 
SCRAM mechanism enabled, credentials must be created
+                        Zookeeper using <code>kafka-configs.sh</code>. For 
each SCRAM mechanism enabled, credentials must be created
                         by adding a config with the mechanism name. 
Credentials for inter-broker communication must be created
                         before Kafka brokers are started. Client credentials 
may be created and updated dynamically and updated
                         credentials will be used to authenticate new 
connections.</p>
@@ -877,7 +877,7 @@ sasl.mechanism=PLAIN</code></pre></li>
     username="admin"
     password="admin-secret";
 };</code></pre>
-                            The properties <tt>username</tt> and 
<tt>password</tt> in the <tt>KafkaServer</tt> section are used by
+                            The properties <code>username</code> and 
<code>password</code> in the <code>KafkaServer</code> section are used by
                             the broker to initiate connections to other 
brokers. In this example, <i>admin</i> is the user for
                             inter-broker communication.</li>
                         <li>Pass the JAAS config file location as JVM 
parameter to each Kafka broker:
@@ -900,14 +900,14 @@ sasl.enabled.mechanisms=SCRAM-SHA-256 (or 
SCRAM-SHA-512)</code></pre></li>
     username="alice" \
     password="alice-secret";</code></pre>
 
-                            <p>The options <tt>username</tt> and 
<tt>password</tt> are used by clients to configure
+                            <p>The options <code>username</code> and 
<code>password</code> are used by clients to configure
                                 the user for client connections. In this 
example, clients connect to the broker as user <i>alice</i>.
                                 Different clients within a JVM may connect as 
different users by specifying different user names
                                 and passwords in 
<code>sasl.jaas.config</code>.</p>
 
                             <p>JAAS configuration for clients may 
alternatively be specified as a JVM parameter similar to brokers
                                 as described <a 
href="#security_client_staticjaas">here</a>. Clients use the login section named
-                                <tt>KafkaClient</tt>. This option allows only 
one user for all client connections from a JVM.</p></li>
+                                <code>KafkaClient</code>. This option allows 
only one user for all client connections from a JVM.</p></li>
                         <li>Configure the following properties in 
producer.properties or consumer.properties:
                             <pre class="line-numbers"><code 
class="language-text">security.protocol=SASL_SSL
 sasl.mechanism=SCRAM-SHA-256 (or SCRAM-SHA-512)</code></pre></li>
@@ -948,9 +948,9 @@ sasl.mechanism=SCRAM-SHA-256 (or 
SCRAM-SHA-512)</code></pre></li>
     org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule 
required
     unsecuredLoginStringClaim_sub="admin";
 };</code></pre>
-                            The property 
<tt>unsecuredLoginStringClaim_sub</tt> in the <tt>KafkaServer</tt> section is 
used by
+                            The property 
<code>unsecuredLoginStringClaim_sub</code> in the <code>KafkaServer</code> 
section is used by
                             the broker when it initiates connections to other 
brokers. In this example, <i>admin</i> will appear in the
-                            subject (<tt>sub</tt>) claim and will be the user 
for inter-broker communication.</li>
+                            subject (<code>sub</code>) claim and will be the 
user for inter-broker communication.</li>
                         <li>Pass the JAAS config file location as JVM 
parameter to each Kafka broker:
                             <pre class="line-numbers"><code 
class="language-bash">-Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf</code></pre></li>
                         <li>Configure SASL port and SASL mechanisms in 
server.properties as described <a href="#security_sasl_brokerconfig">here</a>. 
For example:
@@ -970,15 +970,15 @@ sasl.enabled.mechanisms=OAUTHBEARER</code></pre></li>
                             <pre class="line-numbers"><code 
class="language-text">sasl.jaas.config=org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule
 required \
     unsecuredLoginStringClaim_sub="alice";</code></pre>
 
-                            <p>The option 
<tt>unsecuredLoginStringClaim_sub</tt> is used by clients to configure
-                                the subject (<tt>sub</tt>) claim, which 
determines the user for client connections.
+                            <p>The option 
<code>unsecuredLoginStringClaim_sub</code> is used by clients to configure
+                                the subject (<code>sub</code>) claim, which 
determines the user for client connections.
                                 In this example, clients connect to the broker 
as user <i>alice</i>.
-                                Different clients within a JVM may connect as 
different users by specifying different subject (<tt>sub</tt>)
+                                Different clients within a JVM may connect as 
different users by specifying different subject (<code>sub</code>)
                                 claims in <code>sasl.jaas.config</code>.</p>
 
                             <p>JAAS configuration for clients may 
alternatively be specified as a JVM parameter similar to brokers
                                 as described <a 
href="#security_client_staticjaas">here</a>. Clients use the login section named
-                                <tt>KafkaClient</tt>. This option allows only 
one user for all client connections from a JVM.</p></li>
+                                <code>KafkaClient</code>. This option allows 
only one user for all client connections from a JVM.</p></li>
                         <li>Configure the following properties in 
producer.properties or consumer.properties:
                             <pre class="line-numbers"><code 
class="language-text">security.protocol=SASL_SSL (or SASL_PLAINTEXT if 
non-production)
 sasl.mechanism=OAUTHBEARER</code></pre></li>
@@ -997,48 +997,48 @@ sasl.mechanism=OAUTHBEARER</code></pre></li>
                                     <th>Documentation</th>
                                 </tr>
                                 <tr>
-                                    
<td><tt>unsecuredLoginStringClaim_&lt;claimname&gt;="value"</tt></td>
-                                    <td>Creates a <tt>String</tt> claim with 
the given name and value. Any valid
-                                        claim name can be specified except 
'<tt>iat</tt>' and '<tt>exp</tt>' (these are
+                                    
<td><code>unsecuredLoginStringClaim_&lt;claimname&gt;="value"</code></td>
+                                    <td>Creates a <code>String</code> claim 
with the given name and value. Any valid
+                                        claim name can be specified except 
'<code>iat</code>' and '<code>exp</code>' (these are
                                         automatically generated).</td>
                                 </tr>
                                 <tr>
-                                    
<td><tt>unsecuredLoginNumberClaim_&lt;claimname&gt;="value"</tt></td>
-                                    <td>Creates a <tt>Number</tt> claim with 
the given name and value. Any valid
-                                        claim name can be specified except 
'<tt>iat</tt>' and '<tt>exp</tt>' (these are
+                                    
<td><code>unsecuredLoginNumberClaim_&lt;claimname&gt;="value"</code></td>
+                                    <td>Creates a <code>Number</code> claim 
with the given name and value. Any valid
+                                        claim name can be specified except 
'<code>iat</code>' and '<code>exp</code>' (these are
                                         automatically generated).</td>
                                 </tr>
                                 <tr>
-                                    
<td><tt>unsecuredLoginListClaim_&lt;claimname&gt;="value"</tt></td>
-                                    <td>Creates a <tt>String List</tt> claim 
with the given name and values parsed
+                                    
<td><code>unsecuredLoginListClaim_&lt;claimname&gt;="value"</code></td>
+                                    <td>Creates a <code>String List</code> 
claim with the given name and values parsed
                                         from the given value where the first 
character is taken as the delimiter. For
-                                        example: 
<tt>unsecuredLoginListClaim_fubar="|value1|value2"</tt>. Any valid
-                                        claim name can be specified except 
'<tt>iat</tt>' and '<tt>exp</tt>' (these are
+                                        example: 
<code>unsecuredLoginListClaim_fubar="|value1|value2"</code>. Any valid
+                                        claim name can be specified except 
'<code>iat</code>' and '<code>exp</code>' (these are
                                         automatically generated).</td>
                                 </tr>
                                 <tr>
-                                    
<td><tt>unsecuredLoginExtension_&lt;extensionname&gt;="value"</tt></td>
-                                    <td>Creates a <tt>String</tt> extension 
with the given name and value.
-                                        For example: 
<tt>unsecuredLoginExtension_traceId="123"</tt>. A valid extension name
+                                    
<td><code>unsecuredLoginExtension_&lt;extensionname&gt;="value"</code></td>
+                                    <td>Creates a <code>String</code> 
extension with the given name and value.
+                                        For example: 
<code>unsecuredLoginExtension_traceId="123"</code>. A valid extension name
                                         is any sequence of lowercase or 
uppercase alphabet characters. In addition, the "auth" extension name is 
reserved.
                                         A valid extension value is any 
combination of characters with ASCII codes 1-127.
                                 </tr>
                                 <tr>
-                                    
<td><tt>unsecuredLoginPrincipalClaimName</tt></td>
-                                    <td>Set to a custom claim name if you wish 
the name of the <tt>String</tt>
-                                        claim holding the principal name to be 
something other than '<tt>sub</tt>'.</td>
+                                    
<td><code>unsecuredLoginPrincipalClaimName</code></td>
+                                    <td>Set to a custom claim name if you wish 
the name of the <code>String</code>
+                                        claim holding the principal name to be 
something other than '<code>sub</code>'.</td>
                                 </tr>
                                 <tr>
-                                    
<td><tt>unsecuredLoginLifetimeSeconds</tt></td>
+                                    
<td><code>unsecuredLoginLifetimeSeconds</code></td>
                                     <td>Set to an integer value if the token 
expiration is to be set to something
                                         other than the default value of 3600 
seconds (which is 1 hour). The
-                                        '<tt>exp</tt>' claim will be set to 
reflect the expiration time.</td>
+                                        '<code>exp</code>' claim will be set 
to reflect the expiration time.</td>
                                 </tr>
                                 <tr>
-                                    
<td><tt>unsecuredLoginScopeClaimName</tt></td>
-                                    <td>Set to a custom claim name if you wish 
the name of the <tt>String</tt> or
-                                        <tt>String List</tt> claim holding any 
token scope to be something other than
-                                        '<tt>scope</tt>'.</td>
+                                    
<td><code>unsecuredLoginScopeClaimName</code></td>
+                                    <td>Set to a custom claim name if you wish 
the name of the <code>String</code> or
+                                        <code>String List</code> claim holding 
any token scope to be something other than
+                                        '<code>scope</code>'.</td>
                                 </tr>
                             </table>
                         </li>
@@ -1053,25 +1053,25 @@ sasl.mechanism=OAUTHBEARER</code></pre></li>
                                     <th>Documentation</th>
                                 </tr>
                                 <tr>
-                                    
<td><tt>unsecuredValidatorPrincipalClaimName="value"</tt></td>
-                                    <td>Set to a non-empty value if you wish a 
particular <tt>String</tt> claim
+                                    
<td><code>unsecuredValidatorPrincipalClaimName="value"</code></td>
+                                    <td>Set to a non-empty value if you wish a 
particular <code>String</code> claim
                                         holding a principal name to be checked 
for existence; the default is to check
-                                        for the existence of the 
'<tt>sub</tt>' claim.</td>
+                                        for the existence of the 
'<code>sub</code>' claim.</td>
                                 </tr>
                                 <tr>
-                                    
<td><tt>unsecuredValidatorScopeClaimName="value"</tt></td>
-                                    <td>Set to a custom claim name if you wish 
the name of the <tt>String</tt> or
-                                        <tt>String List</tt> claim holding any 
token scope to be something other than
-                                        '<tt>scope</tt>'.</td>
+                                    
<td><code>unsecuredValidatorScopeClaimName="value"</code></td>
+                                    <td>Set to a custom claim name if you wish 
the name of the <code>String</code> or
+                                        <code>String List</code> claim holding 
any token scope to be something other than
+                                        '<code>scope</code>'.</td>
                                 </tr>
                                 <tr>
-                                    
<td><tt>unsecuredValidatorRequiredScope="value"</tt></td>
+                                    
<td><code>unsecuredValidatorRequiredScope="value"</code></td>
                                     <td>Set to a space-delimited list of scope 
values if you wish the
-                                        <tt>String/String List</tt> claim 
holding the token scope to be checked to
+                                        <code>String/String List</code> claim 
holding the token scope to be checked to
                                         make sure it contains certain 
values.</td>
                                 </tr>
                                 <tr>
-                                    
<td><tt>unsecuredValidatorAllowableClockSkewMs="value"</tt></td>
+                                    
<td><code>unsecuredValidatorAllowableClockSkewMs="value"</code></td>
                                     <td>Set to a positive integer value if you 
wish to allow up to some number of
                                         positive milliseconds of clock skew 
(the default is 0).</td>
                                 </tr>
@@ -1094,33 +1094,33 @@ sasl.mechanism=OAUTHBEARER</code></pre></li>
                             <th>Producer/Consumer/Broker Configuration 
Property</th>
                         </tr>
                         <tr>
-                            <td><tt>sasl.login.refresh.window.factor</tt></td>
+                            
<td><code>sasl.login.refresh.window.factor</code></td>
                         </tr>
                         <tr>
-                            <td><tt>sasl.login.refresh.window.jitter</tt></td>
+                            
<td><code>sasl.login.refresh.window.jitter</code></td>
                         </tr>
                         <tr>
-                            
<td><tt>sasl.login.refresh.min.period.seconds</tt></td>
+                            
<td><code>sasl.login.refresh.min.period.seconds</code></td>
                         </tr>
                         <tr>
-                            
<td><tt>sasl.login.refresh.min.buffer.seconds</tt></td>
+                            
<td><code>sasl.login.refresh.min.buffer.seconds</code></td>
                         </tr>
                     </table>
                 </li>
                 <li><h5><a id="security_sasl_oauthbearer_prod" 
href="#security_sasl_oauthbearer_prod">Secure/Production Use of 
SASL/OAUTHBEARER</a></h5>
                     Production use cases will require writing an 
implementation of
-                    
<tt>org.apache.kafka.common.security.auth.AuthenticateCallbackHandler</tt> that 
can handle an instance of
-                    
<tt>org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback</tt> 
and declaring it via either the
-                    <tt>sasl.login.callback.handler.class</tt> configuration 
option for a
+                    
<code>org.apache.kafka.common.security.auth.AuthenticateCallbackHandler</code> 
that can handle an instance of
+                    
<code>org.apache.kafka.common.security.oauthbearer.OAuthBearerTokenCallback</code>
 and declaring it via either the
+                    <code>sasl.login.callback.handler.class</code> 
configuration option for a
                     non-broker client or via the
-                    
<tt>listener.name.sasl_ssl.oauthbearer.sasl.login.callback.handler.class</tt>
+                    
<code>listener.name.sasl_ssl.oauthbearer.sasl.login.callback.handler.class</code>
                     configuration option for brokers (when SASL/OAUTHBEARER is 
the inter-broker
                     protocol).
                     <p>
                         Production use cases will also require writing an 
implementation of
-                        
<tt>org.apache.kafka.common.security.auth.AuthenticateCallbackHandler</tt> that 
can handle an instance of
-                        
<tt>org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback</tt>
 and declaring it via the
-                        
<tt>listener.name.sasl_ssl.oauthbearer.sasl.server.callback.handler.class</tt>
+                        
<code>org.apache.kafka.common.security.auth.AuthenticateCallbackHandler</code> 
that can handle an instance of
+                        
<code>org.apache.kafka.common.security.oauthbearer.OAuthBearerValidatorCallback</code>
 and declaring it via the
+                        
<code>listener.name.sasl_ssl.oauthbearer.sasl.server.callback.handler.class</code>
                         broker configuration option.
                 </li>
                 <li><h5><a id="security_sasl_oauthbearer_security" 
href="#security_sasl_oauthbearer_security">Security Considerations for 
SASL/OAUTHBEARER</a></h5>
@@ -1138,7 +1138,7 @@ sasl.mechanism=OAUTHBEARER</code></pre></li>
 
         <li><h4 class="anchor-heading"><a id="security_sasl_multimechanism" 
class="anchor-link"></a><a href="#security_sasl_multimechanism">Enabling 
multiple SASL mechanisms in a broker</a></h4>
             <ol>
-                <li>Specify configuration for the login modules of all enabled 
mechanisms in the <tt>KafkaServer</tt> section of the JAAS config file. For 
example:
+                <li>Specify configuration for the login modules of all enabled 
mechanisms in the <code>KafkaServer</code> section of the JAAS config file. For 
example:
                     <pre class="line-numbers"><code 
class="language-text">KafkaServer {
     com.sun.security.auth.module.Krb5LoginModule required
     useKeyTab=true
@@ -1165,12 +1165,12 @@ sasl.mechanism.inter.broker.protocol=GSSAPI (or one of 
the other enabled mechani
         <li><h4 class="anchor-heading"><a id="saslmechanism_rolling_upgrade" 
class="anchor-link"></a><a href="#saslmechanism_rolling_upgrade">Modifying SASL 
mechanism in a Running Cluster</a></h4>
             <p>SASL mechanism can be modified in a running cluster using the 
following sequence:</p>
             <ol>
-                <li>Enable new SASL mechanism by adding the mechanism to 
<tt>sasl.enabled.mechanisms</tt> in server.properties for each broker. Update 
JAAS config file to include both
+                <li>Enable new SASL mechanism by adding the mechanism to 
<code>sasl.enabled.mechanisms</code> in server.properties for each broker. 
Update JAAS config file to include both
                     mechanisms as described <a 
href="#security_sasl_multimechanism">here</a>. Incrementally bounce the cluster 
nodes.</li>
                 <li>Restart clients using the new mechanism.</li>
-                <li>To change the mechanism of inter-broker communication (if 
this is required), set <tt>sasl.mechanism.inter.broker.protocol</tt> in 
server.properties to the new mechanism and
+                <li>To change the mechanism of inter-broker communication (if 
this is required), set <code>sasl.mechanism.inter.broker.protocol</code> in 
server.properties to the new mechanism and
                     incrementally bounce the cluster again.</li>
-                <li>To remove old mechanism (if this is required), remove the 
old mechanism from <tt>sasl.enabled.mechanisms</tt> in server.properties and 
remove the entries for the
+                <li>To remove old mechanism (if this is required), remove the 
old mechanism from <code>sasl.enabled.mechanisms</code> in server.properties 
and remove the entries for the
                     old mechanism from JAAS config file. Incrementally bounce 
the cluster again.</li>
             </ol>
         </li>
@@ -1186,7 +1186,7 @@ sasl.mechanism.inter.broker.protocol=GSSAPI (or one of 
the other enabled mechani
             <p>Typical steps for delegation token usage are:</p>
             <ol>
                 <li>User authenticates with the Kafka cluster via SASL or SSL, 
and obtains a delegation token. This can be done
-                    using Admin APIs or using 
<tt>kafka-delegation-tokens.sh</tt> script.</li>
+                    using Admin APIs or using 
<code>kafka-delegation-tokens.sh</code> script.</li>
                 <li>User securely passes the delegation token to Kafka clients 
for authenticating with the Kafka cluster.</li>
                 <li>Token owner/renewer can renew/expire the delegation 
tokens.</li>
             </ol>
@@ -1194,7 +1194,7 @@ sasl.mechanism.inter.broker.protocol=GSSAPI (or one of 
the other enabled mechani
             <ol>
                 <li><h5 class="anchor-heading"><a 
id="security_token_management" class="anchor-link"></a><a 
href="#security_token_management">Token Management</a></h5>
                     <p> A secret is used to generate and verify delegation 
tokens. This is supplied using config
-                        option <tt>delegation.token.secret.key</tt>. The same 
secret key must be configured across all the brokers.
+                        option <code>delegation.token.secret.key</code>. The 
same secret key must be configured across all the brokers.
                         If using Kafka with KRaft the controllers must also be 
configured with the secret using the same config option.
                         If the secret is not set or set to empty string, 
delegation token authentication and API operations will fail.</p>
 
@@ -1206,21 +1206,21 @@ sasl.mechanism.inter.broker.protocol=GSSAPI (or one of 
the other enabled mechani
                         We intend to make these configurable in a future Kafka 
release.</p>
 
                     <p>A token has a current life, and a maximum renewable 
life. By default, tokens must be renewed once every 24 hours
-                        for up to 7 days. These can be configured using 
<tt>delegation.token.expiry.time.ms</tt>
-                        and <tt>delegation.token.max.lifetime.ms</tt> config 
options.</p>
+                        for up to 7 days. These can be configured using 
<code>delegation.token.expiry.time.ms</code>
+                        and <code>delegation.token.max.lifetime.ms</code> 
config options.</p>
 
                     <p>Tokens can also be cancelled explicitly.  If a token is 
not renewed by the token’s expiration time or if token
                         is beyond the max life time, it will be deleted from 
all broker caches as well as from zookeeper.</p>
                 </li>
 
                 <li><h5 class="anchor-heading"><a 
id="security_sasl_create_tokens" class="anchor-link"></a><a 
href="#security_sasl_create_tokens">Creating Delegation Tokens</a></h5>
-                    <p>Tokens can be created by using Admin APIs or using 
<tt>kafka-delegation-tokens.sh</tt> script.
+                    <p>Tokens can be created by using Admin APIs or using 
<code>kafka-delegation-tokens.sh</code> script.
                         Delegation token requests 
(create/renew/expire/describe) should be issued only on SASL or SSL 
authenticated channels.
                         Tokens can not be requests if the initial 
authentication is done through delegation token.
-                        A token can be created by the user for that user or 
others as well by specifying the <tt>--owner-principal</tt> parameter.
+                        A token can be created by the user for that user or 
others as well by specifying the <code>--owner-principal</code> parameter.
                         Owner/Renewers can renew or expire tokens. 
Owner/renewers can always describe their own tokens.
                         To describe other tokens, a DESCRIBE_TOKEN permission 
needs to be added on the User resource representing the owner of the token.
-                        <tt>kafka-delegation-tokens.sh</tt> script examples 
are given below.</p>
+                        <code>kafka-delegation-tokens.sh</code> script 
examples are given below.</p>
                     <p>Create a delegation token:
                     <pre class="line-numbers"><code class="language-bash">&gt; 
bin/kafka-delegation-tokens.sh --bootstrap-server localhost:9092 --create   
--max-life-time-period -1 --command-config client.properties 
--renewer-principal User:user1</code></pre>
                     <p>Create a delegation token for a different owner:
@@ -1246,14 +1246,14 @@ sasl.mechanism.inter.broker.protocol=GSSAPI (or one of 
the other enabled mechani
     password="lAYYSFmLs4bTjf+lTZ1LCHR/ZZFNA==" \
     tokenauth="true";</code></pre>
 
-                            <p>The options <tt>username</tt> and 
<tt>password</tt> are used by clients to configure the token id and
-                                token HMAC. And the option <tt>tokenauth</tt> 
is used to indicate the server about token authentication.
+                            <p>The options <code>username</code> and 
<code>password</code> are used by clients to configure the token id and
+                                token HMAC. And the option 
<code>tokenauth</code> is used to indicate the server about token 
authentication.
                                 In this example, clients connect to the broker 
using token id: <i>tokenID123</i>. Different clients within a
                                 JVM may connect using different tokens by 
specifying different token details in <code>sasl.jaas.config</code>.</p>
 
                             <p>JAAS configuration for clients may 
alternatively be specified as a JVM parameter similar to brokers
                                 as described <a 
href="#security_client_staticjaas">here</a>. Clients use the login section named
-                                <tt>KafkaClient</tt>. This option allows only 
one user for all client connections from a JVM.</p></li>
+                                <code>KafkaClient</code>. This option allows 
only one user for all client connections from a JVM.</p></li>
                     </ol>
                 </li>
 
@@ -1273,7 +1273,7 @@ sasl.mechanism.inter.broker.protocol=GSSAPI (or one of 
the other enabled mechani
     </ol>
 
     <h3 class="anchor-heading"><a id="security_authz" 
class="anchor-link"></a><a href="#security_authz">7.5 Authorization and 
ACLs</a></h3>
-    Kafka ships with a pluggable authorization framework, which is configured 
with the <tt>authorizer.class.name</tt> property in the server configuration.
+    Kafka ships with a pluggable authorization framework, which is configured 
with the <code>authorizer.class.name</code> property in the server 
configuration.
     Configured implementations must extend 
<code>org.apache.kafka.server.authorizer.Authorizer</code>.
     Kafka provides default implementations which store ACLs in the cluster 
metadata (either Zookeeper or the KRaft metadata log).
 
@@ -1332,7 +1332,7 @@ DEFAULT</code></pre>
     <h5 class="anchor-heading"><a id="security_authz_sasl" 
class="anchor-link"></a><a href="#security_authz_sasl">Customizing SASL User 
Name</a></h5>
 
     By default, the SASL user name will be the primary part of the Kerberos 
principal. One can change that by setting 
<code>sasl.kerberos.principal.to.local.rules</code> to a customized rule in 
server.properties.
-    The format of <code>sasl.kerberos.principal.to.local.rules</code> is a 
list where each rule works in the same way as the auth_to_local in <a 
href="http://web.mit.edu/Kerberos/krb5-latest/doc/admin/conf_files/krb5_conf.html";>Kerberos
 configuration file (krb5.conf)</a>. This also support additional 
lowercase/uppercase rule, to force the translated result to be all 
lowercase/uppercase. This is done by adding a "/L" or "/U" to the end of the 
rule. check below formats for syntax.
+    The format of <code>sasl.kerberos.principal.to.local.rules</code> is a 
list where each rule works in the same way as the auth_to_local in <a 
href="https://web.mit.edu/Kerberos/krb5-latest/doc/admin/conf_files/krb5_conf.html";>Kerberos
 configuration file (krb5.conf)</a>. This also support additional 
lowercase/uppercase rule, to force the translated result to be all 
lowercase/uppercase. This is done by adding a "/L" or "/U" to the end of the 
rule. check below formats for syntax.
     Each rules starts with RULE: and contains an expression as the following 
formats. See the kerberos documentation for more details.
     <pre class="line-numbers"><code 
class="language-text">RULE:[n:string](regexp)s/pattern/replacement/
 RULE:[n:string](regexp)s/pattern/replacement/g
@@ -2308,19 +2308,19 @@ security.inter.broker.protocol=SSL</code></pre>
         Use the broker properties file to set TLS configs for brokers as 
described below.
     </p>
     <p>
-        Use the <tt>--zk-tls-config-file &lt;file&gt;</tt> option to set TLS 
configs in the Zookeeper Security Migration Tool.
-        The <tt>kafka-acls.sh</tt> and <tt>kafka-configs.sh</tt> CLI tools 
also support the <tt>--zk-tls-config-file &lt;file&gt;</tt> option.
+        Use the <code>--zk-tls-config-file &lt;file&gt;</code> option to set 
TLS configs in the Zookeeper Security Migration Tool.
+        The <code>kafka-acls.sh</code> and <code>kafka-configs.sh</code> CLI 
tools also support the <code>--zk-tls-config-file &lt;file&gt;</code> option.
     </p>
     <p>
-        Use the <tt>-zk-tls-config-file &lt;file&gt;</tt> option (note the 
single-dash rather than double-dash)
-        to set TLS configs for the <tt>zookeeper-shell.sh</tt> CLI tool.
+        Use the <code>-zk-tls-config-file &lt;file&gt;</code> option (note the 
single-dash rather than double-dash)
+        to set TLS configs for the <code>zookeeper-shell.sh</code> CLI tool.
     </p>
     <h4 class="anchor-heading"><a id="zk_authz_new" class="anchor-link"></a><a 
href="#zk_authz_new">7.7.1 New clusters</a></h4>
     <h5 class="anchor-heading"><a id="zk_authz_new_sasl" 
class="anchor-link"></a><a href="#zk_authz_new_sasl">7.7.1.1 ZooKeeper SASL 
Authentication</a></h5>
     To enable ZooKeeper SASL authentication on brokers, there are two 
necessary steps:
     <ol>
         <li> Create a JAAS login file and set the appropriate system property 
to point to it as described above</li>
-        <li> Set the configuration property <tt>zookeeper.set.acl</tt> in each 
broker to true</li>
+        <li> Set the configuration property <code>zookeeper.set.acl</code> in 
each broker to true</li>
     </ol>
 
     The metadata stored in ZooKeeper for the Kafka cluster is world-readable, 
but can only be modified by the brokers. The rationale behind this decision is 
that the data stored in ZooKeeper is not sensitive, but inappropriate 
manipulation of that data can cause cluster disruption. We also recommend 
limiting the access to ZooKeeper via network segmentation (only brokers and 
some admin tools need access to ZooKeeper).
@@ -2333,10 +2333,10 @@ security.inter.broker.protocol=SSL</code></pre>
     hostname verification of the brokers and any CLI tool by ZooKeeper will 
succeed.
     <p>
         It is possible to use something other than the DN for the identity of 
mTLS clients by writing a class that
-        extends 
<tt>org.apache.zookeeper.server.auth.X509AuthenticationProvider</tt> and 
overrides the method
-        <tt>protected String getClientId(X509Certificate clientCert)</tt>.
-        Choose a scheme name and set <tt>authProvider.[scheme]</tt> in 
ZooKeeper to be the fully-qualified class name
-        of the custom implementation; then set 
<tt>ssl.authProvider=[scheme]</tt> to use it.
+        extends 
<code>org.apache.zookeeper.server.auth.X509AuthenticationProvider</code> and 
overrides the method
+        <code>protected String getClientId(X509Certificate clientCert)</code>.
+        Choose a scheme name and set <code>authProvider.[scheme]</code> in 
ZooKeeper to be the fully-qualified class name
+        of the custom implementation; then set 
<code>ssl.authProvider=[scheme]</code> to use it.
     </p>
     Here is a sample (partial) ZooKeeper configuration for enabling TLS 
authentication.
     These configurations are described in the
@@ -2387,13 +2387,13 @@ ssl.trustStore.password=zk-ts-passwd</code></pre>
         </li>
         <li>Perform a rolling restart of brokers setting the JAAS login file 
and/or defining ZooKeeper mutual TLS configurations (including connecting to 
the TLS-enabled ZooKeeper port) as required, which enables brokers to 
authenticate to ZooKeeper. At the end of the rolling restart, brokers are able 
to manipulate znodes with strict ACLs, but they will not create znodes with 
those ACLs</li>
         <li>If you enabled mTLS, disable the non-TLS port in ZooKeeper</li>
-        <li>Perform a second rolling restart of brokers, this time setting the 
configuration parameter <tt>zookeeper.set.acl</tt> to true, which enables the 
use of secure ACLs when creating znodes</li>
-        <li>Execute the ZkSecurityMigrator tool. To execute the tool, there is 
this script: <tt>bin/zookeeper-security-migration.sh</tt> with 
<tt>zookeeper.acl</tt> set to secure. This tool traverses the corresponding 
sub-trees changing the ACLs of the znodes. Use the <code>--zk-tls-config-file 
&lt;file&gt;</code> option if you enable mTLS.</li>
+        <li>Perform a second rolling restart of brokers, this time setting the 
configuration parameter <code>zookeeper.set.acl</code> to true, which enables 
the use of secure ACLs when creating znodes</li>
+        <li>Execute the ZkSecurityMigrator tool. To execute the tool, there is 
this script: <code>bin/zookeeper-security-migration.sh</code> with 
<code>zookeeper.acl</code> set to secure. This tool traverses the corresponding 
sub-trees changing the ACLs of the znodes. Use the <code>--zk-tls-config-file 
&lt;file&gt;</code> option if you enable mTLS.</li>
     </ol>
     <p>It is also possible to turn off authentication in a secure cluster. To 
do it, follow these steps:</p>
     <ol>
-        <li>Perform a rolling restart of brokers setting the JAAS login file 
and/or defining ZooKeeper mutual TLS configurations, which enables brokers to 
authenticate, but setting <tt>zookeeper.set.acl</tt> to false. At the end of 
the rolling restart, brokers stop creating znodes with secure ACLs, but are 
still able to authenticate and manipulate all znodes</li>
-        <li>Execute the ZkSecurityMigrator tool. To execute the tool, run this 
script <tt>bin/zookeeper-security-migration.sh</tt> with <tt>zookeeper.acl</tt> 
set to unsecure. This tool traverses the corresponding sub-trees changing the 
ACLs of the znodes. Use the <code>--zk-tls-config-file &lt;file&gt;</code> 
option if you need to set TLS configuration.</li>
+        <li>Perform a rolling restart of brokers setting the JAAS login file 
and/or defining ZooKeeper mutual TLS configurations, which enables brokers to 
authenticate, but setting <code>zookeeper.set.acl</code> to false. At the end 
of the rolling restart, brokers stop creating znodes with secure ACLs, but are 
still able to authenticate and manipulate all znodes</li>
+        <li>Execute the ZkSecurityMigrator tool. To execute the tool, run this 
script <code>bin/zookeeper-security-migration.sh</code> with 
<code>zookeeper.acl</code> set to unsecure. This tool traverses the 
corresponding sub-trees changing the ACLs of the znodes. Use the 
<code>--zk-tls-config-file &lt;file&gt;</code> option if you need to set TLS 
configuration.</li>
         <li>If you are disabling mTLS, enable the non-TLS port in 
ZooKeeper</li>
         <li>Perform a second rolling restart of brokers, this time omitting 
the system property that sets the JAAS login file and/or removing ZooKeeper 
mutual TLS configuration (including connecting to the non-TLS-enabled ZooKeeper 
port) as required</li>
         <li>If you are disabling mTLS, disable the TLS port in ZooKeeper</li>
@@ -2415,8 +2415,8 @@ ssl.trustStore.password=zk-ts-passwd</code></pre>
     <h3 class="anchor-heading"><a id="zk_encryption" 
class="anchor-link"></a><a href="#zk_encryption">7.8 ZooKeeper 
Encryption</a></h3>
     ZooKeeper connections that use mutual TLS are encrypted.
     Beginning with ZooKeeper version 3.5.7 (the version shipped with Kafka 
version 2.5) ZooKeeper supports a sever-side config
-    <tt>ssl.clientAuth</tt> (case-insensitively: 
<tt>want</tt>/<tt>need</tt>/<tt>none</tt> are the valid options, the default is 
<tt>need</tt>),
-    and setting this value to <tt>none</tt> in ZooKeeper allows clients to 
connect via a TLS-encrypted connection
+    <code>ssl.clientAuth</code> (case-insensitively: 
<code>want</code>/<code>need</code>/<code>none</code> are the valid options, 
the default is <code>need</code>),
+    and setting this value to <code>none</code> in ZooKeeper allows clients to 
connect via a TLS-encrypted connection
     without presenting their own certificate.  Here is a sample (partial) 
Kafka Broker configuration for connecting to ZooKeeper with just TLS encryption.
     These configurations are described above in <a 
href="#brokerconfigs">Broker Configs</a>.
     <pre class="line-numbers"><code class="language-text"># connect to the 
ZooKeeper port configured for TLS
diff --git a/docs/toc.html b/docs/toc.html
index 73bd66ae41c..d2902d8f164 100644
--- a/docs/toc.html
+++ b/docs/toc.html
@@ -98,7 +98,7 @@
                     </ul>
                 </li>
                 <li><a href="#datacenters">6.2 Datacenters</a></li>
-                <li><a href="#georeplication">6.3 Geo-Replication 
(Cross-Cluster Data Mirroring)</a></li>
+                <li><a href="#georeplication">6.3 Geo-Replication 
(Cross-Cluster Data Mirroring)</a>
                     <ul>
                         <li><a href="#georeplication-overview">Geo-Replication 
Overview</a></li>
                         <li><a href="#georeplication-flows">What Are 
Replication Flows</a></li>
@@ -109,7 +109,7 @@
                         <li><a href="#georeplication-monitoring">Monitoring 
Geo-Replication</a></li>
                     </ul>
                 </li>
-                <li><a href="#multitenancy">6.4 Multi-Tenancy</a></li>
+                <li><a href="#multitenancy">6.4 Multi-Tenancy</a>
                     <ul>
                         <li><a href="#multitenancy-overview">Multi-Tenancy 
Overview</a></li>
                         <li><a href="#multitenancy-topic-naming">Creating User 
Spaces (Namespaces)</a></li>
@@ -187,9 +187,9 @@
                 <li><a href="#security_sasl">7.4 Authentication using 
SASL</a></li>
                 <li><a href="#security_authz">7.5 Authorization and 
ACLs</a></li>
                 <li><a href="#security_rolling_upgrade">7.6 Incorporating 
Security Features in a Running Cluster</a></li>
-                <li><a href="#zk_authz">7.7 ZooKeeper Authentication</a></li>
+                <li><a href="#zk_authz">7.7 ZooKeeper Authentication</a>
                 <ul>
-                    <li><a href="#zk_authz_new">New Clusters</a></li>
+                    <li><a href="#zk_authz_new">New Clusters</a>
                     <ul>
                         <li><a href="#zk_authz_new_sasl">ZooKeeper SASL 
Authentication</a></li>
                         <li><a href="#zk_authz_new_mtls">ZooKeeper Mutual TLS 
Authentication</a></li>
@@ -204,7 +204,7 @@
         <li><a href="#connect">8. Kafka Connect</a>
             <ul>
                 <li><a href="#connect_overview">8.1 Overview</a></li>
-                <li><a href="#connect_user">8.2 User Guide</a></li>
+                <li><a href="#connect_user">8.2 User Guide</a>
                 <ul>
                     <li><a href="#connect_running">Running Kafka 
Connect</a></li>
                     <li><a href="#connect_configuring">Configuring 
Connectors</a></li>
@@ -214,7 +214,7 @@
                     <li><a href="#connect_exactlyonce">Exactly-once 
support</a></li>
                     <li><a href="#connect_plugindiscovery">Plugin 
Discovery</a></li>
                 </ul>
-                <li><a href="#connect_development">8.3 Connector Development 
Guide</a></li>
+                <li><a href="#connect_development">8.3 Connector Development 
Guide</a>
                 <ul>
                     <li><a href="#connect_concepts">Core Concepts and 
APIs</a></li>
                     <li><a href="#connect_developing">Developing a Simple 
Connector</a></li>
diff --git a/docs/upgrade.html b/docs/upgrade.html
index 5cd1b7cca86..d3713263206 100644
--- a/docs/upgrade.html
+++ b/docs/upgrade.html
@@ -1067,11 +1067,11 @@
         if there are no snapshot files in 3.4 data directory. For more details 
about the workaround please refer to <a 
href="https://cwiki.apache.org/confluence/display/ZOOKEEPER/Upgrade+FAQ";>ZooKeeper
 Upgrade FAQ</a>.
     </li>
     <li>
-        An embedded Jetty based <a 
href="http://zookeeper.apache.org/doc/r3.5.6/zookeeperAdmin.html#sc_adminserver";>AdminServer</a>
 added in ZooKeeper 3.5.
+        An embedded Jetty based <a 
href="https://zookeeper.apache.org/doc/r3.5.6/zookeeperAdmin.html#sc_adminserver";>AdminServer</a>
 added in ZooKeeper 3.5.
         AdminServer is enabled by default in ZooKeeper and is started on port 
8080.
         AdminServer is disabled by default in the ZooKeeper config 
(<code>zookeeper.properties</code>) provided by the Apache Kafka distribution.
         Make sure to update your local <code>zookeeper.properties</code> file 
with <code>admin.enableServer=false</code> if you wish to disable the 
AdminServer.
-        Please refer <a 
href="http://zookeeper.apache.org/doc/r3.5.6/zookeeperAdmin.html#sc_adminserver";>AdminServer
 config</a> to configure the AdminServer.
+        Please refer <a 
href="https://zookeeper.apache.org/doc/r3.5.6/zookeeperAdmin.html#sc_adminserver";>AdminServer
 config</a> to configure the AdminServer.
     </li>
 </ol>
 
diff --git a/docs/uses.html b/docs/uses.html
index 51f16a87267..94d22b47749 100644
--- a/docs/uses.html
+++ b/docs/uses.html
@@ -28,7 +28,7 @@ solution for large scale message processing applications.
 In our experience messaging uses are often comparatively low-throughput, but 
may require low end-to-end latency and often depend on the strong
 durability guarantees Kafka provides.
 <p>
-In this domain Kafka is comparable to traditional messaging systems such as <a 
href="http://activemq.apache.org";>ActiveMQ</a> or
+In this domain Kafka is comparable to traditional messaging systems such as <a 
href="https://activemq.apache.org";>ActiveMQ</a> or
 <a href="https://www.rabbitmq.com";>RabbitMQ</a>.
 
 <h4 class="anchor-heading"><a id="uses_website" class="anchor-link"></a><a 
href="#uses_website">Website Activity Tracking</a></h4>
@@ -66,11 +66,11 @@ Such processing pipelines create graphs of real-time data 
flows based on the ind
 Starting in 0.10.0.0, a light-weight but powerful stream processing library 
called <a href="/documentation/streams">Kafka Streams</a>
 is available in Apache Kafka to perform such data processing as described 
above.
 Apart from Kafka Streams, alternative open source stream processing tools 
include <a href="https://storm.apache.org/";>Apache Storm</a> and
-<a href="http://samza.apache.org/";>Apache Samza</a>.
+<a href="https://samza.apache.org/";>Apache Samza</a>.
 
 <h4 class="anchor-heading"><a id="uses_eventsourcing" 
class="anchor-link"></a><a href="#uses_eventsourcing">Event Sourcing</a></h4>
 
-<a href="http://martinfowler.com/eaaDev/EventSourcing.html";>Event sourcing</a> 
is a style of application design where state changes are logged as a
+<a href="https://martinfowler.com/eaaDev/EventSourcing.html";>Event 
sourcing</a> is a style of application design where state changes are logged as 
a
 time-ordered sequence of records. Kafka's support for very large stored log 
data makes it an excellent backend for an application built in this style.
 
 <h4 class="anchor-heading"><a id="uses_commitlog" class="anchor-link"></a><a 
href="#uses_commitlog">Commit Log</a></h4>


Reply via email to