├── .github ├── FUNDING.yml └── workflows │ └── ci.yml ├── .gitignore ├── LICENSE ├── README.md ├── docs ├── AbstractConfig.md ├── AbstractRequest.md ├── ApiRequestHandler.md ├── AutoTopicCreationManager.md ├── BrokerMetadataCheckpoint.md ├── Configurable.md ├── DefaultAutoTopicCreationManager.md ├── FetchResponseData.md ├── InterBrokerSendThread.md ├── Kafka.md ├── KafkaApis.md ├── KafkaConfig.md ├── Log.md ├── MetadataCache.md ├── Partition.md ├── PartitionData.md ├── ReplicaAlterLogDirsThread.md ├── ReplicaManager.md ├── RequestHandlerHelper.md ├── Serde.md ├── Serdes.md ├── Server.md ├── TopicConfig.md ├── Utils.md ├── ZkAdminManager.md ├── authentication.md ├── authorization │ ├── .pages │ ├── AclApis.md │ ├── AclAuthorizer.md │ ├── AclPublisher.md │ ├── AuthHelper.md │ ├── Authorizer.md │ ├── ClusterMetadataAuthorizer.md │ ├── StandardAuthorizer.md │ └── index.md ├── broker │ ├── .pages │ ├── KafkaBroker.md │ ├── KafkaServer.md │ ├── RaftControllerNodeProvider.md │ └── index.md ├── building-from-sources.md ├── clients │ ├── CommonClientConfigs.md │ ├── KafkaClient.md │ ├── Metadata.md │ ├── MetadataUpdater.md │ ├── NetworkClient.md │ ├── NetworkClientUtils.md │ ├── admin │ │ ├── Admin.md │ │ ├── AdminClientRunnable.md │ │ ├── KafkaAdminClient.md │ │ └── index.md │ ├── consumer │ │ ├── AbstractCoordinator.md │ │ ├── AbstractPartitionAssignor.md │ │ ├── Consumer.md │ │ ├── ConsumerConfig.md │ │ ├── ConsumerCoordinator.md │ │ ├── ConsumerMetadata.md │ │ ├── ConsumerNetworkClient.md │ │ ├── ConsumerPartitionAssignor.md │ │ ├── Fetcher.md │ │ ├── KafkaConsumer.md │ │ ├── SubscriptionState.md │ │ ├── TopicPartitionState.md │ │ ├── index.md │ │ └── preferred-read-replica.md │ ├── index.md │ └── producer │ │ ├── BufferPool.md │ │ ├── Callback.md │ │ ├── DefaultPartitioner.md │ │ ├── KafkaProducer.md │ │ ├── Partitioner.md │ │ ├── Producer.md │ │ ├── ProducerBatch.md │ │ ├── ProducerConfig.md │ │ ├── ProducerInterceptors.md │ │ ├── RecordAccumulator.md │ │ ├── Sender.md │ │ ├── TransactionManager.md │ │ └── index.md ├── consumer-groups │ ├── GroupCoordinator.md │ ├── GroupMetadataManager.md │ ├── OffsetConfig.md │ └── index.md ├── controller │ ├── AbstractControlRequest.md │ ├── AbstractControllerBrokerRequestBatch.md │ ├── AutoPreferredReplicaLeaderElection.md │ ├── ControllerBrokerRequestBatch.md │ ├── ControllerChannelManager.md │ ├── ControllerContext.md │ ├── ControllerEvent.md │ ├── ControllerEventManager.md │ ├── ControllerEventProcessor.md │ ├── ControllerEventThread.md │ ├── ControllerState.md │ ├── Election.md │ ├── KafkaController.md │ ├── LeaderAndIsrRequest.md │ ├── PartitionLeaderElectionAlgorithms.md │ ├── PartitionStateMachine.md │ ├── QueuedEvent.md │ ├── ReplicaStateMachine.md │ ├── TopicDeletionManager.md │ ├── TopicUncleanLeaderElectionEnable.md │ ├── ZkPartitionStateMachine.md │ ├── ZkReplicaStateMachine.md │ ├── controller-election.md │ └── index.md ├── demo │ ├── acl-authorization.md │ ├── controller-election.md │ ├── index.md │ ├── kafka-and-kcat-in-docker.md │ ├── partition-leader-election.md │ ├── secure-inter-broker-communication.md │ ├── securing-communication-between-clients-and-brokers.md │ ├── ssl-authentication.md │ └── transactional-kafka-producer.md ├── dynamic-broker-configuration │ ├── .pages │ ├── BrokerConfigHandler.md │ ├── BrokerReconfigurable.md │ ├── ConfigHandler.md │ ├── DynamicBrokerConfig.md │ ├── DynamicLogConfig.md │ ├── DynamicThreadPool.md │ ├── TopicConfigHandler.md │ ├── ZkConfigManager.md │ ├── ZkNodeChangeNotificationListener.md │ └── index.md ├── dynamic-configuration │ ├── .pages │ ├── Reconfigurable.md │ └── index.md ├── features │ └── index.md ├── images │ ├── AclAuthorizer-jconsole.png │ ├── ControllerEventManager-jconsole.png │ ├── ControllerEventManager.png │ ├── ControllerEventThread-doWork.png │ ├── GroupCoordinator-startup.png │ ├── GroupMetadataManager-creating-instance.png │ ├── GroupMetadataManager-jconsole.png │ ├── KafkaApis.png │ ├── KafkaController.png │ ├── LogCleanerManager-jconsole.png │ ├── LogManager.png │ ├── ZkPartitionStateMachine.png │ └── kafka-controller-jconsole.png ├── index.md ├── kraft │ ├── .pages │ ├── BrokerServer.md │ ├── ControllerServer.md │ ├── FileBasedStateStore.md │ ├── KafkaMetadataLog.md │ ├── KafkaRaftClient.ListenerContext.md │ ├── KafkaRaftClient.md │ ├── KafkaRaftManager.md │ ├── KafkaRaftServer.md │ ├── MetaProperties.md │ ├── QuorumState.md │ ├── QuorumStateStore.md │ ├── RaftClient.Listener.md │ ├── RaftClient.md │ ├── RaftConfig.md │ ├── RaftIoThread.md │ ├── RaftManager.md │ ├── SharedServer.md │ └── index.md ├── log-cleanup │ └── index.md ├── log │ ├── CleanerConfig.md │ ├── CleanerThread.md │ ├── LocalLog.md │ ├── LogCleaner.md │ ├── LogCleanerManager.md │ ├── LogConfig.md │ ├── LogLoader.md │ ├── LogManager.md │ ├── LogSegment.md │ ├── UnifiedLog.md │ └── index.md ├── logging.md ├── metadata │ ├── .pages │ ├── BrokerMetadataPublisher.md │ ├── LoaderManifest.md │ ├── MetadataLoader.md │ ├── MetadataLoaderMetrics.md │ ├── MetadataPublisher.md │ ├── QuorumController.md │ ├── SnapshotGenerator.md │ ├── SnapshotManifest.md │ └── index.md ├── metrics │ ├── .pages │ ├── KafkaMetricsGroup.md │ ├── MetricConfig.md │ ├── Metrics.md │ ├── MetricsReporter.md │ ├── Sensor.md │ └── index.md ├── overview.md ├── partition-leader-election │ └── index.md ├── tiered-storage │ ├── .pages │ ├── RemoteLogManager.md │ ├── RemoteLogManagerConfig.md │ └── index.md ├── tools │ ├── ConsoleConsumer.md │ ├── index.md │ ├── kafka-acls │ │ ├── AclCommand.md │ │ ├── AclCommandOptions.md │ │ ├── AclCommandService.md │ │ ├── AdminClientService.md │ │ ├── AuthorizerService.md │ │ └── index.md │ ├── kafka-cluster │ │ └── index.md │ ├── kafka-configs │ │ ├── ConfigCommand.md │ │ └── index.md │ ├── kafka-dump-log │ │ └── index.md │ ├── kafka-get-offsets │ │ ├── GetOffsetShell.md │ │ └── index.md │ ├── kafka-leader-election │ │ └── index.md │ ├── kafka-metadata-shell │ │ ├── MetadataShell.md │ │ ├── MetadataShellPublisher.md │ │ └── index.md │ ├── kafka-reassign-partitions │ │ ├── ReassignPartitionsCommand.md │ │ ├── demo.md │ │ └── index.md │ ├── kafka-replica-verification │ │ ├── ReplicaBuffer.md │ │ ├── ReplicaFetcher.md │ │ ├── ReplicaVerificationTool.md │ │ └── index.md │ └── kafka-storage │ │ ├── StorageTool.md │ │ └── index.md ├── transactions │ ├── TransactionConfig.md │ ├── TransactionCoordinator.md │ ├── TransactionMarkerChannelManager.md │ ├── TransactionStateManager.md │ └── index.md └── zk │ ├── AdminZkClient.md │ ├── KafkaZkClient.md │ └── TopicPartitionStateZNode.md ├── graffles ├── ControllerEventManager.graffle ├── ControllerEventThread-doWork.graffle ├── GroupCoordinator-startup.graffle ├── GroupMetadataManager-creating-instance.graffle ├── KafkaApis.graffle ├── KafkaController.graffle ├── LogManager.graffle └── ZkPartitionStateMachine.graffle ├── mkdocs.yml └── requirements.txt /.github/FUNDING.yml: -------------------------------------------------------------------------------- 1 | github: jaceklaskowski 2 | ko_fi: jaceklaskowski 3 | custom: "https://paypal.me/JacekLaskowski" 4 | -------------------------------------------------------------------------------- /.github/workflows/ci.yml: -------------------------------------------------------------------------------- 1 | # Based on https://github.com/squidfunk/mkdocs-material/blob/master/.github/workflows/ci.yml 2 | 3 | name: Deploying the docs 4 | 5 | on: 6 | push: 7 | branches: 8 | - main 9 | 10 | jobs: 11 | deploy: 12 | if: github.event.pull_request.head.repo.fork == false 13 | runs-on: ubuntu-latest 14 | 15 | steps: 16 | - uses: actions/checkout@v2 17 | with: 18 | fetch-depth: 0 19 | - uses: actions/setup-python@v2 20 | with: 21 | python-version: 3.x 22 | - name: Install dependencies 23 | env: 24 | GH_TOKEN: ${{ secrets.GH_TOKEN }} 25 | run: | 26 | pip install -r requirements.txt 27 | - name: Build documentation 28 | env: 29 | GOOGLE_ANALYTICS_KEY: ${{ secrets.GOOGLE_ANALYTICS_KEY }} 30 | run: | 31 | mkdocs gh-deploy --force 32 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | site/ 2 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # The Internals of Apache Kafka Online Book 2 | 3 | [![Deploying the docs](https://github.com/japila-books/kafka-internals/actions/workflows/ci.yml/badge.svg)](https://github.com/japila-books/kafka-internals/actions/workflows/ci.yml) 4 | 5 | The project contains the sources of [The Internals of Apache Kafka](https://books.japila.pl/kafka-internals) online book. 6 | -------------------------------------------------------------------------------- /docs/AbstractConfig.md: -------------------------------------------------------------------------------- 1 | # AbstractConfig 2 | 3 | `AbstractConfig` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/AbstractRequest.md: -------------------------------------------------------------------------------- 1 | # AbstractRequest 2 | 3 | `AbstractRequest` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/ApiRequestHandler.md: -------------------------------------------------------------------------------- 1 | # ApiRequestHandler 2 | 3 | `ApiRequestHandler` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/AutoTopicCreationManager.md: -------------------------------------------------------------------------------- 1 | # AutoTopicCreationManager 2 | 3 | `AutoTopicCreationManager` is an [abstraction](#contract) of [managers](#implementations) that can [create topics](#createTopics). 4 | 5 | ## Contract 6 | 7 | ###  createTopics 8 | 9 | ```scala 10 | createTopics( 11 | topicNames: Set[String], 12 | controllerMutationQuota: ControllerMutationQuota): Seq[MetadataResponseTopic] 13 | ``` 14 | 15 | Used when: 16 | 17 | * `KafkaApis` is requested to [getTopicMetadata](KafkaApis.md#getTopicMetadata) and [handleFindCoordinatorRequest](KafkaApis.md#handleFindCoordinatorRequest) 18 | 19 | ###  shutdown 20 | 21 | ```scala 22 | shutdown(): Unit 23 | ``` 24 | 25 | Used when: 26 | 27 | * `BrokerServer` is requested to [shutdown](kraft/BrokerServer.md#shutdown) 28 | * `KafkaServer` is requested to [shutdown](broker/KafkaServer.md#shutdown) 29 | 30 | ###  start 31 | 32 | ```scala 33 | start(): Unit 34 | ``` 35 | 36 | Used when: 37 | 38 | * `BrokerServer` is requested to [startup](kraft/BrokerServer.md#startup) 39 | * `KafkaServer` is requested to [startup](broker/KafkaServer.md#startup) 40 | 41 | ## Implementations 42 | 43 | * [DefaultAutoTopicCreationManager](DefaultAutoTopicCreationManager.md) 44 | 45 | ## Creating AutoTopicCreationManager 46 | 47 | ```scala 48 | apply( 49 | config: KafkaConfig, 50 | metadataCache: MetadataCache, 51 | time: Time, 52 | metrics: Metrics, 53 | threadNamePrefix: Option[String], 54 | adminManager: Option[ZkAdminManager], 55 | controller: Option[KafkaController], 56 | groupCoordinator: GroupCoordinator, 57 | txnCoordinator: TransactionCoordinator, 58 | enableForwarding: Boolean): AutoTopicCreationManager 59 | ``` 60 | 61 | `apply` creates a [DefaultAutoTopicCreationManager](DefaultAutoTopicCreationManager.md). 62 | 63 | `apply` is used when: 64 | 65 | * `KafkaServer` is requested to [startup](broker/KafkaServer.md#autoTopicCreationManager) 66 | -------------------------------------------------------------------------------- /docs/BrokerMetadataCheckpoint.md: -------------------------------------------------------------------------------- 1 | # BrokerMetadataCheckpoint 2 | 3 | `BrokerMetadataCheckpoint` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/Configurable.md: -------------------------------------------------------------------------------- 1 | # Configurable 2 | 3 | `Configurable` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/FetchResponseData.md: -------------------------------------------------------------------------------- 1 | # FetchResponseData 2 | 3 | `FetchResponseData` is a `ApiMessage`. 4 | -------------------------------------------------------------------------------- /docs/InterBrokerSendThread.md: -------------------------------------------------------------------------------- 1 | # InterBrokerSendThread 2 | -------------------------------------------------------------------------------- /docs/Kafka.md: -------------------------------------------------------------------------------- 1 | # Kafka Utility 2 | 3 | `kafka.Kafka` is a [command-line application](#main) that is executed using `kafka-server-start` shell script. 4 | 5 | ## Launching Kafka Server 6 | 7 | ```scala 8 | main( 9 | args: Array[String]): Unit 10 | ``` 11 | 12 | `main`...FIXME 13 | 14 | ### buildServer 15 | 16 | ```scala 17 | buildServer( 18 | props: Properties): Server 19 | ``` 20 | 21 | `buildServer` [creates a KafkaConfig](KafkaConfig.md#fromProps) (from the given `Properties`). 22 | 23 | `buildServer` creates a [Server](Server.md) based on [process.roles](KafkaConfig.md#requiresZookeeper) configuration property: 24 | 25 | * [KafkaServer](broker/KafkaServer.md) if [empty](KafkaConfig.md#requiresZookeeper) 26 | * [KafkaRaftServer](kraft/KafkaRaftServer.md), otherwise 27 | -------------------------------------------------------------------------------- /docs/Log.md: -------------------------------------------------------------------------------- 1 | # Log 2 | 3 | ## Creating Instance 4 | 5 | `Log` takes the following to be created: 6 | 7 | * Directory 8 | * `LogConfig` 9 | * `LogSegments` 10 | * logStartOffset 11 | * recoveryPoint 12 | * `LogOffsetMetadata` 13 | * `Scheduler` 14 | * `BrokerTopicStats` 15 | * `Time` 16 | * producerIdExpirationCheckIntervalMs 17 | * `TopicPartition` 18 | * Optional `LeaderEpochFileCache` 19 | * `ProducerStateManager` 20 | * `LogDirFailureChannel` 21 | * Optional Topic ID 22 | * keepPartitionMetadataFile 23 | 24 | `Log` is created using [apply](#apply) utility. 25 | 26 | ## Creating Log 27 | 28 | ```scala 29 | apply( 30 | dir: File, 31 | config: LogConfig, 32 | logStartOffset: Long, 33 | recoveryPoint: Long, 34 | scheduler: Scheduler, 35 | brokerTopicStats: BrokerTopicStats, 36 | time: Time = Time.SYSTEM, 37 | maxProducerIdExpirationMs: Int, 38 | producerIdExpirationCheckIntervalMs: Int, 39 | logDirFailureChannel: LogDirFailureChannel, 40 | lastShutdownClean: Boolean = true, 41 | topicId: Option[Uuid], 42 | keepPartitionMetadataFile: Boolean): Log 43 | ``` 44 | 45 | `apply`...FIXME 46 | 47 | `apply` is used when: 48 | 49 | * `LogManager` is requested to `loadLog` and `getOrCreateLog` 50 | * `KafkaMetadataLog` is requested to `apply` 51 | 52 | ## Reading Messages { #read } 53 | 54 | ```scala 55 | read( 56 | startOffset: Long, 57 | maxLength: Int, 58 | isolation: FetchIsolation, 59 | minOneMessage: Boolean): FetchDataInfo 60 | ``` 61 | 62 | `read` prints out the following TRACE message to the logs: 63 | 64 | ```text 65 | Reading maximum [maxLength] bytes at offset [startOffset] from log with total length [size] bytes 66 | ``` 67 | 68 | `read`...FIXME 69 | 70 | `read` requests the `LogSegment` to [read messages](log/LogSegment.md#read). 71 | 72 | `read`...FIXME 73 | 74 | --- 75 | 76 | `read` is used when: 77 | 78 | * `Partition` is requested to [readRecords](Partition.md#readRecords) 79 | * `GroupMetadataManager` is requested to `doLoadGroupsAndOffsets` 80 | * `TransactionStateManager` is requested to `loadTransactionMetadata` 81 | * `Log` is requested to [convertToOffsetMetadataOrThrow](#convertToOffsetMetadataOrThrow) 82 | * `KafkaMetadataLog` is requested to `read` 83 | 84 | ## Logging 85 | 86 | Enable `ALL` logging level for `kafka.log.Log` logger to see what happens inside. 87 | 88 | Add the following line to `log4j.properties`: 89 | 90 | ```text 91 | log4j.logger.kafka.log.Log=ALL 92 | ``` 93 | 94 | Refer to [Logging](logging.md). 95 | -------------------------------------------------------------------------------- /docs/MetadataCache.md: -------------------------------------------------------------------------------- 1 | # MetadataCache 2 | 3 | `MetadataCache` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/Partition.md: -------------------------------------------------------------------------------- 1 | # Partition 2 | 3 | ## readRecords 4 | 5 | ```scala 6 | readRecords( 7 | lastFetchedEpoch: Optional[Integer], 8 | fetchOffset: Long, 9 | currentLeaderEpoch: Optional[Integer], 10 | maxBytes: Int, 11 | fetchIsolation: FetchIsolation, 12 | fetchOnlyFromLeader: Boolean, 13 | minOneMessage: Boolean): LogReadInfo 14 | ``` 15 | 16 | `readRecords`...FIXME 17 | 18 | In the end, `readRecords` requests the `Log` to [read messages](Log.md#read). 19 | 20 | --- 21 | 22 | `readRecords` is used when: 23 | 24 | * `ReplicaManager` is requested to [readFromLocalLog](ReplicaManager.md#readFromLocalLog) 25 | 26 | ## makeLeader 27 | 28 | ```scala 29 | makeLeader( 30 | partitionState: LeaderAndIsrPartitionState, 31 | highWatermarkCheckpoints: OffsetCheckpoints, 32 | topicId: Option[Uuid]): Boolean 33 | ``` 34 | 35 | `makeLeader`...FIXME 36 | 37 | --- 38 | 39 | `makeLeader` is used when: 40 | 41 | * `ReplicaManager` is requested to [makeLeaders](ReplicaManager.md#makeLeaders), [applyLocalLeadersDelta](ReplicaManager.md#applyLocalLeadersDelta) 42 | 43 | ## createLogIfNotExists 44 | 45 | ```scala 46 | createLogIfNotExists( 47 | isNew: Boolean, 48 | isFutureReplica: Boolean, 49 | offsetCheckpoints: OffsetCheckpoints, 50 | topicId: Option[Uuid]): Unit 51 | ``` 52 | 53 | `createLogIfNotExists`...FIXME 54 | 55 | --- 56 | 57 | `createLogIfNotExists` is used when: 58 | 59 | * `Partition` is requested to [maybeCreateFutureReplica](#maybeCreateFutureReplica), [makeLeader](#makeLeader), [makeFollower](#makeFollower) 60 | * `ReplicaManager` is requested to [maybeAddLogDirFetchers](ReplicaManager.md#maybeAddLogDirFetchers), [makeFollowers](ReplicaManager.md#makeFollowers), [applyLocalFollowersDelta](ReplicaManager.md#applyLocalFollowersDelta) 61 | 62 | ### createLog 63 | 64 | ```scala 65 | createLog( 66 | isNew: Boolean, 67 | isFutureReplica: Boolean, 68 | offsetCheckpoints: OffsetCheckpoints, 69 | topicId: Option[Uuid]): UnifiedLog 70 | ``` 71 | 72 | `createLog`...FIXME 73 | -------------------------------------------------------------------------------- /docs/PartitionData.md: -------------------------------------------------------------------------------- 1 | # PartitionData 2 | 3 | `PartitionData` is a `Message` of [FetchResponseData](FetchResponseData.md). 4 | 5 | ## abortedTransactions 6 | 7 | ```java 8 | List abortedTransactions 9 | List abortedTransactions() 10 | ``` 11 | 12 | `abortedTransactions`...FIXME 13 | 14 | `abortedTransactions` is used when: 15 | 16 | * FIXME 17 | 18 | ## Preferred Read Replica 19 | 20 | Default: `-1` 21 | 22 | `preferredReadReplica` is used when: 23 | 24 | * `Fetcher` is requested to [initializeCompletedFetch](clients/consumer/Fetcher.md#initializeCompletedFetch) 25 | -------------------------------------------------------------------------------- /docs/ReplicaAlterLogDirsThread.md: -------------------------------------------------------------------------------- 1 | # ReplicaAlterLogDirsThread 2 | 3 | `ReplicaAlterLogDirsThread` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/ReplicaManager.md: -------------------------------------------------------------------------------- 1 | # ReplicaManager 2 | 3 | ## fetchMessages 4 | 5 | ```scala 6 | fetchMessages( 7 | timeout: Long, 8 | replicaId: Int, 9 | fetchMinBytes: Int, 10 | fetchMaxBytes: Int, 11 | hardMaxBytesLimit: Boolean, 12 | fetchInfos: Seq[(TopicPartition, PartitionData)], 13 | quota: ReplicaQuota, 14 | responseCallback: Seq[(TopicPartition, FetchPartitionData)] => Unit, 15 | isolationLevel: IsolationLevel, 16 | clientMetadata: Option[ClientMetadata]): Unit 17 | ``` 18 | 19 | `fetchMessages` determines whether the request comes from a follower or a consumer (based on the given `replicaId`). 20 | 21 | `fetchMessages` determines `FetchIsolation`: 22 | 23 | * `FetchLogEnd` if the request comes from a follower 24 | * `FetchTxnCommitted` if the request comes from a consumer with `READ_COMMITTED` isolation level 25 | * `FetchHighWatermark` otherwise 26 | 27 | `fetchMessages` [readFromLocalLog](#readFromLocalLog) (passing in the `FetchIsolation` among the others). 28 | 29 | `fetchMessages`...FIXME 30 | 31 | `fetchMessages` is used when: 32 | 33 | * `KafkaApis` is requested to [handle a Fetch request](KafkaApis.md#handleFetchRequest) 34 | * `ReplicaAlterLogDirsThread` is requested to [fetchFromLeader](ReplicaAlterLogDirsThread.md#fetchFromLeader) 35 | 36 | ## readFromLocalLog 37 | 38 | ```scala 39 | readFromLocalLog( 40 | replicaId: Int, 41 | fetchOnlyFromLeader: Boolean, 42 | fetchIsolation: FetchIsolation, 43 | fetchMaxBytes: Int, 44 | hardMaxBytesLimit: Boolean, 45 | readPartitionInfo: Seq[(TopicPartition, PartitionData)], 46 | quota: ReplicaQuota, 47 | clientMetadata: Option[ClientMetadata]): Seq[(TopicPartition, LogReadResult)] 48 | ``` 49 | 50 | `readFromLocalLog`...FIXME 51 | 52 | `readFromLocalLog` finds the `Partition` and requests it to [readRecords](Partition.md#readRecords). 53 | 54 | `readFromLocalLog`...FIXME 55 | 56 | `readFromLocalLog` is used when: 57 | 58 | * `DelayedFetch` is requested to `onComplete` 59 | * `ReplicaManager` is requested to [fetchMessages](#fetchMessages) 60 | 61 | ## becomeLeaderOrFollower 62 | 63 | ```scala 64 | becomeLeaderOrFollower( 65 | correlationId: Int, 66 | leaderAndIsrRequest: LeaderAndIsrRequest, 67 | onLeadershipChange: (Iterable[Partition], Iterable[Partition]) => Unit): LeaderAndIsrResponse 68 | ``` 69 | 70 | `becomeLeaderOrFollower`...FIXME 71 | 72 | --- 73 | 74 | `becomeLeaderOrFollower` is used when: 75 | 76 | * `KafkaApis` is requested to [handleLeaderAndIsrRequest](KafkaApis.md#handleLeaderAndIsrRequest) 77 | 78 | ### makeLeaders 79 | 80 | ```scala 81 | makeLeaders( 82 | controllerId: Int, 83 | controllerEpoch: Int, 84 | partitionStates: Map[Partition, LeaderAndIsrPartitionState], 85 | correlationId: Int, 86 | responseMap: mutable.Map[TopicPartition, Errors], 87 | highWatermarkCheckpoints: OffsetCheckpoints, 88 | topicIds: String => Option[Uuid]): Set[Partition] 89 | ``` 90 | 91 | `makeLeaders`...FIXME 92 | -------------------------------------------------------------------------------- /docs/RequestHandlerHelper.md: -------------------------------------------------------------------------------- 1 | # RequestHandlerHelper 2 | 3 | ## Creating Instance 4 | 5 | `RequestHandlerHelper` takes the following to be created: 6 | 7 | * `RequestChannel` 8 | * `QuotaManagers` 9 | * `Time` 10 | * Log Prefix 11 | 12 | `RequestHandlerHelper` is created when: 13 | 14 | * `ControllerApis` is created (`requestHelper`) 15 | * `KafkaApis` is [created](KafkaApis.md#requestHelper) 16 | 17 | ## onLeadershipChange 18 | 19 | ```scala 20 | onLeadershipChange( 21 | groupCoordinator: GroupCoordinator, 22 | txnCoordinator: TransactionCoordinator, 23 | updatedLeaders: Iterable[Partition], 24 | updatedFollowers: Iterable[Partition]): Unit 25 | ``` 26 | 27 | `onLeadershipChange`...FIXME 28 | 29 | `onLeadershipChange` is used when: 30 | 31 | * `BrokerServer` is requested to [start up](kraft/BrokerServer.md#startup) 32 | * `KafkaApis` is requested to [handleLeaderAndIsrRequest](KafkaApis.md#handleLeaderAndIsrRequest) 33 | * `BrokerMetadataListener` is requested to `handleCommits` and `execCommits` 34 | -------------------------------------------------------------------------------- /docs/Serde.md: -------------------------------------------------------------------------------- 1 | # Serde 2 | 3 | `Serde` is an [abstraction](#contract) of [wrappers](#implementations) with [Serializer](#serializer)s and [Deserializer](#deserializer)s. 4 | 5 | !!! note 6 | `Serde` seems to be of more use in libraries like Kafka Streams or Kafka Connect (that in the Kafka Core). 7 | 8 | ## Contract 9 | 10 | ###  configure 11 | 12 | ```java 13 | void configure( 14 | Map configs, 15 | boolean isKey) 16 | ``` 17 | 18 | ###  deserializer 19 | 20 | ```java 21 | Deserializer deserializer() 22 | ``` 23 | 24 | ###  serializer 25 | 26 | ```java 27 | Serializer serializer() 28 | ``` 29 | 30 | ## Implementations 31 | 32 | * [WrapperSerde](Serdes.md#WrapperSerde) 33 | -------------------------------------------------------------------------------- /docs/Serdes.md: -------------------------------------------------------------------------------- 1 | # Serdes Utility 2 | 3 | `Serdes` is a utility with the serializers and deserializers for many built-in types in Java and [allows defining new ones](#serdeFrom). 4 | 5 | ```text 6 | import org.apache.kafka.common.serialization.Serdes 7 | 8 | val longSerde = Serdes.Long 9 | scala> :type longSerde 10 | org.apache.kafka.common.serialization.Serde[Long] 11 | 12 | scala> :type longSerde.serializer 13 | org.apache.kafka.common.serialization.Serializer[Long] 14 | 15 | scala> :type longSerde.deserializer 16 | org.apache.kafka.common.serialization.Deserializer[Long] 17 | ``` 18 | 19 | ## WrapperSerde 20 | 21 | `Serdes` defines a `static public class WrapperSerde` that is a [Serde](Serde.md) from and to `T` values. 22 | 23 | * `ShortSerde` 24 | * `BytesSerde` 25 | * `IntegerSerde` 26 | * `ListSerde` 27 | * `UUIDSerde` 28 | * `FloatSerde` 29 | * `FullTimeWindowedSerde` (Kafka Streams) 30 | * `VoidSerde` 31 | * `LongSerde` 32 | * `DoubleSerde` 33 | * `ByteArraySerde` 34 | * `TimeWindowedSerde` (Kafka Streams) 35 | * `SessionWindowedSerde` (Kafka Streams) 36 | * `ByteBufferSerde` 37 | * `StringSerde` 38 | 39 | ## serdeFrom 40 | 41 | ```java 42 | Serde serdeFrom( 43 | Class type) 44 | Serde serdeFrom( 45 | Serializer serializer, 46 | Deserializer deserializer) 47 | ``` 48 | 49 | `serdeFrom` looks up the [Serde](Serde.md) for a given type (if supported) or creates a new one based on the given pair of `Serializer` and `Deserializer`. 50 | -------------------------------------------------------------------------------- /docs/Server.md: -------------------------------------------------------------------------------- 1 | # Server 2 | 3 | `Server` is an [abstraction](#contract) of [Kafka servers](#implementations) (_brokers_) for [Kafka](Kafka.md) utility. 4 | 5 | ## Contract 6 | 7 | ###  awaitShutdown 8 | 9 | ```scala 10 | awaitShutdown(): Unit 11 | ``` 12 | 13 | Awaits termination signal (and keeps this server alive) 14 | 15 | Used when: 16 | 17 | * `Kafka` utility is [executed](Kafka.md#main) (on command line) 18 | 19 | ###  Starting Up 20 | 21 | ```scala 22 | startup(): Unit 23 | ``` 24 | 25 | Starts up this server 26 | 27 | Used when: 28 | 29 | * `Kafka` utility is [executed](Kafka.md#main) (on command line) 30 | 31 | ###  shutdown 32 | 33 | ```scala 34 | shutdown(): Unit 35 | ``` 36 | 37 | Shuts down this server (as part of `kafka-shutdown-hook` shutdown hook) 38 | 39 | Used when: 40 | 41 | * `Kafka` utility is [executed](Kafka.md#main) (on command line and handles termination signals) 42 | 43 | ## Implementations 44 | 45 | * [KafkaRaftServer](kraft/KafkaRaftServer.md) 46 | * [KafkaServer](broker/KafkaServer.md) 47 | -------------------------------------------------------------------------------- /docs/TopicConfig.md: -------------------------------------------------------------------------------- 1 | # TopicConfig 2 | 3 | `TopicConfig` is configuration properties of Kafka topics. In other words, `TopicConfig` is a topic-specific configuration properties (while [KafkaConfig](KafkaConfig.md) is broker-wide). 4 | 5 | !!! tip 6 | While reviewing the source code it can get tricky to find broker-wide properties (e.g. `log.cleanup.policy`). The reason is that broker-wide properties as split into `log.` prefix and a corresponding topic-specific property. 7 | 8 | A solution is to search for a topic-specific property removing `log.` prefix. 9 | 10 | ## cleanup.policy 11 | 12 | A comma-separated list of [cleanup policies](log-cleanup/index.md): 13 | 14 | * `compact` 15 | * `delete` 16 | 17 | Default: `delete` 18 | 19 | Broker-wide configuration: [log.cleanup.policy](KafkaConfig.md#LogCleanupPolicyProp) 20 | 21 | Used when: 22 | 23 | * `GroupCoordinator` is requested for the [offsetsTopicConfigs](consumer-groups/GroupCoordinator.md#offsetsTopicConfigs) 24 | * `LogConfig` is requested to [compact](log/LogConfig.md#compact), [delete](log/LogConfig.md#delete), [TopicConfigSynonyms](log/LogConfig.md#TopicConfigSynonyms) and [extractLogConfigMap](log/LogConfig.md#extractLogConfigMap) 25 | * `TopicBasedRemoteLogMetadataManager` is requested to `createRemoteLogMetadataTopicRequest` 26 | * `TransactionStateManager` is requested for the [transactionTopicConfigs](transactions/TransactionStateManager.md#transactionTopicConfigs) 27 | 28 | ## unclean.leader.election.enable 29 | 30 | [LogConfig](log/LogConfig.md#UncleanLeaderElectionEnableProp) 31 | -------------------------------------------------------------------------------- /docs/Utils.md: -------------------------------------------------------------------------------- 1 | # Utils 2 | 3 | ## murmur2 4 | 5 | ```java 6 | int murmur2( 7 | byte[] data) 8 | ``` 9 | 10 | `murmur2` generates a 32-bit murmur2 hash for the given byte array. 11 | 12 | `murmur2` is used when: 13 | 14 | * `DefaultPartitioner` is requested to [compute a partition for a record](clients/producer/DefaultPartitioner.md#partition) 15 | 16 | ### Demo 17 | 18 | ```text 19 | import org.apache.kafka.common.utils.Utils 20 | 21 | val keyBytes = "hello".getBytes 22 | val hash = Utils.murmur2(keyBytes) 23 | 24 | println(hash) 25 | ``` 26 | 27 | ## toPositive 28 | 29 | ```java 30 | int toPositive( 31 | int number) 32 | ``` 33 | 34 | `toPositive` converts a number to a positive value. 35 | -------------------------------------------------------------------------------- /docs/ZkAdminManager.md: -------------------------------------------------------------------------------- 1 | # ZkAdminManager 2 | 3 | ## incrementalAlterConfigs 4 | 5 | ```scala 6 | incrementalAlterConfigs( 7 | configs: Map[ConfigResource, 8 | Seq[AlterConfigOp]], 9 | validateOnly: Boolean): Map[ConfigResource, ApiError] 10 | ``` 11 | 12 | `incrementalAlterConfigs`...FIXME 13 | 14 | --- 15 | 16 | `incrementalAlterConfigs` is used when: 17 | 18 | * `KafkaApis` is requested to [processIncrementalAlterConfigsRequest](KafkaApis.md#processIncrementalAlterConfigsRequest) 19 | 20 | ## alterConfigs 21 | 22 | ```scala 23 | alterConfigs( 24 | configs: Map[ConfigResource, 25 | AlterConfigsRequest.Config], 26 | validateOnly: Boolean): Map[ConfigResource, ApiError] 27 | ``` 28 | 29 | `alterConfigs`...FIXME 30 | 31 | --- 32 | 33 | `alterConfigs` is used when: 34 | 35 | * `KafkaApis` is requested to [processLegacyAlterConfigsRequest](KafkaApis.md#processLegacyAlterConfigsRequest) 36 | 37 | ## alterTopicConfigs 38 | 39 | ```scala 40 | alterTopicConfigs( 41 | resource: ConfigResource, 42 | validateOnly: Boolean, 43 | configProps: Properties, 44 | configEntriesMap: Map[String, String]): (ConfigResource, ApiError) 45 | ``` 46 | 47 | `alterTopicConfigs`...FIXME 48 | 49 | --- 50 | 51 | `alterTopicConfigs` is used when: 52 | 53 | * `ZkAdminManager` is requested to [alterConfigs](#alterConfigs) (of a topic), [incrementalAlterConfigs](#incrementalAlterConfigs) (of a topic) 54 | 55 | ## Creating Topics 56 | 57 | ```scala 58 | createTopics( 59 | timeout: Int, 60 | validateOnly: Boolean, 61 | toCreate: Map[String, CreatableTopic], 62 | includeConfigsAndMetadata: Map[String, CreatableTopicResult], 63 | controllerMutationQuota: ControllerMutationQuota, 64 | responseCallback: Map[String, ApiError] => Unit): Unit 65 | ``` 66 | 67 | `createTopics`...FIXME 68 | 69 | --- 70 | 71 | `createTopics` is used when: 72 | 73 | * `DefaultAutoTopicCreationManager` is requested to [createTopicsInZk](DefaultAutoTopicCreationManager.md#createTopicsInZk) 74 | * `KafkaApis` is requested to [handleCreateTopicsRequest](KafkaApis.md#handleCreateTopicsRequest) 75 | -------------------------------------------------------------------------------- /docs/authentication.md: -------------------------------------------------------------------------------- 1 | # Authentication 2 | 3 | **Kafka Authentication** is based on the following: 4 | 5 | * [security.protocol](clients/CommonClientConfigs.md#security.protocol) configuration property 6 | * [ssl.client.auth](KafkaConfig.md#ssl.client.auth) configuration property 7 | 8 | For SSL with client authentication enabled, `TransportLayer#handshake()` performs authentication. For SASL, authentication is performed by `Authenticator#authenticate()`. 9 | 10 | For SSL authentication, the principal will be derived using the rules defined by [ssl.principal.mapping.rules](KafkaConfig.md#ssl.principal.mapping.rules) applied on the distinguished name from the client certificate if one is provided. Otherwise, if client authentication is not required, the principal name will be `ANONYMOUS`. 11 | 12 | For `PLAINTEXT` listeners or when client authentication is not required, the principal will always be `ANONYMOUS`. 13 | -------------------------------------------------------------------------------- /docs/authorization/.pages: -------------------------------------------------------------------------------- 1 | title: Authorization 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/authorization/AclApis.md: -------------------------------------------------------------------------------- 1 | # AclApis 2 | 3 | `AclApis` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/authorization/AclPublisher.md: -------------------------------------------------------------------------------- 1 | # AclPublisher 2 | 3 | `AclPublisher` is a [MetadataPublisher](../metadata/MetadataPublisher.md). 4 | 5 | ## Creating Instance 6 | 7 | `AclPublisher` takes the following to be created: 8 | 9 | * Node ID 10 | * `FaultHandler` 11 | * Node Type 12 | * [Authorizer](Authorizer.md) 13 | 14 | `AclPublisher` is created when: 15 | 16 | * `BrokerServer` is requested to [start up](../kraft/BrokerServer.md#startup) (to create a [BrokerMetadataPublisher](../kraft/BrokerServer.md#brokerMetadataPublisher)) 17 | * `ControllerServer` is requested to [start up](../kraft/ControllerServer.md#startup) 18 | -------------------------------------------------------------------------------- /docs/authorization/ClusterMetadataAuthorizer.md: -------------------------------------------------------------------------------- 1 | # ClusterMetadataAuthorizer 2 | 3 | `ClusterMetadataAuthorizer` is an [extension](#contract) of the [Authorizer](Authorizer.md) abstraction for [authorizers](#implementations) that store state in the `__cluster_metadata` log. 4 | 5 | ## Contract (Subset) 6 | 7 | ### loadSnapshot { #loadSnapshot } 8 | 9 | ```java 10 | void loadSnapshot( 11 | Map acls) 12 | ``` 13 | 14 | See: 15 | 16 | * [StandardAuthorizer](StandardAuthorizer.md#loadSnapshot) 17 | 18 | Used when: 19 | 20 | * `AclPublisher` is requested to [onMetadataUpdate](AclPublisher.md#onMetadataUpdate) 21 | 22 | ## Implementations 23 | 24 | * [StandardAuthorizer](StandardAuthorizer.md) 25 | -------------------------------------------------------------------------------- /docs/authorization/StandardAuthorizer.md: -------------------------------------------------------------------------------- 1 | # StandardAuthorizer 2 | 3 | `StandardAuthorizer` is a [ClusterMetadataAuthorizer](ClusterMetadataAuthorizer.md). 4 | 5 | ## loadSnapshot { #loadSnapshot } 6 | 7 | ??? note "ClusterMetadataAuthorizer" 8 | 9 | ```java 10 | void loadSnapshot( 11 | Map acls) 12 | ``` 13 | 14 | `loadSnapshot` is part of the [ClusterMetadataAuthorizer](ClusterMetadataAuthorizer.md#loadSnapshot) abstraction. 15 | 16 | `loadSnapshot`...FIXME 17 | -------------------------------------------------------------------------------- /docs/authorization/index.md: -------------------------------------------------------------------------------- 1 | # Authorization 2 | 3 | **Kafka Authorization** is based on the following: 4 | 5 | * [kafka-acls.sh](../tools/kafka-acls/index.md) utility for ACL management 6 | * [authorizer.class.name](../KafkaConfig.md#authorizer.class.name) configuration property 7 | * [Authorizer](Authorizer.md) (and [AclAuthorizer](AclAuthorizer.md)) 8 | 9 | ## Resource Types 10 | 11 | Type Name | Resource 12 | -----------|--------- 13 | ANY | Any resource 14 | TOPIC | A topic 15 | GROUP | [Consumer group](../consumer-groups/index.md) 16 | CLUSTER | Kafka cluster as a whole 17 | TRANSACTIONAL_ID | [Transactional ID](../transactions/index.md) 18 | DELEGATION_TOKEN | A delegation token 19 | -------------------------------------------------------------------------------- /docs/broker/.pages: -------------------------------------------------------------------------------- 1 | title: Broker 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/broker/KafkaBroker.md: -------------------------------------------------------------------------------- 1 | # KafkaBroker 2 | 3 | `KafkaBroker` is an [abstraction](#contract) of [Kafka brokers](#implementations). 4 | 5 | Every `KafkaBroker` is a [KafkaMetricsGroup](../metrics/KafkaMetricsGroup.md). 6 | 7 | ## Contract 8 | 9 | ### LogManager 10 | 11 | ```scala 12 | logManager: LogManager 13 | ``` 14 | 15 | [LogManager](../log/LogManager.md) 16 | 17 | Used when: 18 | 19 | * `DynamicBrokerConfig` is requested to `addReconfigurables` 20 | * `DynamicThreadPool` is requested to `reconfigure` 21 | 22 | ### Starting Up 23 | 24 | ```scala 25 | startup(): Unit 26 | ``` 27 | 28 | Used when: 29 | 30 | * `KafkaServerTestHarness` is requested to `restartDeadBrokers` and `createBrokers` 31 | 32 | ### others 33 | 34 | !!! note 35 | There are other methods. 36 | 37 | ## Implementations 38 | 39 | * [BrokerServer](../kraft/BrokerServer.md) (for [KRaft mode](../kraft/index.md)) 40 | * [KafkaServer](KafkaServer.md) 41 | -------------------------------------------------------------------------------- /docs/broker/RaftControllerNodeProvider.md: -------------------------------------------------------------------------------- 1 | # RaftControllerNodeProvider 2 | 3 | `RaftControllerNodeProvider` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/broker/index.md: -------------------------------------------------------------------------------- 1 | # Broker 2 | -------------------------------------------------------------------------------- /docs/building-from-sources.md: -------------------------------------------------------------------------------- 1 | # Building from Sources 2 | 3 | Based on [README.md](https://github.com/apache/kafka/blob/2.8.0/README.md#apache-kafka): 4 | 5 | ```text 6 | KAFKA_VERSION={{ kafka.version }} 7 | SCALA_VERSION={{ scala.short_version }} 8 | ``` 9 | 10 | ```text 11 | $ java -version 12 | openjdk version "11.0.12" 2021-07-20 13 | OpenJDK Runtime Environment Temurin-11.0.12+7 (build 11.0.12+7) 14 | OpenJDK 64-Bit Server VM Temurin-11.0.12+7 (build 11.0.12+7, mixed mode) 15 | ``` 16 | 17 | ```text 18 | ./gradlew clean releaseTarGz install -PskipSigning=true && \ 19 | tar -zxvf core/build/distributions/kafka_$SCALA_VERSION-$KAFKA_VERSION.tgz 20 | ``` 21 | 22 | ```text 23 | cd kafka_$SCALA_VERSION-$KAFKA_VERSION 24 | ``` 25 | 26 | ```text 27 | $ ./bin/kafka-server-start.sh --version | tail -1 28 | 3.0.0 (Commit:8cb0a5e9d3441962) 29 | ``` 30 | -------------------------------------------------------------------------------- /docs/clients/CommonClientConfigs.md: -------------------------------------------------------------------------------- 1 | # CommonClientConfigs 2 | 3 | ## client.id 4 | 5 | ## group.id 6 | 7 | A unique identifier of the consumer group a consumer belongs to. Required if the consumer uses either the group management functionality by using [Consumer.subscribe](consumer/Consumer.md#subscribe) or the Kafka-based offset management strategy. 8 | 9 | Default: (undefined) 10 | 11 | ## retries 12 | 13 | ## retry.backoff.ms 14 | 15 | ## request.timeout.ms 16 | -------------------------------------------------------------------------------- /docs/clients/KafkaClient.md: -------------------------------------------------------------------------------- 1 | # KafkaClient 2 | 3 | `KafkaClient` is an [interface](#contract) to [NetworkClient](NetworkClient.md). 4 | 5 | ## Contract 6 | 7 | ###  inFlightRequestCount 8 | 9 | ```java 10 | int inFlightRequestCount() 11 | int inFlightRequestCount( 12 | String nodeId) 13 | ``` 14 | 15 | Used when: 16 | 17 | * `ConsumerNetworkClient` is requested to [pendingRequestCount](consumer/ConsumerNetworkClient.md#pendingRequestCount) and [poll](consumer/ConsumerNetworkClient.md#poll) 18 | * `Sender` is requested to [run](producer/Sender.md#run) 19 | * `SenderMetrics` is requested for `requests-in-flight` performance metric 20 | 21 | ###  leastLoadedNode 22 | 23 | ```java 24 | Node leastLoadedNode( 25 | long now) 26 | ``` 27 | 28 | Used when: 29 | 30 | * `ConsumerNetworkClient` is requested for the [leastLoadedNode](consumer/ConsumerNetworkClient.md#leastLoadedNode) 31 | * `DefaultMetadataUpdater` is requested to `maybeUpdate` 32 | * `KafkaAdminClient` is used 33 | * `Sender` is requested for the [maybeSendAndPollTransactionalRequest](producer/Sender.md#maybeSendAndPollTransactionalRequest) 34 | 35 | ###  newClientRequest 36 | 37 | ```java 38 | ClientRequest newClientRequest( 39 | String nodeId, 40 | AbstractRequest.Builder requestBuilder, 41 | long createdTimeMs, 42 | boolean expectResponse) 43 | ClientRequest newClientRequest( 44 | String nodeId, 45 | AbstractRequest.Builder requestBuilder, 46 | long createdTimeMs, 47 | boolean expectResponse, 48 | int requestTimeoutMs, 49 | RequestCompletionHandler callback) 50 | ``` 51 | 52 | Used when: 53 | 54 | * `AdminClientRunnable` is requested to `sendEligibleCalls` 55 | * `ConsumerNetworkClient` is requested to [send](consumer/ConsumerNetworkClient.md#send) 56 | * `NetworkClient` is requested to [newClientRequest](NetworkClient.md#newClientRequest), [sendInternalMetadataRequest](NetworkClient.md#sendInternalMetadataRequest) and [handleInitiateApiVersionRequests](NetworkClient.md#handleInitiateApiVersionRequests) 57 | * `RequestSendThread` is requested to `doWork` 58 | * `Sender` is requested to [run](producer/Sender.md#run) 59 | * `KafkaServer` is requested to `controlledShutdown` 60 | * `ReplicaFetcherBlockingSend` is requested to `sendRequest` 61 | 62 | ###  poll 63 | 64 | ```java 65 | List poll( 66 | long timeout, 67 | long now) 68 | ``` 69 | 70 | Used when: 71 | 72 | * FIXME 73 | 74 | ###  pollDelayMs 75 | 76 | ```java 77 | long pollDelayMs( 78 | Node node, 79 | long now) 80 | ``` 81 | 82 | Used when: 83 | 84 | * FIXME 85 | 86 | ###  Is Node Ready and Connected 87 | 88 | ```java 89 | boolean ready( 90 | Node node, 91 | long now); 92 | ``` 93 | 94 | Used when: 95 | 96 | * `AdminClientRunnable` is requested to [sendEligibleCalls](admin/AdminClientRunnable.md#sendEligibleCalls) 97 | * `ConsumerNetworkClient` is requested to [tryConnect](consumer/ConsumerNetworkClient.md#tryConnect) and [trySend](consumer/ConsumerNetworkClient.md#trySend) 98 | * `InterBrokerSendThread` is requested to [sendRequests](../InterBrokerSendThread.md#sendRequests) 99 | * `NetworkClientUtils` is requested to [awaitReady](NetworkClientUtils.md#awaitReady) 100 | * `Sender` is requested to [sendProducerData](producer/Sender.md#sendProducerData) 101 | 102 | ###  send 103 | 104 | ```java 105 | void send( 106 | ClientRequest request, 107 | long now) 108 | ``` 109 | 110 | Used when: 111 | 112 | * FIXME 113 | 114 | ###  wakeup 115 | 116 | ```java 117 | void wakeup() 118 | ``` 119 | 120 | Used when: 121 | 122 | * FIXME 123 | 124 | ## Implementations 125 | 126 | * [NetworkClient](NetworkClient.md) 127 | 128 | ## Closeable 129 | 130 | `KafkaClient` is a `Closeable` ([Java]({{ java.api }}/java/io/Closeable.html)). 131 | -------------------------------------------------------------------------------- /docs/clients/Metadata.md: -------------------------------------------------------------------------------- 1 | # Metadata 2 | 3 | ## update 4 | 5 | ```java 6 | void update( 7 | int requestVersion, 8 | MetadataResponse response, 9 | boolean isPartialUpdate, 10 | long nowMs) 11 | ``` 12 | 13 | `update`...FIXME 14 | 15 | `update` is used when: 16 | 17 | * `ProducerMetadata` is requested to `update` 18 | * `Metadata` is requested to [updateWithCurrentRequestVersion](#updateWithCurrentRequestVersion) 19 | * `DefaultMetadataUpdater` is requested to `handleSuccessfulResponse` 20 | -------------------------------------------------------------------------------- /docs/clients/MetadataUpdater.md: -------------------------------------------------------------------------------- 1 | # MetadataUpdater 2 | 3 | `MetadataUpdater` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/clients/NetworkClient.md: -------------------------------------------------------------------------------- 1 | # NetworkClient 2 | 3 | `NetworkClient` is a [KafkaClient](KafkaClient.md). 4 | 5 | ## leastLoadedNode 6 | 7 | ```scala 8 | Node leastLoadedNode( 9 | long now) 10 | ``` 11 | 12 | `leastLoadedNode` requests the [MetadataUpdater](#metadataUpdater) for the [nodes](MetadataUpdater.md#fetchNodes) (in a non-blocking fashion). 13 | 14 | `leastLoadedNode` generates a random number to offset the first node to start checking node candidates from. 15 | 16 | `leastLoadedNode` finds three nodes: 17 | 18 | 1. `foundReady` that is ready (perhaps with some [in-flight requests](#inFlightRequests)) 19 | 1. `foundConnecting` with a connection already being established (using the `ClusterConnectionStates` registry) 20 | 1. `foundCanConnect` that [can be connected](#canConnect) 21 | 22 | When a node is found that is ready and has no [in-flight requests](#inFlightRequests), `leastLoadedNode` prints out the following TRACE message to the logs and returns the node immediately: 23 | 24 | ```text 25 | Found least loaded node [node] connected with no in-flight requests 26 | ``` 27 | 28 | When a node candidate does not meet any of the above requirements, `leastLoadedNode` prints out the following TRACE message to the logs: 29 | 30 | ```text 31 | Removing node [node] from least loaded node selection since it is neither ready for sending nor connecting 32 | ``` 33 | 34 | `leastLoadedNode` prefers the `foundReady` node over the `foundConnecting` with the `foundCanConnect` as the last resort. 35 | 36 | When no node could be found, `leastLoadedNode` prints out the following TRACE message to the logs (and returns `null`): 37 | 38 | ```text 39 | Least loaded node selection failed to find an available node 40 | ``` 41 | 42 | `leastLoadedNode` is part of the [KafkaClient](KafkaClient.md#leastLoadedNode) abstraction. 43 | -------------------------------------------------------------------------------- /docs/clients/NetworkClientUtils.md: -------------------------------------------------------------------------------- 1 | # NetworkClientUtils 2 | -------------------------------------------------------------------------------- /docs/clients/admin/Admin.md: -------------------------------------------------------------------------------- 1 | # Admin 2 | 3 | ## create 4 | 5 | ```java 6 | Admin create( 7 | Map conf) 8 | Admin create( 9 | Properties props) 10 | ``` 11 | 12 | `create` creates a [KafkaAdminClient](KafkaAdminClient.md#createInternal). 13 | 14 | --- 15 | 16 | `create` is used when: 17 | 18 | * FIXME 19 | -------------------------------------------------------------------------------- /docs/clients/admin/AdminClientRunnable.md: -------------------------------------------------------------------------------- 1 | # AdminClientRunnable 2 | -------------------------------------------------------------------------------- /docs/clients/admin/KafkaAdminClient.md: -------------------------------------------------------------------------------- 1 | # KafkaAdminClient 2 | 3 | `KafkaAdminClient` is a `AdminClient` that is used in [Kafka administration utilities](../../tools/index.md). 4 | 5 | ## Creating Instance 6 | 7 | `KafkaAdminClient` takes the following to be created: 8 | 9 | * `AdminClientConfig` 10 | * Client ID 11 | * Time 12 | * `AdminMetadataManager` 13 | * [Metrics](../../metrics/Metrics.md) 14 | * [KafkaClient](../KafkaClient.md) 15 | * `TimeoutProcessorFactory` 16 | * `LogContext` 17 | 18 | `KafkaAdminClient` is created using [createInternal](#createInternal). 19 | 20 | ## createInternal 21 | 22 | ```java 23 | KafkaAdminClient createInternal( 24 | AdminClientConfig config, 25 | AdminMetadataManager metadataManager, 26 | KafkaClient client, 27 | Time time) 28 | KafkaAdminClient createInternal( 29 | AdminClientConfig config, 30 | TimeoutProcessorFactory timeoutProcessorFactory) // (1)! 31 | KafkaAdminClient createInternal( 32 | AdminClientConfig config, 33 | TimeoutProcessorFactory timeoutProcessorFactory, 34 | HostResolver hostResolver) 35 | ``` 36 | 37 | 1. Uses an undefined `HostResolver` 38 | 39 | `createInternal`...FIXME 40 | 41 | --- 42 | 43 | `createInternal` is used when: 44 | 45 | * `Admin` is requested to [create](Admin.md#create) 46 | 47 | ## Logging 48 | 49 | Enable `ALL` logging level for `org.apache.kafka.clients.admin.KafkaAdminClient` logger to see what happens inside. 50 | 51 | Add the following line to `config/log4j.properties`: 52 | 53 | ```text 54 | log4j.logger.org.apache.kafka.clients.admin.KafkaAdminClient=ALL 55 | ``` 56 | 57 | Refer to [Logging](../../logging.md). 58 | 59 | ## Review Me 60 | 61 | ## Triggerring Preferred Replica Leader Election 62 | 63 | ```java 64 | ElectPreferredLeadersResult electPreferredLeaders( 65 | Collection partitions, 66 | ElectPreferredLeadersOptions options) 67 | ``` 68 | 69 | NOTE: `electPreferredLeaders` is part of the <> to trigger <>. 70 | 71 | `electPreferredLeaders` creates a *electPreferredLeaders* call that simply uses <> to <> and, when a response comes in, requests the `ElectPreferredLeadersRequest` to <>. 72 | 73 | In the end, `electPreferredLeaders` requests the <> to <> the `electPreferredLeaders` call. 74 | -------------------------------------------------------------------------------- /docs/clients/admin/index.md: -------------------------------------------------------------------------------- 1 | # Kafka Admin 2 | -------------------------------------------------------------------------------- /docs/clients/consumer/AbstractPartitionAssignor.md: -------------------------------------------------------------------------------- 1 | # AbstractPartitionAssignor 2 | 3 | `AbstractPartitionAssignor` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/clients/consumer/Consumer.md: -------------------------------------------------------------------------------- 1 | # Consumer 2 | 3 | `Consumer` is an [interface](#contract) to [KafkaConsumer](KafkaConsumer.md) for Kafka developers to use to consume records (with `K` keys and `V` values) from a Kafka cluster. 4 | 5 | ## Contract (Subset) 6 | 7 | ###  enforceRebalance 8 | 9 | ```java 10 | void enforceRebalance() 11 | ``` 12 | 13 | ###  groupMetadata 14 | 15 | ```java 16 | ConsumerGroupMetadata groupMetadata() 17 | ``` 18 | 19 | ###  Subscribing to Topics 20 | 21 | ```java 22 | void subscribe( 23 | Collection topics) 24 | void subscribe( 25 | Collection topics, 26 | ConsumerRebalanceListener callback) 27 | void subscribe( 28 | Pattern pattern) 29 | void subscribe( 30 | Pattern pattern, 31 | ConsumerRebalanceListener callback) 32 | ``` 33 | 34 | ###  Waking Up 35 | 36 | ```java 37 | void wakeup() 38 | ``` 39 | -------------------------------------------------------------------------------- /docs/clients/consumer/ConsumerCoordinator.md: -------------------------------------------------------------------------------- 1 | # ConsumerCoordinator 2 | 3 | `ConsumerCoordinator` is a [consumer group coordination manager](AbstractCoordinator.md). 4 | 5 | ## Creating Instance 6 | 7 | `ConsumerCoordinator` takes the following to be created: 8 | 9 | * `GroupRebalanceConfig` 10 | * `LogContext` 11 | * [ConsumerNetworkClient](ConsumerNetworkClient.md) 12 | * [ConsumerPartitionAssignor](ConsumerPartitionAssignor.md)s 13 | * [ConsumerMetadata](ConsumerMetadata.md) 14 | * [SubscriptionState](SubscriptionState.md) 15 | * `Metrics` 16 | * Metrics Group Prefix 17 | * `Time` 18 | * [autoCommitEnabled](#autoCommitEnabled) 19 | * [auto.commit.interval.ms](ConsumerConfig.md#AUTO_COMMIT_INTERVAL_MS_CONFIG) 20 | * `ConsumerInterceptors` 21 | * [internal.throw.on.fetch.stable.offset.unsupported](ConsumerConfig.md#THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED) 22 | 23 | While being created, `ConsumerCoordinator` requests the [ConsumerMetadata](#metadata) for an [update of the current cluster metadata](../Metadata.md#requestUpdate). 24 | 25 | `ConsumerCoordinator` is created when: 26 | 27 | * `KafkaConsumer` is [created](KafkaConsumer.md#coordinator) (and [group.id](KafkaConsumer.md#groupId) configuration property is specified) 28 | 29 | ## autoCommitEnabled 30 | 31 | `ConsumerCoordinator` is given `autoCommitEnabled` flag when [created](#creating-instance) with the value based on [group.id and enable.auto.commit configuration properties](ConsumerConfig.md#maybeOverrideEnableAutoCommit). 32 | 33 | ## metadata 34 | 35 | ```java 36 | JoinGroupRequestData.JoinGroupRequestProtocolCollection metadata() 37 | ``` 38 | 39 | `metadata`...FIXME 40 | 41 | `metadata` is part of the [AbstractCoordinator](AbstractCoordinator.md#metadata) abstraction. 42 | -------------------------------------------------------------------------------- /docs/clients/consumer/ConsumerMetadata.md: -------------------------------------------------------------------------------- 1 | # ConsumerMetadata 2 | 3 | `ConsumerMetadata` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/clients/consumer/ConsumerNetworkClient.md: -------------------------------------------------------------------------------- 1 | # ConsumerNetworkClient -------------------------------------------------------------------------------- /docs/clients/consumer/ConsumerPartitionAssignor.md: -------------------------------------------------------------------------------- 1 | # ConsumerPartitionAssignor 2 | 3 | `ConsumerPartitionAssignor` is an [abstraction](#contract) of [partition assignors](#implementations). 4 | 5 | ## Contract 6 | 7 | ###  assign 8 | 9 | ```java 10 | GroupAssignment assign( 11 | Cluster metadata, 12 | GroupSubscription groupSubscription) 13 | ``` 14 | 15 | Used when: 16 | 17 | * FIXME 18 | 19 | ###  name 20 | 21 | ```java 22 | String name() 23 | ``` 24 | 25 | Used when: 26 | 27 | * FIXME 28 | 29 | ## Implementations 30 | 31 | * [AbstractPartitionAssignor](AbstractPartitionAssignor.md) 32 | * `StreamsPartitionAssignor` ([Kafka Streams]({{ book.kafka_streams }}/StreamsPartitionAssignor)) 33 | 34 | ##  onAssignment 35 | 36 | ```java 37 | void onAssignment( 38 | Assignment assignment, 39 | ConsumerGroupMetadata metadata) 40 | ``` 41 | 42 | `onAssignment`...FIXME 43 | 44 | `onAssignment` is used when: 45 | 46 | * FIXME 47 | 48 | ##  supportedProtocols 49 | 50 | ```java 51 | List supportedProtocols() 52 | ``` 53 | 54 | Default: `RebalanceProtocol.EAGER` 55 | 56 | `supportedProtocols` is used when: 57 | 58 | * FIXME 59 | 60 | ##  subscriptionUserData 61 | 62 | ```java 63 | ByteBuffer subscriptionUserData( 64 | Set topics) 65 | ``` 66 | 67 | `subscriptionUserData` is `null` by default. 68 | 69 | `subscriptionUserData` is used when: 70 | 71 | * `ConsumerCoordinator` is requested for [metadata](ConsumerCoordinator.md#metadata) 72 | -------------------------------------------------------------------------------- /docs/clients/consumer/SubscriptionState.md: -------------------------------------------------------------------------------- 1 | # SubscriptionState 2 | 3 | ## Preferred Read Replica 4 | 5 | ### preferredReadReplica 6 | 7 | ```java 8 | Optional preferredReadReplica( 9 | TopicPartition tp, 10 | long timeMs) 11 | ``` 12 | 13 | `preferredReadReplica` [looks up the state](#assignedStateOrNull) of the given `TopicPartition` and, if found, requests it for the [preferredReadReplica](TopicPartitionState.md#preferredReadReplica). Otherwise, `preferredReadReplica` returns an undefined preferred read replica. 14 | 15 | `preferredReadReplica` is used when: 16 | 17 | * `Fetcher` is requested to [selectReadReplica](Fetcher.md#selectReadReplica) 18 | 19 | ### updatePreferredReadReplica 20 | 21 | ```java 22 | void updatePreferredReadReplica( 23 | TopicPartition tp, 24 | int preferredReadReplicaId, 25 | LongSupplier timeMs) 26 | ``` 27 | 28 | `updatePreferredReadReplica` [looks up the state](#assignedState) of the given `TopicPartition` and requests it to [updatePreferredReadReplica](TopicPartitionState.md#updatePreferredReadReplica). 29 | 30 | `updatePreferredReadReplica` is used when: 31 | 32 | * `Fetcher` is requested to [initializeCompletedFetch](Fetcher.md#initializeCompletedFetch) 33 | 34 | ### clearPreferredReadReplica 35 | 36 | ```java 37 | Optional clearPreferredReadReplica( 38 | TopicPartition tp) 39 | ``` 40 | 41 | `clearPreferredReadReplica` [looks up the state](#assignedState) of the given `TopicPartition` and requests it to [clearPreferredReadReplica](TopicPartitionState.md#clearPreferredReadReplica). 42 | 43 | `clearPreferredReadReplica` is used when: 44 | 45 | * `Fetcher` is requested to [selectReadReplica](Fetcher.md#selectReadReplica) and [initializeCompletedFetch](Fetcher.md#initializeCompletedFetch) 46 | -------------------------------------------------------------------------------- /docs/clients/consumer/TopicPartitionState.md: -------------------------------------------------------------------------------- 1 | # TopicPartitionState 2 | 3 | # preferredReadReplica 4 | 5 | ```java 6 | Integer preferredReadReplica 7 | ``` 8 | 9 | `TopicPartitionState` manages the preferred read replica (of a `TopicPartition`) for a specified amount of time (until expires or is cleared out). 10 | 11 | `preferredReadReplica` is used when: 12 | 13 | * `SubscriptionState` is requested for the [preferredReadReplica](SubscriptionState.md#preferredReadReplica), [updatePreferredReadReplica](SubscriptionState.md#updatePreferredReadReplica) and [clearPreferredReadReplica](SubscriptionState.md#clearPreferredReadReplica) 14 | -------------------------------------------------------------------------------- /docs/clients/consumer/index.md: -------------------------------------------------------------------------------- 1 | # Kafka Consumers 2 | 3 | [KafkaConsumer](KafkaConsumer.md) uses [Fetcher](Fetcher.md) to fetch records from a Kafka cluster. One could say that `KafkaConsumer` is a developer-oriented interface to `Fetcher`. 4 | 5 | `KafkaConsumer` is assigned a `IsolationLevel` based on [isolation.level](ConsumerConfig.md#ISOLATION_LEVEL_CONFIG) configuration property. 6 | -------------------------------------------------------------------------------- /docs/clients/consumer/preferred-read-replica.md: -------------------------------------------------------------------------------- 1 | # Preferred Read Replica 2 | 3 | **Preferred Read Replica** is the broker ID of one of the in-sync replicas of a partition for [Kafka Consumer](KafkaConsumer.md) to read records from. 4 | -------------------------------------------------------------------------------- /docs/clients/index.md: -------------------------------------------------------------------------------- 1 | # Kafka Clients 2 | -------------------------------------------------------------------------------- /docs/clients/producer/BufferPool.md: -------------------------------------------------------------------------------- 1 | # BufferPool 2 | 3 | `BufferPool` is [created](#creating-instance) alongsize [KafkaProducer](KafkaProducer.md) for the [RecordAccumulator](KafkaProducer.md#accumulator). 4 | 5 | ## Creating Instance 6 | 7 | `BufferPool` takes the following to be created: 8 | 9 | * [Total Memory Size](#memory) 10 | * [Batch Size](#poolableSize) 11 | * [Metrics](../../metrics/Metrics.md) 12 | * `Time` 13 | * Metric Group Name 14 | 15 | `BufferPool` is created when: 16 | 17 | * `KafkaProducer` is [created](KafkaProducer.md) (to create a [RecordAccumulator](KafkaProducer.md#accumulator)) 18 | 19 | ### Total Memory Size { #memory } 20 | 21 | `BufferPool` is given the total memory size when [created](#creating-instance). 22 | 23 | The size is by default the value of [buffer.memory](ProducerConfig.md#buffer.memory) configuration property. 24 | 25 | ### Batch Size { #poolableSize } 26 | 27 | `BufferPool` is given the batch size when [created](#creating-instance). 28 | 29 | The size is by default the value of [batch.size](ProducerConfig.md#batch.size) configuration property. 30 | 31 | ## Metrics 32 | 33 | `BufferPool` registers the metrics under the [producer-metrics](KafkaProducer.md#PRODUCER_METRIC_GROUP_NAME) group name. 34 | 35 | ### buffer-exhausted-rate { #buffer-exhausted-rate } 36 | 37 | The average per-second number of record sends that are dropped due to buffer exhaustion 38 | 39 | ### buffer-exhausted-records { #buffer-exhausted-records } 40 | 41 | ### buffer-exhausted-total { #buffer-exhausted-total } 42 | 43 | The total number of record sends that are dropped due to buffer exhaustion 44 | 45 | ### bufferpool-wait-ratio { #bufferpool-wait-ratio } 46 | 47 | The fraction of time an appender waits for space allocation 48 | 49 | ### bufferpool-wait-time-ns-total { #bufferpool-wait-time-ns-total } 50 | 51 | The total time (in ns) an appender waits for space allocation 52 | -------------------------------------------------------------------------------- /docs/clients/producer/Callback.md: -------------------------------------------------------------------------------- 1 | # Callback 2 | 3 | `Callback` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/clients/producer/DefaultPartitioner.md: -------------------------------------------------------------------------------- 1 | # DefaultPartitioner 2 | 3 | `DefaultPartitioner` is a [Partitioner](Partitioner.md). 4 | 5 | ## Demo 6 | 7 | ```text 8 | import org.apache.kafka.clients.producer.internals.DefaultPartitioner 9 | val partitioner = new DefaultPartitioner 10 | 11 | val keyBytes = "hello".getBytes 12 | val numPartitions = 3 13 | 14 | val p = partitioner.partition(null, null, keyBytes, null, null, null, numPartitions) 15 | 16 | println(p) 17 | ``` 18 | 19 | The following snippet should generate the same partition value (since it is exactly how `DefaultPartitioner` does it). 20 | 21 | ```text 22 | import org.apache.kafka.common.utils.Utils 23 | 24 | val keyBytes = "hello".getBytes 25 | val numPartitions = 3 26 | 27 | val p = Utils.toPositive(Utils.murmur2(keyBytes)) % numPartitions 28 | 29 | println(p) 30 | ``` 31 | -------------------------------------------------------------------------------- /docs/clients/producer/Partitioner.md: -------------------------------------------------------------------------------- 1 | # Partitioner 2 | 3 | `Partitioner` is an [abstraction](#contract) of [partitioners](#implementations) for a [KafkaProducer](KafkaProducer.md#partitioner) to determine the [partition](#partition) of records (to be [sent out](KafkaProducer.md#send)). 4 | 5 | ## Configurable 6 | 7 | `Partitioner` is a [Configurable](../../Configurable.md). 8 | 9 | ## Closeable 10 | 11 | `Partitioner` is a `Closeable` ([Java]({{ java.api }}/java/io/Closeable.html)). 12 | 13 | ## Contract 14 | 15 | ###  onNewBatch 16 | 17 | ```java 18 | void onNewBatch( 19 | String topic, 20 | Cluster cluster, 21 | int prevPartition) 22 | ``` 23 | 24 | Used when: 25 | 26 | * `KafkaProducer` is requested to [send a record](KafkaProducer.md#send) (and [doSend](KafkaProducer.md#doSend)) 27 | 28 | ###  Computing Partition 29 | 30 | ```java 31 | int partition( 32 | String topic, 33 | Object key, 34 | byte[] keyBytes, 35 | Object value, 36 | byte[] valueBytes, 37 | Cluster cluster) 38 | ``` 39 | 40 | Used when: 41 | 42 | * `KafkaProducer` is requested to [send a record](KafkaProducer.md#send) (and determines the [partition](KafkaProducer.md#partition)) 43 | 44 | ## Implementations 45 | 46 | * [DefaultPartitioner](DefaultPartitioner.md) 47 | * UniformStickyPartitioner 48 | * RoundRobinPartitioner 49 | -------------------------------------------------------------------------------- /docs/clients/producer/Producer.md: -------------------------------------------------------------------------------- 1 | # Producer 2 | 3 | `Producer` is an [interface](#contract) to [KafkaProducer](KafkaProducer.md) for Kafka developers to use to send records (with `K` keys and `V` values) to a Kafka cluster. 4 | 5 | ## Contract (Subset) 6 | 7 | ###  abortTransaction 8 | 9 | ```java 10 | void abortTransaction() 11 | ``` 12 | 13 | ###  beginTransaction 14 | 15 | ```java 16 | void beginTransaction() 17 | ``` 18 | 19 | ###  commitTransaction 20 | 21 | ```java 22 | void commitTransaction() 23 | ``` 24 | 25 | ###  initTransactions 26 | 27 | ```java 28 | void initTransactions() 29 | ``` 30 | 31 | ###  sendOffsetsToTransaction 32 | 33 | ```java 34 | void sendOffsetsToTransaction( 35 | Map offsets, 36 | ConsumerGroupMetadata groupMetadata) 37 | ``` 38 | 39 | Used when the producer is also a [Consumer](../consumer/index.md) for a consume-transform-produce pattern 40 | -------------------------------------------------------------------------------- /docs/clients/producer/ProducerBatch.md: -------------------------------------------------------------------------------- 1 | # ProducerBatch 2 | 3 | ## Creating Instance 4 | 5 | `ProducerBatch` takes the following to be created: 6 | 7 | * `TopicPartition` 8 | * MemoryRecordsBuilder 9 | * createdMs 10 | * `isSplitBatch` flag (default: `false`) 11 | 12 | `ProducerBatch` is created when: 13 | 14 | * `ProducerBatch` is requested to [createBatchOffAccumulatorForRecord](#createBatchOffAccumulatorForRecord) 15 | * `RecordAccumulator` is requested to [append a record](RecordAccumulator.md#append) 16 | 17 | ## tryAppend 18 | 19 | ```java 20 | FutureRecordMetadata tryAppend( 21 | long timestamp, 22 | byte[] key, 23 | byte[] value, 24 | Header[] headers, 25 | Callback callback, 26 | long now) 27 | ``` 28 | 29 | `tryAppend`...FIXME 30 | 31 | `tryAppend` is used when: 32 | 33 | * `RecordAccumulator` is requested to [append a record](RecordAccumulator.md#append) 34 | -------------------------------------------------------------------------------- /docs/clients/producer/ProducerInterceptors.md: -------------------------------------------------------------------------------- 1 | # ProducerInterceptors 2 | 3 | `ProducerInterceptors` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/clients/producer/index.md: -------------------------------------------------------------------------------- 1 | # Kafka Producers 2 | 3 | [KafkaProducer](KafkaProducer.md) uses [RecordAccumulator](KafkaProducer.md#accumulator) with records to be sent out. 4 | 5 | `KafkaProducer` groups together records that arrive in-between request transmissions into a single batched request. 6 | Normally this occurs only under load when records arrive faster than they can be sent out. 7 | However in some circumstances the client may want to reduce the number of requests even under moderate load using [linger.ms](ProducerConfig.md#linger.ms) configuration property. 8 | 9 | `KafkaProducer` uses [Sender](Sender.md) to send records to a Kafka cluster. 10 | 11 | `KafkaProducer` can be [transactional or idempotent](KafkaProducer.md#configureTransactionState) (and associated with a [TransactionManager](TransactionManager.md)). 12 | 13 | ## Demo 14 | 15 | ```text 16 | // Necessary imports 17 | import org.apache.kafka.clients.producer.KafkaProducer 18 | import org.apache.kafka.clients.producer.ProducerConfig 19 | import org.apache.kafka.common.serialization.StringSerializer 20 | 21 | // Creating a KafkaProducer 22 | import java.util.Properties 23 | val props = new Properties() 24 | props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[StringSerializer].getName) 25 | props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[StringSerializer].getName) 26 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, ":9092") 27 | val producer = new KafkaProducer[String, String](props) 28 | 29 | // Creating a record to be sent 30 | import org.apache.kafka.clients.producer.ProducerRecord 31 | val r = new ProducerRecord[String, String]("0", "this is a message") 32 | 33 | // Sending the record (with no Callback) 34 | import java.util.concurrent.Future 35 | import org.apache.kafka.clients.producer.RecordMetadata 36 | val metadataF: Future[RecordMetadata] = producer.send(r) 37 | ``` 38 | -------------------------------------------------------------------------------- /docs/consumer-groups/GroupMetadataManager.md: -------------------------------------------------------------------------------- 1 | # GroupMetadataManager 2 | 3 | ![Creating GroupMetadataManager](../images/GroupMetadataManager-creating-instance.png) 4 | 5 | ## Creating Instance 6 | 7 | `GroupMetadataManager` takes the following to be created: 8 | 9 | * [broker.id](../KafkaConfig.md#brokerId) 10 | * [ApiVersion](../KafkaConfig.md#interBrokerProtocolVersion) 11 | * [OffsetConfig](OffsetConfig.md) 12 | * [ReplicaManager](../ReplicaManager.md) 13 | * `Time` 14 | * [Metrics](../metrics/Metrics.md) 15 | 16 | `GroupMetadataManager` is created alongside a [GroupCoordinator](GroupCoordinator.md#groupManager). 17 | 18 | ## Performance Metrics 19 | 20 | `GroupMetadataManager` is a [KafkaMetricsGroup](../metrics/KafkaMetricsGroup.md) and registers the following performance metrics in **kafka.coordinator.group:type=GroupMetadataManager** group. 21 | 22 | Metric Name | Description 23 | ------------|------------ 24 | NumGroups | 25 | NumGroupsCompletingRebalance | 26 | NumGroupsDead | 27 | NumGroupsEmpty | 28 | NumGroupsPreparingRebalance | 29 | NumGroupsStable | 30 | NumOffsets | 31 | 32 | ![GroupMetadataManager in jconsole](../images/GroupMetadataManager-jconsole.png) 33 | 34 | ## Logging 35 | 36 | Enable `ALL` logging level for `kafka.coordinator.group.GroupMetadataManager` logger to see what happens inside. 37 | 38 | Add the following line to `config/log4j.properties`: 39 | 40 | ```text 41 | log4j.logger.kafka.coordinator.group.GroupMetadataManager=ALL 42 | ``` 43 | 44 | Refer to [Logging](../logging.md). 45 | -------------------------------------------------------------------------------- /docs/consumer-groups/OffsetConfig.md: -------------------------------------------------------------------------------- 1 | # OffsetConfig 2 | 3 | `OffsetConfig` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/consumer-groups/index.md: -------------------------------------------------------------------------------- 1 | # Consumer Groups 2 | 3 | [GroupCoordinator](GroupCoordinator.md) is the main actor here and is started alongside [KafkaServer](../broker/KafkaServer.md#groupCoordinator). 4 | 5 | ## __consumer_offsets 6 | 7 | `__consumer_offsets` is the offset commit topic (with [offsets.topic.num.partitions](../KafkaConfig.md#offsets.topic.num.partitions)). 8 | -------------------------------------------------------------------------------- /docs/controller/AbstractControlRequest.md: -------------------------------------------------------------------------------- 1 | # AbstractControlRequest 2 | 3 | `AbstractControlRequest` is an [extension](#contract) of the [AbstractRequest](../AbstractRequest.md) abstraction for [controller requests](#implementations) that [KafkaController](KafkaController.md) uses to propage broker and partition state changes to brokers. 4 | 5 | ## Contract 6 | 7 | ### brokerEpoch 8 | 9 | ```java 10 | long brokerEpoch() 11 | ``` 12 | 13 | Used when: 14 | 15 | * `KafkaApis` is requested to [handleLeaderAndIsrRequest](../KafkaApis.md#handleLeaderAndIsrRequest), [handleStopReplicaRequest](../KafkaApis.md#handleStopReplicaRequest), [handleUpdateMetadataRequest](../KafkaApis.md#handleUpdateMetadataRequest) 16 | 17 | ### controllerEpoch 18 | 19 | ```java 20 | int controllerEpoch() 21 | ``` 22 | 23 | Used when: 24 | 25 | * `KafkaApis` is requested to [handleStopReplicaRequest](../KafkaApis.md#handleStopReplicaRequest) 26 | * `ReplicaManager` is requested to [maybeUpdateMetadataCache](../ReplicaManager.md#maybeUpdateMetadataCache), [becomeLeaderOrFollower](../ReplicaManager.md#becomeLeaderOrFollower) 27 | * `ZkMetadataCache` is requested to `updateMetadata` 28 | 29 | ### controllerId 30 | 31 | ```java 32 | int controllerId() 33 | ``` 34 | 35 | Used when: 36 | 37 | * `KafkaApis` is requested to [handleStopReplicaRequest](../KafkaApis.md#handleStopReplicaRequest) 38 | * `ReplicaManager` is requested to [maybeUpdateMetadataCache](../ReplicaManager.md#maybeUpdateMetadataCache), [becomeLeaderOrFollower](../ReplicaManager.md#becomeLeaderOrFollower) 39 | * `ZkMetadataCache` is requested to `updateMetadata` 40 | 41 | ## Implementations 42 | 43 | * [LeaderAndIsrRequest](LeaderAndIsrRequest.md) 44 | * `StopReplicaRequest` 45 | * `UpdateMetadataRequest` 46 | 47 | ## Creating Instance 48 | 49 | `AbstractControlRequest` takes the following to be created: 50 | 51 | * `ApiKeys` 52 | * Version 53 | 54 | !!! note "Abstract Class" 55 | `AbstractControlRequest` is an abstract class and cannot be created directly. It is created indirectly for the [concrete AbstractControlRequests](#implementations). 56 | -------------------------------------------------------------------------------- /docs/controller/AutoPreferredReplicaLeaderElection.md: -------------------------------------------------------------------------------- 1 | # AutoPreferredReplicaLeaderElection 2 | 3 | ## State 4 | 5 | `AutoPreferredReplicaLeaderElection` is a [ControllerEvent](ControllerEvent.md) that [transition](ControllerEvent.md#state) the [KafkaController](KafkaController.md) to `AutoLeaderBalance` state. 6 | 7 | `AutoPreferredReplicaLeaderElection` is [enqueued](ControllerEventManager.md#put) (to the [ControllerEventManager](KafkaController.md#eventManager)) exclusively from the <>. 8 | 9 | ## Process 10 | 11 | When [processed](ControllerEvent.md#process) on a [controller broker](KafkaController.md#isActive), `AutoPreferredReplicaLeaderElection` event [checkAndTriggerAutoLeaderRebalance](KafkaController.md#checkAndTriggerAutoLeaderRebalance) and in the end [scheduleAutoLeaderRebalanceTask](KafkaController.md#scheduleAutoLeaderRebalanceTask) with the delay based on [leader.imbalance.check.interval.seconds](../KafkaConfig.md#leader.imbalance.check.interval.seconds) configuration property. 12 | 13 | !!! note 14 | `AutoPreferredReplicaLeaderElection` event is ignored (skipped) when processed on any broker but [controller broker](KafkaController.md#isActive). 15 | -------------------------------------------------------------------------------- /docs/controller/ControllerBrokerRequestBatch.md: -------------------------------------------------------------------------------- 1 | # ControllerBrokerRequestBatch 2 | 3 | `ControllerBrokerRequestBatch` is an [AbstractControllerBrokerRequestBatch](AbstractControllerBrokerRequestBatch.md). 4 | 5 | Every time `ControllerBrokerRequestBatch` is used it is first requested to [prepare a new batch](AbstractControllerBrokerRequestBatch.md#newBatch) (of controller requests) followed by [sending them out to brokers](AbstractControllerBrokerRequestBatch.md#sendRequestsToBrokers). 6 | 7 | ## Creating Instance 8 | 9 | `ControllerBrokerRequestBatch` takes the following to be created: 10 | 11 | * [KafkaConfig](../KafkaConfig.md) 12 | * `ControllerChannelManager` 13 | * [ControllerEventManager](ControllerEventManager.md) 14 | * [ControllerContext](ControllerContext.md) 15 | * `StateChangeLogger` 16 | 17 | `ControllerBrokerRequestBatch` is created along with a [KafkaController](KafkaController.md), separately for the following: 18 | 19 | * [ControllerBrokerRequestBatch](KafkaController.md#brokerRequestBatch) 20 | * [ZkReplicaStateMachine](KafkaController.md#replicaStateMachine) 21 | * [ZkPartitionStateMachine](KafkaController.md#partitionStateMachine) 22 | 23 | ## Sending ControllerEvent 24 | 25 | ```scala 26 | sendEvent( 27 | event: ControllerEvent): Unit 28 | ``` 29 | 30 | `sendEvent` is part of the [AbstractControllerBrokerRequestBatch](AbstractControllerBrokerRequestBatch.md#sendEvent) abstraction. 31 | 32 | --- 33 | 34 | `sendEvent` requests the [ControllerEventManager](#controllerEventManager) to [enqueue](ControllerEventManager.md#put) the input [ControllerEvent](ControllerEvent.md). 35 | 36 | ## Sending Request (to Broker) 37 | 38 | ```scala 39 | sendRequest( 40 | brokerId: Int, 41 | request: AbstractControlRequest.Builder[_ <: AbstractControlRequest], 42 | callback: AbstractResponse => Unit = null): Unit 43 | ``` 44 | 45 | `sendRequest` is part of the [AbstractControllerBrokerRequestBatch](AbstractControllerBrokerRequestBatch.md#sendRequest) abstraction. 46 | 47 | --- 48 | 49 | `sendRequest` requests the given [ControllerChannelManager](#controllerChannelManager) to [send an controller request out to a broker](ControllerChannelManager.md#sendRequest). 50 | -------------------------------------------------------------------------------- /docs/controller/ControllerEventProcessor.md: -------------------------------------------------------------------------------- 1 | # ControllerEventProcessor 2 | 3 | `ControllerEventProcessor` is an [abstraction](#contract) of [processors](#implementations) that can [process](#process) and [preempt](#preempt) controller events. 4 | 5 | ## Contract 6 | 7 | ### preempt 8 | 9 | ```scala 10 | preempt( 11 | event: ControllerEvent): Unit 12 | ``` 13 | 14 | Preempts a [ControllerEvent](ControllerEvent.md) 15 | 16 | Used when: 17 | 18 | * `QueuedEvent` is requested to [preempt a ControllerEventProcessor](QueuedEvent.md#preempt) 19 | 20 | ### process 21 | 22 | ```scala 23 | process( 24 | event: ControllerEvent): Unit 25 | ``` 26 | 27 | Processes a [ControllerEvent](ControllerEvent.md) 28 | 29 | Used when: 30 | 31 | * `QueuedEvent` is requested to [process a ControllerEventProcessor](QueuedEvent.md#process) 32 | 33 | ## Implementations 34 | 35 | * [KafkaController](KafkaController.md) 36 | -------------------------------------------------------------------------------- /docs/controller/ControllerEventThread.md: -------------------------------------------------------------------------------- 1 | # ControllerEventThread 2 | 3 | `ControllerEventThread` is a `ShutdownableThread` that [ControllerEventManager](ControllerEventManager.md#thread) uses to [process QueuedEvents](#doWork) (asynchronously on a separate thread). 4 | 5 | ![ControllerEventThread is Started Alongside ControllerEventManager](../images/ControllerEventThread-doWork.png) 6 | 7 | ## Review Me 8 | 9 | `ControllerEventThread` is a `ShutdownableThread` that is <> for <> (with the <> being *controller-event-thread*). 10 | 11 | ```text 12 | // jstack [brokerPid] 13 | "controller-event-thread" #42 prio=5 os_prio=31 cpu=387,10ms elapsed=82679,68s tid=0x00007f920e489800 nid=0x14703 waiting on condition [0x000070000fcea000] 14 | java.lang.Thread.State: WAITING (parking) 15 | at jdk.internal.misc.Unsafe.park(java.base@12.0.2/Native Method) 16 | - parking to wait for <0x00000007c07f0298> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject) 17 | at java.util.concurrent.locks.LockSupport.park(java.base@12.0.2/LockSupport.java:194) 18 | at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(java.base@12.0.2/AbstractQueuedSynchronizer.java:2081) 19 | at java.util.concurrent.LinkedBlockingQueue.take(java.base@12.0.2/LinkedBlockingQueue.java:433) 20 | at kafka.controller.ControllerEventManager$ControllerEventThread.doWork(ControllerEventManager.scala:127) 21 | at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:89) 22 | ``` 23 | 24 | `ControllerEventThread` is <> with <>. 25 | 26 | [[creating-instance]][[name]] 27 | `ControllerEventThread` takes the name of the thread to be created. 28 | 29 | [[logIdent]] 30 | `ControllerEventThread` uses *[ControllerEventThread controllerId=[controllerId]]* as the logging prefix (aka `logIdent`). 31 | 32 | === [[doWork]] Processing Controller Events -- `doWork` Method 33 | 34 | [source, scala] 35 | ---- 36 | doWork(): Unit 37 | ---- 38 | 39 | `doWork` takes (and removes) a <> from the head of the <> (waiting for a `QueuedEvent` to be available if the queue is empty). 40 | 41 | NOTE: The very first event in the event queue is `Startup` that `KafkaController` puts when it is link:kafka-controller-KafkaController.adoc#startup[started]. 42 | 43 | `doWork` sets <> (of `ControllerEventManager`) to be the <> of the controller event. 44 | 45 | `doWork` requests the <> to `update` (with the time between the current time and the <>). 46 | 47 | `doWork` finds the `KafkaTimer` for the current controller state (in <> lookup table) to measure and record the time to requests the event to <> using the <>. 48 | 49 | In the end, `doWork` sets the <> (of `ControllerEventManager`) as `Idle`. 50 | 51 | In case of any error (`Throwable`), `doWork` simply prints out the following ERROR message to the logs: 52 | 53 | ``` 54 | Uncaught error processing event [controllerEvent] 55 | ``` 56 | -------------------------------------------------------------------------------- /docs/controller/Election.md: -------------------------------------------------------------------------------- 1 | # Election 2 | 3 | `Election` is a utility with the algorithms for [partition leader election](../partition-leader-election/index.md). 4 | 5 | ## leaderForOffline 6 | 7 | ```scala 8 | leaderForOffline( 9 | controllerContext: ControllerContext, 10 | isLeaderRecoverySupported: Boolean, 11 | partitionsWithUncleanLeaderRecoveryState: Seq[(TopicPartition, Option[LeaderAndIsr], Boolean)] 12 | ): Seq[ElectionResult] // (1)! 13 | leaderForOffline( 14 | partition: TopicPartition, 15 | leaderAndIsrOpt: Option[LeaderAndIsr], 16 | uncleanLeaderElectionEnabled: Boolean, 17 | isLeaderRecoverySupported: Boolean, 18 | controllerContext: ControllerContext): ElectionResult 19 | ``` 20 | 21 | 1. Uses the other `leaderForOffline` for every tuple in `partitionsWithUncleanLeaderRecoveryState` 22 | 23 | `leaderForOffline` requests the given [ControllerContext](ControllerContext.md) for the [partition replicas](ControllerContext.md#partitionReplicaAssignment) and removes replicas that are not [online](ControllerContext.md#isReplicaOnline). 24 | 25 | For the input `LeaderAndIsr` defined, `leaderForOffline` [offlinePartitionLeaderElection](PartitionLeaderElectionAlgorithms.md#offlinePartitionLeaderElection) (with the replica brokers) and...FIXME 26 | 27 | --- 28 | 29 | `leaderForOffline` is used when: 30 | 31 | * `ZkPartitionStateMachine` is requested to [doElectLeaderForPartitions](ZkPartitionStateMachine.md#doElectLeaderForPartitions) (with [OfflinePartitionLeaderElectionStrategy](PartitionStateMachine.md#OfflinePartitionLeaderElectionStrategy)) 32 | -------------------------------------------------------------------------------- /docs/controller/LeaderAndIsrRequest.md: -------------------------------------------------------------------------------- 1 | # LeaderAndIsrRequest 2 | 3 | `LeaderAndIsrRequest` is a [controller request](AbstractControlRequest.md) with `LeaderAndIsr` API key and the following properties: 4 | 5 | * Version 6 | * Controller ID ([broker.id](../KafkaConfig.md#brokerId) of the controller broker) 7 | * Controller Epoch 8 | * Broker Epoch 9 | * `PartitionStates` by `TopicPartition` (`Map`) 10 | * Topic IDs 11 | * Live leaders 12 | 13 | `LeaderAndIsrRequest` is created (using [LeaderAndIsrRequest.Builder](#build)) when: 14 | 15 | * `AbstractRequest` is requested to [parse a request](../AbstractRequest.md#parseRequest) (with the [LeaderAndIsr](#LEADER_AND_ISR) API key) 16 | * `LeaderAndIsrRequest` is requested to [parse a byte buffer](#parse) 17 | * `LeaderAndIsrRequest.Builder` is requested to [build a LeaderAndIsrRequest](#build) (when `ControllerBrokerRequestBatch` is requested to [sendRequestsToBrokers](AbstractControllerBrokerRequestBatch.md#sendRequestsToBrokers)) 18 | 19 | ## LeaderAndIsrRequest.Builder 20 | 21 | `LeaderAndIsrRequest` comes with a concrete [AbstractRequest.Builder](../AbstractRequest.md#Builder) factory object that can build a `LeaderAndIsrRequest`. 22 | 23 | `LeaderAndIsrRequest.Builder` is used when: 24 | 25 | * `AbstractControllerBrokerRequestBatch` is requested to [send out LeaderAndIsr requests to leader and follower brokers](AbstractControllerBrokerRequestBatch.md#sendLeaderAndIsrRequest) 26 | -------------------------------------------------------------------------------- /docs/controller/PartitionLeaderElectionAlgorithms.md: -------------------------------------------------------------------------------- 1 | # PartitionLeaderElectionAlgorithms 2 | 3 | `PartitionLeaderElectionAlgorithms` is a utility with the algorithms for [partition leader election](../partition-leader-election/index.md). 4 | 5 | ## offlinePartitionLeaderElection 6 | 7 | ```scala 8 | offlinePartitionLeaderElection( 9 | assignment: Seq[Int], 10 | isr: Seq[Int], 11 | liveReplicas: Set[Int], 12 | uncleanLeaderElectionEnabled: Boolean, 13 | controllerContext: ControllerContext): Option[Int] 14 | ``` 15 | 16 | `offlinePartitionLeaderElection` finds the first broker ID (among the `liveReplicas`) that is among the `isr`. 17 | 18 | If not found and `uncleanLeaderElectionEnabled` flag is enabled, `offlinePartitionLeaderElection` finds the first live replica broker (from the `assignment` that is among `liveReplicas`). When successful and a live replica broker is found, `offlinePartitionLeaderElection` marks the occurrence of this unclean leader election event in [kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec](ControllerContext.md#uncleanLeaderElectionRate) metric. 19 | 20 | In the end, `offlinePartitionLeaderElection` returns a broker ID (if a new partition leader is found) or `None`. 21 | 22 | --- 23 | 24 | `offlinePartitionLeaderElection` is used when: 25 | 26 | * `Election` is requested to [leaderForOffline](Election.md#leaderForOffline) 27 | -------------------------------------------------------------------------------- /docs/controller/QueuedEvent.md: -------------------------------------------------------------------------------- 1 | # QueuedEvent 2 | 3 | `QueuedEvent` is a [ControllerEvent](#event) with the [time it was enqueued](#enqueueTimeMs) to [ControllerEventManager](ControllerEventManager.md). 4 | 5 | ## Creating Instance 6 | 7 | `QueuedEvent` takes the following to be created: 8 | 9 | * [ControllerEvent](ControllerEvent.md) 10 | * Enqueue time (in millis) 11 | 12 | `QueuedEvent` is created when: 13 | 14 | * `ControllerEventManager` is requested to [enqueue a ControllerEvent](ControllerEventManager.md#put) 15 | 16 | ## Processing ControllerEventProcessor 17 | 18 | ```scala 19 | process( 20 | processor: ControllerEventProcessor): Unit 21 | ``` 22 | 23 | `process` requests the input [ControllerEventProcessor](ControllerEventProcessor.md) to [process](ControllerEventProcessor.md#process) the [ControllerEvent](#event). 24 | 25 | --- 26 | 27 | `process` is used when: 28 | 29 | * `ControllerEventThread` is requested to [doWork](ControllerEventThread.md#doWork) 30 | 31 | ## String (Textual) Representation 32 | 33 | ```scala 34 | toString: String 35 | ``` 36 | 37 | `toString` is part of the [java.lang.Object](https://docs.oracle.com/en/java/javase/12/docs/api/java.base/java/lang/Object.html#toString()) abstraction. 38 | 39 | --- 40 | 41 | `toString` returns the following string representation (with the [ControllerEvent](#event) and the [enqueue time](#enqueueTimeMs)): 42 | 43 | ```text 44 | QueuedEvent(event=[event], enqueueTimeMs=[enqueueTimeMs]) 45 | ``` 46 | -------------------------------------------------------------------------------- /docs/controller/TopicUncleanLeaderElectionEnable.md: -------------------------------------------------------------------------------- 1 | # TopicUncleanLeaderElectionEnable 2 | 3 | `TopicUncleanLeaderElectionEnable` is a [ControllerEvent](ControllerEvent.md) that [KafkaController](KafkaController.md) uses to trigger [processTopicUncleanLeaderElectionEnable](KafkaController.md#processTopicUncleanLeaderElectionEnable). 4 | 5 | ## Creating Instance 6 | 7 | `TopicUncleanLeaderElectionEnable` takes the following to be created: 8 | 9 | * Topic Name 10 | 11 | `TopicUncleanLeaderElectionEnable` is created when: 12 | 13 | * `KafkaController` is requested to [enableTopicUncleanLeaderElection](KafkaController.md#enableTopicUncleanLeaderElection) (on an active controller) 14 | 15 | ## ControllerState 16 | 17 | ```scala 18 | state: ControllerState 19 | ``` 20 | 21 | `state` is part of the [ControllerEvent](ControllerEvent.md#state) abstraction. 22 | 23 | --- 24 | 25 | `state` is `TopicUncleanLeaderElectionEnable`. 26 | 27 | ## KafkaController 28 | 29 | 30 | -------------------------------------------------------------------------------- /docs/controller/ZkReplicaStateMachine.md: -------------------------------------------------------------------------------- 1 | # ZkReplicaStateMachine 2 | 3 | `ZkReplicaStateMachine` is a [ReplicaStateMachine](ReplicaStateMachine.md) to [handle changes of the state of partition replicas](#handleStateChanges). 4 | 5 | `ZkReplicaStateMachine` uses [ControllerBrokerRequestBatch](#controllerBrokerRequestBatch) to propagate _replica state changes_ to all brokers in a Kafka cluster. 6 | 7 | ## Creating Instance 8 | 9 | `ZkReplicaStateMachine` takes the following to be created: 10 | 11 | * [KafkaConfig](../KafkaConfig.md) 12 | * `StateChangeLogger` 13 | * [ControllerContext](ControllerContext.md) 14 | * [KafkaZkClient](../zk/KafkaZkClient.md) 15 | * [ControllerBrokerRequestBatch](ControllerBrokerRequestBatch.md) 16 | 17 | `ZkReplicaStateMachine` is created along with a [KafkaController](KafkaController.md#replicaStateMachine). 18 | 19 | ## Handling Replica State Changes 20 | 21 | ```scala 22 | handleStateChanges( 23 | replicas: Seq[PartitionAndReplica], 24 | targetState: ReplicaState): Unit 25 | ``` 26 | 27 | `handleStateChanges` is part of the [ReplicaStateMachine](ReplicaStateMachine.md#handleStateChanges) abstraction. 28 | 29 | --- 30 | 31 | !!! note 32 | `handleStateChanges` is a _noop_ and does nothing when the input `replicas` collection is empty. 33 | 34 | `handleStateChanges` requests the [ControllerBrokerRequestBatch](#controllerBrokerRequestBatch) for a [new batch](ControllerBrokerRequestBatch.md#newBatch). 35 | 36 | `handleStateChanges` groups the `replicas` by the replica ID and [doHandleStateChanges](#doHandleStateChanges) for every replica ID (with the `ReplicaState`). 37 | 38 | In the end, `handleStateChanges` requests the [ControllerBrokerRequestBatch](#controllerBrokerRequestBatch) to [sendRequestsToBrokers](ControllerBrokerRequestBatch.md#sendRequestsToBrokers). 39 | 40 | ### doHandleStateChanges 41 | 42 | ```scala 43 | doHandleStateChanges( 44 | replicaId: Int, 45 | replicas: Seq[PartitionAndReplica], 46 | targetState: ReplicaState): Unit 47 | ``` 48 | 49 | For every replica (in the `replicas`), `doHandleStateChanges` requests the [ControllerBrokerRequestBatch](#controllerBrokerRequestBatch) to [putReplicaStateIfNotExists](ControllerBrokerRequestBatch.md#putReplicaStateIfNotExists) (with `NonExistentReplica` state) 50 | 51 | `doHandleStateChanges` requests the [ControllerBrokerRequestBatch](#controllerBrokerRequestBatch) to [checkValidReplicaStateChange](ControllerBrokerRequestBatch.md#checkValidReplicaStateChange) (that gives valid and invalid replicas). 52 | 53 | For every invalid replica, `doHandleStateChanges` [logInvalidTransition](#logInvalidTransition). 54 | 55 | `doHandleStateChanges` branches off per the input target state (`ReplicaState`): 56 | 57 | * `NewReplica` 58 | * `OnlineReplica` 59 | * `OfflineReplica` 60 | * `ReplicaDeletionStarted` 61 | * `ReplicaDeletionIneligible` 62 | * `ReplicaDeletionSuccessful` 63 | * `NonExistentReplica` 64 | 65 | ## Logging 66 | 67 | Enable `ALL` logging level for `kafka.controller.ZkReplicaStateMachine` logger to see what happens inside. 68 | 69 | Add the following line to `config/log4j.properties`: 70 | 71 | ```text 72 | log4j.logger.kafka.controller.ZkReplicaStateMachine=ALL 73 | ``` 74 | 75 | Refer to [Logging](../logging.md). 76 | 77 | ### logIdent 78 | 79 | `ZkReplicaStateMachine` uses the following logging prefix (with the [broker.id](../KafkaConfig.md#brokerId)): 80 | 81 | ```text 82 | [ReplicaStateMachine controllerId=[brokerId]] 83 | ``` 84 | -------------------------------------------------------------------------------- /docs/controller/index.md: -------------------------------------------------------------------------------- 1 | # Controller Broker 2 | 3 | **Controller Broker** ([KafkaController](KafkaController.md)) is a Kafka service that runs on every broker in a Kafka cluster, but only one can be [active](#isActive) (_elected_) at any point in time. 4 | 5 | The process of promoting a broker to be the active controller is called [Kafka Controller Election](controller-election.md). 6 | 7 | !!! quote "[Kafka Controller Internals](https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Controller+Internals)" 8 | 9 | In a Kafka cluster, one of the brokers serves as the controller, which is responsible for managing the states of partitions and replicas and for performing administrative tasks like reassigning partitions. 10 | 11 | Kafka Controller registers [handlers](KafkaController.md#znode-change-handlers) to be notified about changes in Zookeeper and propagate them across brokers in a Kafka cluster. 12 | 13 | ## Lifecycle 14 | 15 | When [started](KafkaController.md#startup), `KafkaController` emits [Startup](ControllerEvent.md#Startup) controller event. That starts [controller election](KafkaController.md#elect) (on the [controller-event-thread](ControllerEventThread.md)). 16 | 17 | During controller election, one `KafkaController` becomes [active](KafkaController.md#isActive) (_elected_) and [onControllerFailover](KafkaController.md#onControllerFailover). The [ControllerContext](KafkaController.md#controllerContext) is built on what is available in Zookeeper. 18 | 19 | While in [initializeControllerContext](KafkaController.md#initializeControllerContext), `KafkaController` [updateLeaderAndIsrCache](KafkaController.md#updateLeaderAndIsrCache) (and [reads partition state](../zk/KafkaZkClient.md#getTopicPartitionStates) from `/brokers/topics/[topic]/partitions/[partition]/state` paths in Zookeeper that is then stored as [partitionLeadershipInfo](ControllerContext.md#partitionLeadershipInfo) of the [ControllerContext](KafkaController.md#controllerContext)). 20 | -------------------------------------------------------------------------------- /docs/demo/index.md: -------------------------------------------------------------------------------- 1 | --- 2 | hide: 3 | - toc 4 | --- 5 | 6 | # Demos 7 | 8 | The following demos are available: 9 | 10 | - [ACL Authorization](acl-authorization.md) 11 | - [Controller Election](controller-election.md) 12 | - [Kafka and kcat in Docker](kafka-and-kcat-in-docker.md) 13 | - [Secure Inter-Broker Communication](secure-inter-broker-communication.md) 14 | - [Securing Communication Between Clients and Brokers Using SSL](securing-communication-between-clients-and-brokers.md) 15 | - [SSL Authentication](ssl-authentication.md) 16 | - [Transactional Kafka Producer](transactional-kafka-producer.md) 17 | -------------------------------------------------------------------------------- /docs/demo/kafka-and-kcat-in-docker.md: -------------------------------------------------------------------------------- 1 | --- 2 | hide: 3 | - navigation 4 | --- 5 | 6 | # Demo: Kafka and kcat in Docker 7 | 8 | This demo uses Docker to run Apache Kafka and [kcat](https://github.com/edenhill/kcat) utility. 9 | 10 | ## kafka-docker 11 | 12 | Pull [kafka-docker](https://github.com/wurstmeister/kafka-docker) project (or create a `docker-compose.yml` file yourself). 13 | 14 | ### Running Kafka Cluster 15 | 16 | Start Zookeeper and Kafka containers. 17 | 18 | ```text 19 | docker-compose up 20 | ``` 21 | 22 | ```text 23 | $ docker-compose ps 24 | Name Command State Ports 25 | ---------------------------------------------------------------------------------------------------------------------------------------- 26 | kafka-docker_kafka_1 start-kafka.sh Up 0.0.0.0:62687->9092/tcp 27 | kafka-docker_zookeeper_1 /bin/sh -c /usr/sbin/sshd ... Up 0.0.0.0:2181->2181/tcp,:::2181->2181/tcp, 22/tcp, 2888/tcp, 3888/tcp 28 | ``` 29 | 30 | ### Docker Network 31 | 32 | The above creates a Docker network `kafka-docker_default` (if ran from `kafka-docker` directory as described in the [official documentation of docker-compose](https://docs.docker.com/compose/networking/)). 33 | 34 | ```text 35 | $ docker network ls 36 | NETWORK ID NAME DRIVER SCOPE 37 | b8b255710858 bridge bridge local 38 | 3c9c3a969ef2 cda bridge local 39 | 398f9f3196aa host host local 40 | 68611503fde8 kafka-docker_default bridge local 41 | db43a5e50281 none null local 42 | ``` 43 | 44 | ## kcat 45 | 46 | Connect `kcat` container to the network (using `--network` option as described in the [official documentation of docker-compose](https://docs.docker.com/engine/reference/commandline/network_connect/#connect-a-container-to-a-network-when-it-starts)). 47 | 48 | ### Metadata Listing 49 | 50 | ```text 51 | docker run -it --rm \ 52 | --network kafka-docker_default \ 53 | edenhill/kcat:1.7.0 \ 54 | -b kafka-docker_kafka_1:9092 -L 55 | ``` 56 | 57 | ```text 58 | Metadata for all topics (from broker -1: kafka-docker_kafka_1:9092/bootstrap): 59 | 1 brokers: 60 | broker 1001 at 09cc8de4d067:9092 (controller) 61 | 0 topics: 62 | ``` 63 | 64 | ### Producer 65 | 66 | ```text 67 | docker run -it --rm \ 68 | --network kafka-docker_default \ 69 | --name producer \ 70 | edenhill/kcat:1.7.0 \ 71 | -b kafka-docker_kafka_1:9092 -P -t t1 72 | ``` 73 | 74 | !!! caution 75 | For some reason the above command couldn't send messages whenever I pressed ENTER but expected `Ctrl+D` instead (that terminates the shell and the container). Switching to [confluentinc/cp-kafkacat](https://hub.docker.com/r/confluentinc/cp-kafkacat/) made things working fine. 76 | 77 | ```text 78 | docker run -it --rm \ 79 | --network kafka-docker_default \ 80 | --name producer \ 81 | confluentinc/cp-kafkacat \ 82 | kafkacat \ 83 | -b kafka-docker_kafka_1:9092 -P -t t1 84 | ``` 85 | 86 | ### Consumer 87 | 88 | ```text 89 | docker run -it --rm \ 90 | --network kafka-docker_default \ 91 | --name consumer \ 92 | edenhill/kcat:1.7.0 \ 93 | -b kafka-docker_kafka_1:9092 -C -t t1 94 | ``` 95 | 96 | ## Clean Up 97 | 98 | ```text 99 | docker-compose down 100 | ``` 101 | -------------------------------------------------------------------------------- /docs/demo/secure-inter-broker-communication.md: -------------------------------------------------------------------------------- 1 | # Demo: Secure Inter-Broker Communication 2 | 3 | The demo shows how to set up a secure communication between brokers (and disable the unsecure plaintext listener altogether). That will make Kafka brokers available via TLS/SSL only. 4 | 5 | ## Before You Begin 6 | 7 | The demo is a follow-up to [Demo: Securing Communication Between Clients and Brokers Using SSL](securing-communication-between-clients-and-brokers.md). Please finish it first before this demo. 8 | 9 | ## Configure Broker to Trust Certificate Authority 10 | 11 | Import the certificate of the certificate authority (CA) to a broker truststore so the brokers can trust it (when a broker tries to connect using SSL). 12 | 13 | ```shell 14 | $ keytool \ 15 | -import \ 16 | -file ca.crt \ 17 | -keystore server.truststore \ 18 | -alias ca \ 19 | -storepass 123456 \ 20 | -noprompt 21 | Certificate was added to keystore 22 | ``` 23 | 24 | Use `keytool` to print out the certificates in the client keystore. 25 | 26 | ```shell 27 | keytool -list -v -keystore server.truststore -storepass 123456 28 | ``` 29 | 30 | There should be 1 entry for the CA. 31 | 32 | ```shell 33 | $ keytool -list -v -keystore server.truststore -storepass 123456 34 | Keystore type: PKCS12 35 | Keystore provider: SUN 36 | 37 | Your keystore contains 1 entry 38 | 39 | Alias name: ca 40 | # ...removed for brevity 41 | ``` 42 | 43 | ## Enable SSL for Inter-Broker Communication 44 | 45 | Edit `config/server-ssl.properties` and add the following configuration properties to enable SSL for inter-broker communication: 46 | 47 | ```text 48 | security.inter.broker.protocol=SSL 49 | ssl.truststore.location=/tmp/kafka-ssl-demo/server.truststore 50 | ssl.truststore.password=123456 51 | ``` 52 | 53 | Start the broker(s). 54 | 55 | ```shell 56 | ./bin/kafka-server-start.sh config/server-ssl.properties 57 | ``` 58 | 59 | !!! tip 60 | Use `export KAFKA_OPTS=-Djavax.net.debug=all` to debug SSL issues. 61 | 62 | Verify the SSL configuration of the broker. The following uses the Cryptography and SSL/TLS Toolkit (OpenSSL) and the client tool. 63 | 64 | ```shell 65 | openssl s_client -connect localhost:9093 66 | ``` 67 | 68 | ## Disable Plaintext Unsecure Listener 69 | 70 | Edit `config/server-ssl.properties` and change `listeners` property to use `SSL://:9093` only: 71 | 72 | ```text 73 | listeners=SSL://:9093 74 | ``` 75 | 76 | Start the broker(s). 77 | 78 | ```shell 79 | ./bin/kafka-server-start.sh config/server-ssl.properties 80 | ``` 81 | 82 | !!! tip 83 | Use `export KAFKA_OPTS=-Djavax.net.debug=all` to debug SSL-related issues. 84 | 85 | Verify the SSL configuration of the broker. The following uses the Cryptography and SSL/TLS Toolkit (OpenSSL) and the client tool. 86 | 87 | ```shell 88 | openssl s_client -connect localhost:9093 89 | ``` 90 | 91 | Enter `Ctrl-C` to close the session. 92 | 93 | *That's all for the demo. I hope you enjoyed it!* 94 | -------------------------------------------------------------------------------- /docs/dynamic-broker-configuration/.pages: -------------------------------------------------------------------------------- 1 | title: Dynamic Broker Configuration 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/dynamic-broker-configuration/BrokerConfigHandler.md: -------------------------------------------------------------------------------- 1 | # BrokerConfigHandler 2 | 3 | ## processConfigChanges 4 | 5 | ```scala 6 | processConfigChanges( 7 | brokerId: String, 8 | properties: Properties): Unit 9 | ``` 10 | 11 | `processConfigChanges` is part of the [ConfigHandler](ConfigHandler.md#processConfigChanges) abstraction. 12 | 13 | --- 14 | 15 | `processConfigChanges`...FIXME 16 | -------------------------------------------------------------------------------- /docs/dynamic-broker-configuration/BrokerReconfigurable.md: -------------------------------------------------------------------------------- 1 | # BrokerReconfigurable 2 | 3 | `BrokerReconfigurable` is an [abstraction](#contract) of [dynamic reconfigurables](#implementations) that can be [reconfigured](#reconfigure) at runtime. 4 | 5 | ## Contract 6 | 7 | ### Reconfigurable Configs 8 | 9 | ```scala 10 | reconfigurableConfigs: Set[String] 11 | ``` 12 | 13 | Used when: 14 | 15 | * `DynamicBrokerConfig` is requested to [addBrokerReconfigurable](DynamicBrokerConfig.md#addBrokerReconfigurable) and [processReconfiguration](DynamicBrokerConfig.md#processReconfiguration) 16 | 17 | ### validateReconfiguration 18 | 19 | ```scala 20 | validateReconfiguration( 21 | newConfig: KafkaConfig): Unit 22 | ``` 23 | 24 | Validates the updated [KafkaConfig](../KafkaConfig.md) 25 | 26 | Used when: 27 | 28 | * `DynamicBrokerConfig` is requested to [processReconfiguration](DynamicBrokerConfig.md#processReconfiguration) 29 | 30 | ### Reconfiguring Broker 31 | 32 | ```scala 33 | reconfigure( 34 | oldConfig: KafkaConfig, 35 | newConfig: KafkaConfig): Unit 36 | ``` 37 | 38 | Used when: 39 | 40 | * `DynamicBrokerConfig` is requested to [updateCurrentConfig](DynamicBrokerConfig.md#updateCurrentConfig) 41 | 42 | ## Implementations 43 | 44 | * `DynamicListenerConfig` 45 | * [DynamicLogConfig](DynamicLogConfig.md) 46 | * [DynamicThreadPool](DynamicThreadPool.md) 47 | * [LogCleaner](../log/LogCleaner.md) 48 | * `SocketServer` 49 | -------------------------------------------------------------------------------- /docs/dynamic-broker-configuration/ConfigHandler.md: -------------------------------------------------------------------------------- 1 | # ConfigHandler 2 | 3 | `ConfigHandler` is an [abstraction](#contract) of [config change handlers](#implementations) that can [process configuration changes](#processConfigChanges). 4 | 5 | ## Contract 6 | 7 | ### processConfigChanges 8 | 9 | ```scala 10 | processConfigChanges( 11 | entityName: String, 12 | value: Properties): Unit 13 | ``` 14 | 15 | Used when: 16 | 17 | * `ConfigChangedNotificationHandler` is requested to `processEntityConfigChangeVersion1`, `processEntityConfigChangeVersion2` 18 | * `ZkConfigManager` is requested to [start up](ZkConfigManager.md#startup) 19 | * `BrokerMetadataPublisher` is requested to `publish` 20 | 21 | ## Implementations 22 | 23 | * [BrokerConfigHandler](BrokerConfigHandler.md) 24 | * `ClientIdConfigHandler` 25 | * `IpConfigHandler` 26 | * [TopicConfigHandler](TopicConfigHandler.md) 27 | * `UserConfigHandler` 28 | -------------------------------------------------------------------------------- /docs/dynamic-broker-configuration/DynamicLogConfig.md: -------------------------------------------------------------------------------- 1 | # DynamicLogConfig 2 | 3 | `DynamicLogConfig` is a [BrokerReconfigurable](BrokerReconfigurable.md) of [LogManager](#logManager). 4 | 5 | ## Creating Instance 6 | 7 | `DynamicLogConfig` takes the following to be created: 8 | 9 | * [LogManager](../log/LogManager.md) 10 | * [KafkaBroker](../broker/KafkaBroker.md) 11 | 12 | `DynamicLogConfig` is created when: 13 | 14 | * `DynamicBrokerConfig` is requested to [register Reconfigurables](DynamicBrokerConfig.md#addReconfigurables) 15 | 16 | ## Reconfigurable Configs 17 | 18 | ```scala 19 | reconfigurableConfigs: Set[String] 20 | ``` 21 | 22 | `reconfigurableConfigs` is part of the [BrokerReconfigurable](BrokerReconfigurable.md#reconfigurableConfigs) abstraction. 23 | 24 | --- 25 | 26 | `reconfigurableConfigs` returns the values of the [TopicConfigSynonyms](../log/LogConfig.md#TopicConfigSynonyms). 27 | 28 | ## Reconfiguring Broker 29 | 30 | ```scala 31 | reconfigure( 32 | oldConfig: KafkaConfig, 33 | newConfig: KafkaConfig): Unit 34 | ``` 35 | 36 | `reconfigure` is part of the [BrokerReconfigurable](BrokerReconfigurable.md#reconfigure) abstraction. 37 | 38 | --- 39 | 40 | `reconfigure` requests the [LogManager](#logManager) for the [currentDefaultConfig](../log/LogManager.md#currentDefaultConfig) (and the value of [unclean.leader.election.enable](../log/LogConfig.md#uncleanLeaderElectionEnable) configuration property explicitly). 41 | 42 | `reconfigure` updates [reconfigurable configuration properties](DynamicLogConfig.md#ReconfigurableConfigs) only. 43 | 44 | `reconfigure` requests the [LogManager](#logManager) to [reconfigureDefaultLogConfig](../log/LogManager.md#reconfigureDefaultLogConfig) with the new broker configs. 45 | 46 | `reconfigure` [updateLogsConfig](#updateLogsConfig) (with the new broker configs). 47 | 48 | In the end, `reconfigure` requests the [KafkaController](../broker/KafkaServer.md#kafkaController) to [enableDefaultUncleanLeaderElection](../controller/KafkaController.md#enableDefaultUncleanLeaderElection) when [unclean.leader.election.enable](../log/LogConfig.md#uncleanLeaderElectionEnable) is currently enabled while it was not before. 49 | -------------------------------------------------------------------------------- /docs/dynamic-broker-configuration/DynamicThreadPool.md: -------------------------------------------------------------------------------- 1 | # DynamicThreadPool 2 | 3 | `DynamicThreadPool` is a [BrokerReconfigurable](BrokerReconfigurable.md) 4 | 5 | ## reconfigure 6 | 7 | ```scala 8 | reconfigure( 9 | oldConfig: KafkaConfig, 10 | newConfig: KafkaConfig): Unit 11 | ``` 12 | 13 | `reconfigure` is part of the [BrokerReconfigurable](BrokerReconfigurable.md#reconfigure) abstraction. 14 | 15 | --- 16 | 17 | `reconfigure`...FIXME 18 | -------------------------------------------------------------------------------- /docs/dynamic-broker-configuration/TopicConfigHandler.md: -------------------------------------------------------------------------------- 1 | # TopicConfigHandler 2 | 3 | `TopicConfigHandler` is a [ConfigHandler](ConfigHandler.md). 4 | 5 | ## Creating Instance 6 | 7 | `TopicConfigHandler` takes the following to be created: 8 | 9 | * [LogManager](../log/LogManager.md) 10 | * [KafkaConfig](../KafkaConfig.md) 11 | * `QuotaManagers` 12 | * Optional [KafkaController](../controller/KafkaController.md) 13 | 14 | `TopicConfigHandler` is created when: 15 | 16 | * `BrokerServer` is requested to [startup](../kraft/BrokerServer.md#startup) (and create [dynamicConfigHandlers](../kraft/BrokerServer.md#dynamicConfigHandlers)) 17 | * `KafkaServer` is requested to [startup](../broker/KafkaServer.md#startup) (and create [dynamicConfigHandlers](../broker/KafkaServer.md#dynamicConfigHandlers)) 18 | 19 | ## processConfigChanges 20 | 21 | ```scala 22 | processConfigChanges( 23 | topic: String, 24 | topicConfig: Properties): Unit 25 | ``` 26 | 27 | `processConfigChanges` is part of the [ConfigHandler](ConfigHandler.md#processConfigChanges) abstraction. 28 | 29 | --- 30 | 31 | `processConfigChanges`...FIXME 32 | 33 | ## Logging 34 | 35 | Enable `ALL` logging level for `kafka.server.TopicConfigHandler` logger to see what happens inside. 36 | 37 | Add the following line to `config/log4j.properties`: 38 | 39 | ```text 40 | log4j.logger.kafka.server.TopicConfigHandler=ALL 41 | ``` 42 | 43 | Refer to [Logging](../logging.md). 44 | -------------------------------------------------------------------------------- /docs/dynamic-broker-configuration/ZkConfigManager.md: -------------------------------------------------------------------------------- 1 | # ZkConfigManager 2 | 3 | ## Creating Instance 4 | 5 | `ZkConfigManager` takes the following to be created: 6 | 7 | * [KafkaZkClient](../zk/KafkaZkClient.md) 8 | * [ConfigHandler](ConfigHandler.md)s by name 9 | 10 | `ZkConfigManager` is created when: 11 | 12 | * `KafkaServer` is requested to [start up](../broker/KafkaServer.md#startup) (and initializes the [dynamicConfigManager](../broker/KafkaServer.md#dynamicConfigManager)) 13 | 14 | ## Starting Up 15 | 16 | ```scala 17 | startup(): Unit 18 | ``` 19 | 20 | `startup` begins watching for config changes by requesting the [ZkNodeChangeNotificationListener](#configChangeListener) to [initialize](ZkNodeChangeNotificationListener.md#init). 21 | 22 | `startup`...FIXME 23 | 24 | --- 25 | 26 | `startup` is used when: 27 | 28 | * `KafkaServer` is requested to [start up](../broker/KafkaServer.md#startup) 29 | -------------------------------------------------------------------------------- /docs/dynamic-broker-configuration/ZkNodeChangeNotificationListener.md: -------------------------------------------------------------------------------- 1 | # ZkNodeChangeNotificationListener 2 | 3 | ## Creating Instance 4 | 5 | `ZkNodeChangeNotificationListener` takes the following to be created: 6 | 7 | * [KafkaZkClient](../zk/KafkaZkClient.md) 8 | * Node Root 9 | * Node Prefix 10 | * `NotificationHandler` 11 | * Change Expiration (in millis, default: `15 * 60 * 1000`) 12 | * `Time` 13 | 14 | `ZkNodeChangeNotificationListener` is created when: 15 | 16 | * `DelegationTokenManager` is requested to `startup` 17 | * `ZkConfigManager` is [created](ZkConfigManager.md#configChangeListener) 18 | * `ZkAclChangeStore` is requested to `createListener` 19 | -------------------------------------------------------------------------------- /docs/dynamic-broker-configuration/index.md: -------------------------------------------------------------------------------- 1 | # Dynamic Broker Configuration 2 | 3 | [KIP-226 - Dynamic Broker Configuration](https://cwiki.apache.org/confluence/display/KAFKA/KIP-226+-+Dynamic+Broker+Configuration) 4 | -------------------------------------------------------------------------------- /docs/dynamic-configuration/.pages: -------------------------------------------------------------------------------- 1 | title: Dynamic Configuration 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/dynamic-configuration/Reconfigurable.md: -------------------------------------------------------------------------------- 1 | # Reconfigurable 2 | 3 | `Reconfigurable` is an [extension](#contract) of the [Configurable](../Configurable.md) abstraction for [services](#implementations) that support [Dynamic Configuration](index.md). 4 | 5 | ## Contract (Subset) 6 | 7 | ### reconfigurableConfigs { #reconfigurableConfigs } 8 | 9 | ```java 10 | Set reconfigurableConfigs() 11 | ``` 12 | 13 | See: 14 | 15 | * [MetricsReporter](../metrics/MetricsReporter.md#reconfigurableConfigs) 16 | 17 | Used when: 18 | 19 | * `SslFactory` is requested to `createNewSslEngineFactory` 20 | * `DataPlaneAcceptor` is requested to `validateReconfiguration` 21 | * `DynamicBrokerConfig` is requested to [addReconfigurable](../dynamic-broker-configuration/DynamicBrokerConfig.md#addReconfigurable), [maybeReconfigure](../dynamic-broker-configuration/DynamicBrokerConfig.md#maybeReconfigure), [processListenerReconfigurable](../dynamic-broker-configuration/DynamicBrokerConfig.md#processListenerReconfigurable), [processReconfiguration](../dynamic-broker-configuration/DynamicBrokerConfig.md#processReconfiguration), [reloadUpdatedFilesWithoutConfigChange](../dynamic-broker-configuration/DynamicBrokerConfig.md#reloadUpdatedFilesWithoutConfigChange) 22 | * `DynamicClientQuotaCallback` is requested to `reconfigurableConfigs` 23 | * `DynamicMetricsReporters` is requested to `reconfigurableConfigs` 24 | 25 | ## Implementations 26 | 27 | * `DynamicClientQuotaCallback` 28 | * `DynamicMetricsReporters` 29 | * `KafkaYammerMetrics` 30 | * `ListenerReconfigurable` 31 | * [MetricsReporter](../metrics/MetricsReporter.md) 32 | * `SslFactory` 33 | -------------------------------------------------------------------------------- /docs/dynamic-configuration/index.md: -------------------------------------------------------------------------------- 1 | # Dynamic Configuration 2 | 3 | The main abstraction: [Reconfigurable](Reconfigurable.md). 4 | -------------------------------------------------------------------------------- /docs/features/index.md: -------------------------------------------------------------------------------- 1 | --- 2 | hide: 3 | - toc 4 | --- 5 | 6 | # Features 7 | 8 | The following is a list of the features of {{ book.title }} that make it so amazing (even _[awesomesauce](https://dictionary.cambridge.org/dictionary/english/awesomesauce)_ 😏): 9 | 10 | * [KRaft](../kraft/index.md) 11 | * [Log Cleanup](../log-cleanup/index.md) 12 | * _others_ (listed in the menu on the left) 13 | -------------------------------------------------------------------------------- /docs/images/AclAuthorizer-jconsole.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/docs/images/AclAuthorizer-jconsole.png -------------------------------------------------------------------------------- /docs/images/ControllerEventManager-jconsole.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/docs/images/ControllerEventManager-jconsole.png -------------------------------------------------------------------------------- /docs/images/ControllerEventManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/docs/images/ControllerEventManager.png -------------------------------------------------------------------------------- /docs/images/ControllerEventThread-doWork.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/docs/images/ControllerEventThread-doWork.png -------------------------------------------------------------------------------- /docs/images/GroupCoordinator-startup.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/docs/images/GroupCoordinator-startup.png -------------------------------------------------------------------------------- /docs/images/GroupMetadataManager-creating-instance.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/docs/images/GroupMetadataManager-creating-instance.png -------------------------------------------------------------------------------- /docs/images/GroupMetadataManager-jconsole.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/docs/images/GroupMetadataManager-jconsole.png -------------------------------------------------------------------------------- /docs/images/KafkaApis.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/docs/images/KafkaApis.png -------------------------------------------------------------------------------- /docs/images/KafkaController.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/docs/images/KafkaController.png -------------------------------------------------------------------------------- /docs/images/LogCleanerManager-jconsole.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/docs/images/LogCleanerManager-jconsole.png -------------------------------------------------------------------------------- /docs/images/LogManager.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/docs/images/LogManager.png -------------------------------------------------------------------------------- /docs/images/ZkPartitionStateMachine.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/docs/images/ZkPartitionStateMachine.png -------------------------------------------------------------------------------- /docs/images/kafka-controller-jconsole.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/docs/images/kafka-controller-jconsole.png -------------------------------------------------------------------------------- /docs/index.md: -------------------------------------------------------------------------------- 1 | --- 2 | title: Apache Kafka 3 | icon: material/home 4 | hide: 5 | - toc 6 | - navigation 7 | --- 8 | 9 | # The Internals of {{ book.title }} {{ kafka.version }} 10 | 11 | Welcome to **The Internals of {{ book.title }}** online book! 🤙 12 | 13 | I'm [Jacek Laskowski](https://pl.linkedin.com/in/jaceklaskowski), a Freelance Data Engineer specializing in 14 | [Apache Spark](https://books.japila.pl/apache-spark-internals/) (incl. [Spark SQL](https://books.japila.pl/spark-sql-internals/) and [Spark Structured Streaming](https://books.japila.pl/spark-structured-streaming-internals/)), 15 | [Delta Lake](https://books.japila.pl/delta-lake-internals/), 16 | [Databricks](https://www.databricks.com/), 17 | and [Apache Kafka](https://books.japila.pl/kafka-internals/) (incl. [Kafka Streams](https://books.japila.pl/kafka-streams-internals/)) with brief forays into a wider data engineering space (e.g., [Trino](https://trino.io/), [Dask](https://www.dask.org/) and [dbt](https://www.getdbt.com/), mostly during [Warsaw Data Engineering](https://www.meetup.com/Warsaw-Data-Engineering/) meetups). 18 | 19 | I'm very excited to have you here and hope you will enjoy exploring the internals of {{ book.title }} as much as I have. 20 | 21 | !!! quote "Flannery O'Connor" 22 | I write to discover what I know. 23 | 24 | !!! note ""The Internals Of" series" 25 | I'm also writing other online books in the "The Internals Of" series. Please visit ["The Internals Of" Online Books](https://books.japila.pl) home page. 26 | 27 | Expect text and code snippets from a variety of public sources. Attribution follows. 28 | 29 | Now, let's take a deep dive into [{{ book.title }}](features/index.md) 🔥 30 | 31 | --- 32 | 33 | Last update: {{ git.date.strftime('%Y-%m-%d') }} 34 | -------------------------------------------------------------------------------- /docs/kraft/.pages: -------------------------------------------------------------------------------- 1 | title: Kafka Raft (KRaft) 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/kraft/BrokerServer.md: -------------------------------------------------------------------------------- 1 | # BrokerServer 2 | 3 | `BrokerServer` is a [KafkaBroker](../broker/KafkaBroker.md) that runs in [KRaft mode](index.md). 4 | 5 | ## Creating Instance 6 | 7 | `BrokerServer` takes the following to be created: 8 | 9 | * [KafkaConfig](../KafkaConfig.md) 10 | * [MetaProperties](MetaProperties.md) 11 | * [RaftManager](RaftManager.md) 12 | * `Time` 13 | * [Metrics](../metrics/Metrics.md) 14 | * Optional `threadNamePrefix` 15 | * [Initial Offline Log Directories](#initialOfflineDirs) 16 | * `controllerQuorumVotersFuture` (`CompletableFuture[util.Map[Integer, AddressSpec]]`) 17 | * Supported Features 18 | 19 | `BrokerServer` is created when: 20 | 21 | * `KafkaRaftServer` is [created](KafkaRaftServer.md#broker) (with `BrokerRole` among the [processRoles](../KafkaConfig.md#processRoles)) 22 | 23 | ### Initial Offline Log Directories 24 | 25 | ```scala 26 | initialOfflineDirs: Seq[String] 27 | ``` 28 | 29 | `BrokerServer` is given `initialOfflineDirs` when [created](#creating-instance) (that is [offlineDirs](KafkaRaftServer.md#offlineDirs)). 30 | 31 | `initialOfflineDirs` is used to [create a LogManager](#logManager) when `BrokerServer` is requested to [startup](#startup). 32 | 33 | ## startup 34 | 35 | ```scala 36 | startup(): Unit 37 | ``` 38 | 39 | `startup`...FIXME 40 | 41 | `startup` is used when: 42 | 43 | * `KafkaRaftServer` is requested to [startup](KafkaRaftServer.md#startup) 44 | -------------------------------------------------------------------------------- /docs/kraft/ControllerServer.md: -------------------------------------------------------------------------------- 1 | # ControllerServer 2 | 3 | `ControllerServer` represents the `controller` in [process.roles](../KafkaConfig.md#process.roles). 4 | 5 | ## Creating Instance 6 | 7 | `ControllerServer` takes the following to be created: 8 | 9 | * [SharedServer](SharedServer.md) 10 | * `KafkaConfigSchema` 11 | * `BootstrapMetadata` 12 | 13 | `ControllerServer` is created alongside a [KafkaRaftServer](KafkaRaftServer.md#controller) (for `controller` in [process.roles](../KafkaConfig.md#process.roles)). 14 | 15 | ## Starting Up { #startup } 16 | 17 | ```scala 18 | startup(): Unit 19 | ``` 20 | 21 | `startup` changes status from `SHUTDOWN` to `STARTING`. 22 | 23 | `startup` uses [server.max.startup.time.ms](../KafkaConfig.md#server.max.startup.time.ms) for...FIXME 24 | 25 | `startup` prints out the following INFO message to the logs: 26 | 27 | ```text 28 | Starting controller 29 | ``` 30 | 31 | `startup` requests the [DynamicBrokerConfig](../KafkaConfig.md#dynamicConfig) (of the [KafkaConfig](#config)) to [initialize](../dynamic-broker-configuration/DynamicBrokerConfig.md#initialize) (with no `KafkaZkClient` as it runs in Zookeeper-less KRaft mode). 32 | 33 | `startup` changes status from `STARTING` to `STARTED`. 34 | 35 | `startup` registers new metrics (gauges) in the [KafkaMetricsGroup](#metricsGroup). 36 | 37 | Metric Name | Description 38 | -------------|------------ 39 | `ClusterId` | [clusterId](#clusterId) 40 | `yammer-metrics-count` | 41 | `linux-disk-read-bytes` | (only on Linux) 42 | `linux-disk-write-bytes` | (only on Linux) 43 | 44 | `startup`...FIXME 45 | 46 | !!! note 47 | There is a lot services being registered that seem not necessarily as important at this early stage of the KRaft exploration of mine 😉 48 | 49 | `startup` requests the [SharedServer](#sharedServer) to [startForController](SharedServer.md#startForController). 50 | 51 | `startup`...FIXME 52 | 53 | `startup` builds the [QuorumController](#controller). 54 | 55 | `startup`...FIXME 56 | 57 | In the end, `startup` requests the [DynamicBrokerConfig](../KafkaConfig.md#dynamicConfig) 58 | 59 | `startup` [registers](../dynamic-broker-configuration/DynamicBrokerConfig.md#addReconfigurables) this `ControllerServer` for dynamic config changes (to the [KafkaConfig](#config)). 60 | 61 | --- 62 | 63 | `startup` is used when: 64 | 65 | * `KafkaRaftServer` is requested to [startup](KafkaRaftServer.md#startup) 66 | 67 | ## Logging 68 | 69 | Enable `ALL` logging level for `kafka.server.ControllerServer` logger to see what happens inside. 70 | 71 | Add the following line to `config/log4j.properties`: 72 | 73 | ```text 74 | log4j.logger.kafka.server.ControllerServer=ALL 75 | ``` 76 | 77 | Refer to [Logging](../logging.md). 78 | -------------------------------------------------------------------------------- /docs/kraft/FileBasedStateStore.md: -------------------------------------------------------------------------------- 1 | # FileBasedStateStore 2 | 3 | `FileBasedStateStore` is a [QuorumStateStore](QuorumStateStore.md) that [writes election state](#writeElectionState) to a [file](#stateFile). 4 | 5 | ## Creating Instance 6 | 7 | `FileBasedStateStore` takes the following to be created: 8 | 9 | * [State File](#stateFile) 10 | 11 | `FileBasedStateStore` is created when: 12 | 13 | * `KafkaRaftManager` is requested to [build a RaftClient](KafkaRaftManager.md#buildRaftClient) 14 | 15 | ## State File { #stateFile } 16 | 17 | `FileBasedStateStore` is given a **state file** when [created](#creating-instance). 18 | 19 | The state file is `quorum-state` file under the [data directory](KafkaRaftManager.md#dataDir) (of [KafkaRaftManager](KafkaRaftManager.md)). 20 | 21 | The state file is loaded in [readElectionState](#readElectionState) and updated in [writeElectionState](#writeElectionState). 22 | 23 | `stateFile` is deleted in [clear](#clear). 24 | 25 | ## readElectionState { #readElectionState } 26 | 27 | ??? note "QuorumStateStore" 28 | 29 | ```java 30 | ElectionState readElectionState() 31 | ``` 32 | 33 | `readElectionState` is part of the [QuorumStateStore](QuorumStateStore.md#readElectionState) abstraction. 34 | 35 | `readElectionState` is `null` (undefined) when [stateFile](#stateFile) does not exist. 36 | -------------------------------------------------------------------------------- /docs/kraft/KafkaMetadataLog.md: -------------------------------------------------------------------------------- 1 | # KafkaMetadataLog 2 | 3 | `KafkaMetadataLog` is a `ReplicatedLog`. 4 | 5 | ## Creating Instance 6 | 7 | `KafkaMetadataLog` takes the following to be created: 8 | 9 | * [UnifiedLog](../log/UnifiedLog.md) 10 | * `Time` 11 | * `Scheduler` 12 | * Snapshots 13 | * `TopicPartition` 14 | * `MetadataLogConfig` 15 | 16 | `KafkaMetadataLog` is created using [apply](#apply) utility. 17 | 18 | ## Creating KafkaMetadataLog { #apply } 19 | 20 | ```scala 21 | apply( 22 | topicPartition: TopicPartition, 23 | topicId: Uuid, 24 | dataDir: File, 25 | time: Time, 26 | scheduler: Scheduler, 27 | config: MetadataLogConfig): KafkaMetadataLog 28 | ``` 29 | 30 | `apply`...FIXME 31 | 32 | --- 33 | 34 | `apply` is used when: 35 | 36 | * `KafkaRaftManager` is requested to [buildMetadataLog](KafkaRaftManager.md#buildMetadataLog) 37 | -------------------------------------------------------------------------------- /docs/kraft/KafkaRaftClient.ListenerContext.md: -------------------------------------------------------------------------------- 1 | # KafkaRaftClient.ListenerContext 2 | 3 | ## Creating Instance 4 | 5 | `KafkaRaftClient.ListenerContext` takes the following to be created: 6 | 7 | * [RaftClient.Listener](RaftClient.Listener.md) 8 | 9 | `KafkaRaftClient.ListenerContext` is created when: 10 | 11 | * `KafkaRaftClient` is requested to [processRegistration](KafkaRaftClient.md#processRegistration) 12 | -------------------------------------------------------------------------------- /docs/kraft/KafkaRaftClient.md: -------------------------------------------------------------------------------- 1 | # KafkaRaftClient 2 | 3 | `KafkaRaftClient` is a [RaftClient](RaftClient.md). 4 | 5 | ## Creating Instance 6 | 7 | `KafkaRaftClient` takes the following to be created: 8 | 9 | * `RecordSerde` 10 | * `NetworkChannel` 11 | * `RaftMessageQueue` 12 | * `ReplicatedLog` 13 | * [QuorumStateStore](#quorumStateStore) 14 | * `MemoryPool` 15 | * `Time` 16 | * [Metrics](../metrics/Metrics.md) 17 | * `ExpirationService` 18 | * `fetchMaxWaitMs` (`500` ms) 19 | * Cluster ID 20 | * [Node ID](../KafkaConfig.md#nodeId) 21 | * `LogContext` 22 | * `Random` 23 | * [RaftConfig](RaftConfig.md) 24 | 25 | `KafkaRaftClient` is created when: 26 | 27 | * `KafkaRaftManager` is requested to [build a KafkaRaftClient](KafkaRaftManager.md#buildRaftClient) 28 | 29 | ### QuorumStateStore { #quorumStateStore } 30 | 31 | `KafkaRaftClient` is given a [QuorumStateStore](QuorumStateStore.md) when [created](#creating-instance). 32 | 33 | The `QuorumStateStore` is a [FileBasedStateStore](FileBasedStateStore.md) with the `quorum-state` state file in the [dataDir](KafkaRaftManager.md#dataDir). 34 | 35 | ## poll { #poll } 36 | 37 | ```java 38 | void poll() 39 | ``` 40 | 41 | `poll`...FIXME 42 | 43 | --- 44 | 45 | `poll` is used when: 46 | 47 | * `RaftIoThread` is requested to [doWork](RaftIoThread.md#doWork) 48 | 49 | ### pollListeners { #pollListeners } 50 | 51 | ```java 52 | void pollListeners() 53 | ``` 54 | 55 | `pollListeners`...FIXME 56 | 57 | ### processRegistration { #processRegistration } 58 | 59 | ```java 60 | void processRegistration( 61 | Registration registration) 62 | ``` 63 | 64 | `processRegistration`...FIXME 65 | -------------------------------------------------------------------------------- /docs/kraft/KafkaRaftManager.md: -------------------------------------------------------------------------------- 1 | # KafkaRaftManager 2 | 3 | `KafkaRaftManager` is a [RaftManager](RaftManager.md). 4 | 5 | `KafkaRaftManager` is [created](#creating-instance) and immediately [started](#startup) when `SharedServer` is [started](SharedServer.md#start). 6 | 7 | ## Creating Instance 8 | 9 | `KafkaRaftManager` takes the following to be created: 10 | 11 | * [MetaProperties](MetaProperties.md) 12 | * [KafkaConfig](../KafkaConfig.md) 13 | * `RecordSerde[T]` 14 | * [Partition 0 of __cluster_metadata](#topicPartition) 15 | * Topic ID (UUID) 16 | * `Time` 17 | * [Metrics](../metrics/Metrics.md) 18 | * Thread Name Prefix (optional) 19 | * Controller Quorum Voters (`CompletableFuture[util.Map[Integer, AddressSpec]]`) 20 | * `FaultHandler` 21 | 22 | `KafkaRaftManager` is created when: 23 | 24 | * `KafkaServer` is requested to [start up](../broker/KafkaServer.md#startup) (with [zookeeper.metadata.migration.enable](../KafkaConfig.md#zookeeper.metadata.migration.enable) enabled) 25 | * `SharedServer` is requested to [start](SharedServer.md#start) 26 | 27 | ### \_\_cluster_metadata-0 Partition { #topicPartition } 28 | 29 | `KafkaRaftManager` is given a `TopicPartition` when [created](#creating-instance): 30 | 31 | * `__cluster_metadata` as the name of the cluster metadata topic 32 | * Partition `0` 33 | 34 | ## Metadata Log Directory { #dataDir } 35 | 36 | `KafkaRaftManager` [creates a data directory](KafkaRaftManager.md#createDataDir) when [created](#creating-instance). 37 | 38 | ### createDataDir { #createDataDir } 39 | 40 | ```scala 41 | createDataDir(): File 42 | ``` 43 | 44 | `createDataDir` [creates the name of the log directory](../log/UnifiedLog.md#logDirName) of the [TopicPartition](#topicPartition). 45 | 46 | `createDataDir` [creates the directory](#createLogDirectory) in the [metadataLogDir](../KafkaConfig.md#metadataLogDir). 47 | 48 | ## Starting Up { #startup } 49 | 50 | ```scala 51 | startup(): Unit 52 | ``` 53 | 54 | ??? warning "Procedure" 55 | `startup` is a procedure (returns `Unit`) so _what happens inside stays inside_ (paraphrasing the [former advertising slogan of Las Vegas, Nevada](https://idioms.thefreedictionary.com/what+happens+in+Vegas+stays+in+Vegas)). 56 | 57 | `startup`...FIXME 58 | 59 | --- 60 | 61 | `startup` is used when: 62 | 63 | * `KafkaServer` is requested to [startup](../broker/KafkaServer.md#startup) 64 | * `SharedServer` is requested to [start](SharedServer.md#start) 65 | * `MetadataShell` is requested to [initializeWithRaftManager](../tools/kafka-metadata-shell/MetadataShell.md#initializeWithRaftManager) (upon [run](../tools/kafka-metadata-shell/MetadataShell.md#run)) 66 | -------------------------------------------------------------------------------- /docs/kraft/MetaProperties.md: -------------------------------------------------------------------------------- 1 | # MetaProperties 2 | 3 | `MetaProperties` is [serialized](#toProperties) to `meta.properties` file in [metadata.log.dir](../KafkaConfig.md#metadata.log.dir) (if specified) or the head of the [logDirs](../KafkaConfig.md#logDirs). 4 | 5 | `MetaProperties` is used to create a [KafkaRaftManager](KafkaRaftManager.md#metaProperties). 6 | 7 | ## Creating Instance 8 | 9 | `MetaProperties` takes the following to be created: 10 | 11 | * Cluster ID 12 | * Node ID 13 | 14 | `MetaProperties` is created when: 15 | 16 | * `KafkaServer` is requested to [startup](../broker/KafkaServer.md#startup) (to create a [KafkaRaftManager](KafkaRaftManager.md#metaProperties)) 17 | * `MetaProperties` is requested to [parse](#parse) 18 | -------------------------------------------------------------------------------- /docs/kraft/QuorumState.md: -------------------------------------------------------------------------------- 1 | # QuorumState 2 | 3 | `QuorumState` is used by [KafkaRaftClient](KafkaRaftClient.md#quorum) to...FIXME 4 | 5 | ## Creating Instance 6 | 7 | `QuorumState` takes the following to be created: 8 | 9 | * Local ID 10 | * [Voters](#voters) 11 | * [controller.quorum.election.timeout.ms](RaftConfig.md#controller.quorum.election.timeout.ms) 12 | * [controller.quorum.fetch.timeout.ms](RaftConfig.md#controller.quorum.fetch.timeout.ms) 13 | * `QuorumStateStore` 14 | * `Time` 15 | * `LogContext` 16 | * `Random` 17 | 18 | `QuorumState` is created when: 19 | 20 | * `KafkaRaftClient` is [created](KafkaRaftClient.md#quorum) 21 | 22 | ### voters 23 | 24 | ```java 25 | Set voters 26 | ``` 27 | 28 | `QuorumState` is given quorum voters (their IDs) when [created](#creating-instance) based on [controller.quorum.voters](RaftConfig.md#controller.quorum.voters) configuration property. 29 | -------------------------------------------------------------------------------- /docs/kraft/QuorumStateStore.md: -------------------------------------------------------------------------------- 1 | # QuorumStateStore 2 | 3 | `QuorumStateStore` is an [abstraction](#contract) of [kraft quorum state stores](#implementations). 4 | 5 | ## Contract (Subset) 6 | 7 | ### readElectionState { #readElectionState } 8 | 9 | ```java 10 | ElectionState readElectionState() 11 | ``` 12 | 13 | Reads (_loads_) the latest election state 14 | 15 | See: 16 | 17 | * [FileBasedStateStore](FileBasedStateStore.md#readElectionState) 18 | 19 | Used when: 20 | 21 | * `QuorumState` is requested to [initialize](QuorumState.md#initialize) 22 | 23 | ## Implementations 24 | 25 | * [FileBasedStateStore](FileBasedStateStore.md) 26 | -------------------------------------------------------------------------------- /docs/kraft/RaftClient.Listener.md: -------------------------------------------------------------------------------- 1 | # RaftClient.Listener 2 | 3 | `RaftClient.Listener` is an [abstraction](#contract) of [listeners](#implementations) that can [handleCommit](#handleCommit) and [handleLoadSnapshot](#handleLoadSnapshot) (_among other metadata-related things_). 4 | 5 | ## Contract (Subset) 6 | 7 | ### handleCommit { #handleCommit } 8 | 9 | ```java 10 | void handleCommit( 11 | BatchReader reader) 12 | ``` 13 | 14 | Used when: 15 | 16 | * `SnapshotFileReader` is requested to `handleMetadataBatch` 17 | * `KafkaRaftClient.ListenerContext` is requested to [fireHandleCommit](KafkaRaftClient.ListenerContext.md#fireHandleCommit) 18 | 19 | ### handleLoadSnapshot { #handleLoadSnapshot } 20 | 21 | ```java 22 | void handleLoadSnapshot( 23 | SnapshotReader reader) 24 | ``` 25 | 26 | Used when: 27 | 28 | * `KafkaRaftClient.ListenerContext` is requested to [fireHandleSnapshot](KafkaRaftClient.ListenerContext.md#fireHandleSnapshot) 29 | 30 | ## Implementations 31 | 32 | * [MetadataLoader](../metadata/MetadataLoader.md) 33 | * `OffsetTrackingListener` 34 | * `QuorumMetaLogListener` 35 | * `ReplicatedCounter` 36 | -------------------------------------------------------------------------------- /docs/kraft/RaftClient.md: -------------------------------------------------------------------------------- 1 | # RaftClient 2 | 3 | `RaftClient` is an [abstraction](#contract) of [Raft clients](#implementations). 4 | 5 | `RaftClient` is `AutoCloseable`. 6 | 7 | ## Contract 8 | 9 | ### initialize { #initialize } 10 | 11 | ```java 12 | void initialize() 13 | ``` 14 | 15 | See: 16 | 17 | * [KafkaRaftClient](KafkaRaftClient.md#initialize) 18 | 19 | Used when: 20 | 21 | * `KafkaRaftManager` is requested to [build a RaftClient](KafkaRaftManager.md#buildRaftClient) 22 | 23 | ## Implementations 24 | 25 | * [KafkaRaftClient](KafkaRaftClient.md) 26 | -------------------------------------------------------------------------------- /docs/kraft/RaftConfig.md: -------------------------------------------------------------------------------- 1 | # RaftConfig 2 | 3 | `RaftConfig` is the configuration of [KafkaRaftManager](KafkaRaftManager.md#raftConfig) in [KRaft mode](index.md). 4 | 5 | ## Creating Instance 6 | 7 | `RaftConfig` takes the following to be created: 8 | 9 | * [controller.quorum.voters](#QUORUM_VOTERS_CONFIG) 10 | * `requestTimeoutMs` 11 | * `retryBackoffMs` 12 | * `electionTimeoutMs` 13 | * `electionBackoffMaxMs` 14 | * `fetchTimeoutMs` 15 | * `appendLingerMs` 16 | 17 | `RaftConfig` is created when: 18 | 19 | * `KafkaRaftManager` is [created](KafkaRaftManager.md#raftConfig) 20 | 21 | ## controller.quorum.voters { #controller.quorum.voters } 22 | 23 | A comma-separated list of `{id}@{host}:{port}` with the node IDs and the endpoints of all the controllers (_quorum voters_) in a Kafka cluster in a KRaft mode (e.g., `1@localhost:9092,2@localhost:9093,3@localhost:9094`) 24 | 25 | Default: (empty) 26 | 27 | Importance: High 28 | 29 | For [ProcessRolesProp](../KafkaConfig.md#ProcessRolesProp) with `controller` role, the [node id](../KafkaConfig.md#nodeId) must also be included in `controller.quorum.voters`. 30 | 31 | For [ProcessRolesProp](../KafkaConfig.md#ProcessRolesProp) with `broker` role only, the [node id](../KafkaConfig.md#nodeId) must not be included in `controller.quorum.voters`. 32 | 33 | Available as [KafkaConfig.quorumVoters](../KafkaConfig.md#quorumVoters) 34 | 35 | Used when: 36 | 37 | * `RaftConfig` is requested to [parseVoterConnections](#parseVoterConnections) 38 | * `KafkaConfig` is requested for [QuorumVotersProp](../KafkaConfig.md#QuorumVotersProp) and [quorumVoters](../KafkaConfig.md#quorumVoters) 39 | 40 | ### parseVoterConnections { #parseVoterConnections } 41 | 42 | ```java 43 | Map parseVoterConnections( 44 | List voterEntries) 45 | ``` 46 | 47 | `parseVoterConnections`...FIXME 48 | 49 | --- 50 | 51 | `parseVoterConnections` is used when: 52 | 53 | * `RaftConfig` is [created](#voterConnections) and requested to [quorumVoterStringsToNodes](#quorumVoterStringsToNodes) 54 | * `KafkaConfig` is requested to [validateValues](../KafkaConfig.md#validateValues) 55 | * `KafkaRaftServer` is [created](KafkaRaftServer.md#controllerQuorumVotersFuture) 56 | * `KafkaServer` is requested to [start up](../broker/KafkaServer.md#startup) 57 | -------------------------------------------------------------------------------- /docs/kraft/RaftIoThread.md: -------------------------------------------------------------------------------- 1 | # RaftIoThread 2 | 3 | `RaftIoThread` is an uninterruptible `ShutdownableThread` with the following name (based on the [threadNamePrefix](#threadNamePrefix)): 4 | 5 | ```text 6 | [threadNamePrefix]-io-thread 7 | ``` 8 | 9 | ## Creating Instance 10 | 11 | `RaftIoThread` takes the following to be created: 12 | 13 | * [KafkaRaftClient](KafkaRaftClient.md) 14 | * Thread Name Prefix 15 | * `FaultHandler` 16 | 17 | `RaftIoThread` is created alongside [KafkaRaftManager](KafkaRaftManager.md#raftIoThread). 18 | 19 | ## doWork { #doWork } 20 | 21 | ??? note "ShutdownableThread" 22 | 23 | ```scala 24 | doWork(): Unit 25 | ``` 26 | 27 | `doWork` is part of the `ShutdownableThread` abstraction. 28 | 29 | `doWork` requests the [KafkaRaftClient](#client) to [poll](KafkaRaftClient.md#poll). 30 | -------------------------------------------------------------------------------- /docs/kraft/RaftManager.md: -------------------------------------------------------------------------------- 1 | # RaftManager 2 | 3 | `RaftManager` is an [abstraction](#contract) of [Raft managers](#implementations). 4 | 5 | ## Contract 6 | 7 | ### client { #client } 8 | 9 | ```scala 10 | client: RaftClient[T] 11 | ``` 12 | 13 | [RaftClient](RaftClient.md) 14 | 15 | See: 16 | 17 | * [KafkaRaftManager](KafkaRaftManager.md#client) 18 | 19 | ### leaderAndEpoch { #leaderAndEpoch } 20 | 21 | ```scala 22 | leaderAndEpoch: LeaderAndEpoch 23 | ``` 24 | 25 | See: 26 | 27 | * [KafkaRaftManager](KafkaRaftManager.md#leaderAndEpoch) 28 | 29 | Used when: 30 | 31 | * `RaftControllerNodeProvider` is requested to [getControllerInfo](../broker/RaftControllerNodeProvider.md#getControllerInfo) 32 | 33 | ## Implementations 34 | 35 | * [KafkaRaftManager](KafkaRaftManager.md) 36 | -------------------------------------------------------------------------------- /docs/kraft/SharedServer.md: -------------------------------------------------------------------------------- 1 | # SharedServer 2 | 3 | ## Creating Instance 4 | 5 | `SharedServer` takes the following to be created: 6 | 7 | * [KafkaConfig](../KafkaConfig.md) 8 | * [MetaProperties](MetaProperties.md) 9 | * `Time` 10 | * [Metrics](../metrics/Metrics.md) 11 | * Controller Quorum Voters (`CompletableFuture[util.Map[Integer, AddressSpec]]`) 12 | * `FaultHandlerFactory` 13 | 14 | `SharedServer` is created alongside [KafkaRaftServer](KafkaRaftServer.md#sharedServer). 15 | 16 | ## SnapshotGenerator { #snapshotGenerator } 17 | 18 | `SharedServer` creates a new [SnapshotGenerator](../metadata/SnapshotGenerator.md) at [start up](#start) as follows: 19 | 20 | Property | Value 21 | ---------|------ 22 | [Emitter](../metadata/SnapshotGenerator.md#emitter) | [SnapshotEmitter](#snapshotEmitter) 23 | [maxBytesSinceLastSnapshot](../metadata/SnapshotGenerator.md#maxBytesSinceLastSnapshot) | [metadata.log.max.record.bytes.between.snapshots](../KafkaConfig.md#metadata.log.max.record.bytes.between.snapshots) 24 | [maxTimeSinceLastSnapshotNs](../metadata/SnapshotGenerator.md#maxTimeSinceLastSnapshotNs) | [metadata.log.max.snapshot.interval.ms](../KafkaConfig.md#metadata.log.max.snapshot.interval.ms) 25 | [threadNamePrefix](../metadata/SnapshotGenerator.md#threadNamePrefix) | `kafka-[nodeId]-` 26 | 27 | The `SnapshotGenerator` is closed (and the reference de-referenced, `null`ed) at [stop](#stop). 28 | 29 | The `SnapshotGenerator` is used at [start up](#start) for the [MetadataLoader](#loader) to [installPublishers](../metadata/MetadataLoader.md#installPublishers). 30 | -------------------------------------------------------------------------------- /docs/log-cleanup/index.md: -------------------------------------------------------------------------------- 1 | # Log Cleanup 2 | 3 | Kafka uses [cleanup.policy](#cleanup.policy) configuration property to apply **cleanup strategies** (policy) to logs: 4 | 5 | * [Log Compaction](#log-compaction) 6 | * [Log Retention](#log-retention) 7 | 8 | ## Configuration Properties 9 | 10 | The cluster-wide [log.cleanup.policy](../KafkaConfig.md#log.cleanup.policy) and the per-topic [cleanup.policy](../TopicConfig.md#cleanup.policy) configuration properties are comma-separated lists of cleanup strategies: 11 | 12 | * `compact` - enables [log compaction](#log-compaction) 13 | * `delete` - enables [log retention](#log-retention) 14 | 15 | Unless defined, [cleanup.policy](../TopicConfig.md#cleanup.policy) is exactly [log.cleanup.policy](../KafkaConfig.md#log.cleanup.policy). 16 | 17 | ## Log Compaction 18 | 19 | **Log Compaction** is a cleanup strategy in which...FIXME 20 | 21 | Kafka brokers use [LogCleaner](../log/LogCleaner.md) for [compact](#compact) retention strategy. 22 | 23 | Log compaction can be [reconfigured dynamically at runtime](../log/CleanerConfig.md). 24 | 25 | ## Log Retention 26 | 27 | **Log Retention** (_Garbage Collection_) is a cleanup strategy to discard (_delete_) old log segments when their [retention time](#time-based-retention) or [size limit](#size-based-retention) has been reached. 28 | 29 | By default there is only a time limit and no size limit. 30 | 31 | Kafka brokers schedule `kafka-log-retention` periodic task for [delete](#delete) retention strategy. 32 | 33 | Kafka uses [log.retention.check.interval.ms](../KafkaConfig.md#log.retention.check.interval.ms) configuration property as the interval between regular log checks. 34 | 35 | ### Time-Based Retention 36 | 37 | **Retention Time** is controlled by the cluster-wide [log.retention.ms](../KafkaConfig.md#log.retention.ms), [log.retention.minutes](../KafkaConfig.md#log.retention.minutes) or [log.retention.hours](../KafkaConfig.md#log.retention.hours) configuration properties (from the highest to the lowest priority) or their per-topic [retention.ms](../TopicConfig.md#retention.ms) configuration property. 38 | 39 | ### Size-Based Retention 40 | 41 | **Retention Size** is controlled by the cluster-wide [log.retention.bytes](../KafkaConfig.md#log.retention.bytes) or per-topic [retention.bytes](../TopicConfig.md#retention.bytes) configuration property. 42 | 43 | ## Logging 44 | 45 | Enable `ALL` logging level for [kafka.log.Log](../Log.md#logging) logger to see messages related to log retention. 46 | -------------------------------------------------------------------------------- /docs/log/CleanerConfig.md: -------------------------------------------------------------------------------- 1 | # CleanerConfig 2 | 3 | `CleanerConfig` contains the configuration parameters of the [LogCleaner](LogCleaner.md). 4 | 5 | Parameter | Configuration Property | Default Value 6 | ----------|------------------------|-------------- 7 | numThreads | [log.cleaner.threads](../KafkaConfig.md#logCleanerThreads) | 1 8 | dedupeBufferSize | [logCleanerDedupeBufferSize](../KafkaConfig.md#logCleanerDedupeBufferSize) | 4*1024*1024L 9 | dedupeBufferLoadFactor | [logCleanerDedupeBufferLoadFactor](../KafkaConfig.md#logCleanerDedupeBufferLoadFactor) | 0.9d 10 | ioBufferSize | [logCleanerIoBufferSize](../KafkaConfig.md#logCleanerIoBufferSize) | 1024*1024 11 | maxMessageSize | [messageMaxBytes](../KafkaConfig.md#messageMaxBytes) | 32*1024*1024 12 | maxIoBytesPerSecond | [logCleanerIoMaxBytesPerSecond](../KafkaConfig.md#logCleanerIoMaxBytesPerSecond) | Double.MaxValue 13 | backOffMs | [log.cleaner.backoff.ms](../KafkaConfig.md#logCleanerBackoffMs) | 15*1000 14 | enableCleaner | [log.cleaner.enable](../KafkaConfig.md#logCleanerEnable) | true 15 | hashAlgorithm | | MD5 16 | -------------------------------------------------------------------------------- /docs/log/CleanerThread.md: -------------------------------------------------------------------------------- 1 | # CleanerThread 2 | 3 | `CleanerThread` is a non-daemon thread of execution for [LogCleaner](LogCleaner.md#cleaners) for [log cleanup](#doWork) (one at a time until no more is left). 4 | 5 | ## Creating Instance 6 | 7 | `CleanerThread` takes the following to be created: 8 | 9 | * Thread ID 10 | 11 | `CleanerThread` is created when: 12 | 13 | * `LogCleaner` is requested to [start up](#startup) 14 | 15 | ## Thread Name 16 | 17 | `CleanerThread` uses the following as the thread name (with the [threadId](#threadId)) 18 | 19 | ```text 20 | kafka-log-cleaner-thread-[threadId] 21 | ``` 22 | 23 | ## log.cleaner.threads 24 | 25 | The number of `CleanerThreads` (that [LogCleaner](LogCleaner.md#cleaners) uses) is controlled by [log.cleaner.threads](../KafkaConfig.md#log.cleaner.threads) dynamic configuration. 26 | 27 | ## doWork 28 | 29 | ```scala 30 | doWork(): Unit 31 | ``` 32 | 33 | `doWork` is part of the `ShutdownableThread` abstraction. 34 | 35 | --- 36 | 37 | `doWork` [tryCleanFilthiestLog](#tryCleanFilthiestLog). If no logs was cleaned up, `doWork` pauses the thread for [log.cleaner.backoff.ms](../KafkaConfig.md#log.cleaner.backoff.ms) millis. 38 | 39 | In the end, `doWork` requests the [LogCleanerManager](LogCleaner.md#cleanerManager) to [maintainUncleanablePartitions](LogCleanerManager.md#maintainUncleanablePartitions). 40 | 41 | ### tryCleanFilthiestLog 42 | 43 | ```scala 44 | tryCleanFilthiestLog(): Boolean 45 | ``` 46 | 47 | `tryCleanFilthiestLog` [cleanFilthiestLog](#cleanFilthiestLog). 48 | 49 | #### LogCleaningException 50 | 51 | In case of `LogCleaningException`, `tryCleanFilthiestLog` prints out the following WARN message to the logs: 52 | 53 | ```text 54 | Unexpected exception thrown when cleaning log [log]. 55 | Marking its partition ([topicPartition]) as uncleanable 56 | ``` 57 | 58 | `doWork` requests the [LogCleanerManager](LogCleaner.md#cleanerManager) to [maintainUncleanablePartitions](LogCleanerManager.md#maintainUncleanablePartitions) and returns `false`. 59 | 60 | ### Cleaning Filthiest Log 61 | 62 | ```scala 63 | cleanFilthiestLog(): Boolean 64 | ``` 65 | 66 | `cleanFilthiestLog` returns `cleaned` flag to indicate whether a log to clean was found or not. 67 | 68 | --- 69 | 70 | `cleanFilthiestLog` requests the [LogCleanerManager](LogCleaner.md#cleanerManager) to [grabFilthiestCompactedLog](LogCleanerManager.md#grabFilthiestCompactedLog). 71 | 72 | If there is no log to clean up, `cleanFilthiestLog` "returns" `false` (as `cleaned` flag). Otherwise, `cleanFilthiestLog` [cleanLog](#cleanLog) and "returns" `true`. 73 | 74 | `cleanFilthiestLog` requests the [LogCleanerManager](LogCleaner.md#cleanerManager) for [deletableLogs](LogCleanerManager.md#deletableLogs) and then every [UnifiedLog](UnifiedLog.md) to [deleteOldSegments](UnifiedLog.md#deleteOldSegments). 75 | 76 | In the end, `cleanFilthiestLog` requests the [LogCleanerManager](LogCleaner.md#cleanerManager) to [doneDeleting](LogCleanerManager.md#doneDeleting) (with the `TopicPartition`s). 77 | 78 | ## Logging 79 | 80 | `CleanerThread` uses [kafka.log.LogCleaner](LogCleaner.md#logging) logger. 81 | -------------------------------------------------------------------------------- /docs/log/LocalLog.md: -------------------------------------------------------------------------------- 1 | # LocalLog 2 | 3 | ## Creating Instance 4 | 5 | `LocalLog` takes the following to be created: 6 | 7 | * Directory 8 | * [LogConfig](LogConfig.md) 9 | * `LogSegments` 10 | * Recovery Point 11 | * `LogOffsetMetadata` 12 | * `Scheduler` 13 | * `Time` 14 | * `TopicPartition` 15 | * `LogDirFailureChannel` 16 | 17 | `LocalLog` is created when: 18 | 19 | * `UnifiedLog` is requested to [create a UnifiedLog](UnifiedLog.md#apply) 20 | 21 | ## truncateFullyAndStartAt { #truncateFullyAndStartAt } 22 | 23 | ```scala 24 | truncateFullyAndStartAt( 25 | newOffset: Long): Iterable[LogSegment] 26 | ``` 27 | 28 | `truncateFullyAndStartAt`...FIXME 29 | 30 | --- 31 | 32 | `truncateFullyAndStartAt` is used when: 33 | 34 | * `UnifiedLog` is requested to [truncateFullyAndStartAt](UnifiedLog.md#truncateFullyAndStartAt) 35 | 36 | ## roll { #roll } 37 | 38 | ```scala 39 | roll( 40 | expectedNextOffset: Option[Long] = None): LogSegment 41 | ``` 42 | 43 | `roll`...FIXME 44 | 45 | --- 46 | 47 | `roll` is used when: 48 | 49 | * `UnifiedLog` is requested to [roll](UnifiedLog.md#roll) 50 | 51 | ## splitOverflowedSegment { #splitOverflowedSegment } 52 | 53 | ```scala 54 | splitOverflowedSegment( 55 | segment: LogSegment, 56 | existingSegments: LogSegments, 57 | dir: File, 58 | topicPartition: TopicPartition, 59 | config: LogConfig, 60 | scheduler: Scheduler, 61 | logDirFailureChannel: LogDirFailureChannel, 62 | logPrefix: String): SplitSegmentResult 63 | ``` 64 | 65 | `splitOverflowedSegment`...FIXME 66 | 67 | --- 68 | 69 | `splitOverflowedSegment` is used when: 70 | 71 | * `UnifiedLog` is requested to [splitOverflowedSegment](UnifiedLog.md#splitOverflowedSegment) 72 | 73 | ## createNewCleanedSegment { #createNewCleanedSegment } 74 | 75 | ```scala 76 | createNewCleanedSegment( 77 | dir: File, 78 | logConfig: LogConfig, 79 | baseOffset: Long): LogSegment 80 | ``` 81 | 82 | `createNewCleanedSegment`...FIXME 83 | 84 | --- 85 | 86 | `createNewCleanedSegment` is used when: 87 | 88 | * `LocalLog` is requested to [splitOverflowedSegment](#splitOverflowedSegment) 89 | * `UnifiedLog` is requested to [createNewCleanedSegment](UnifiedLog.md#createNewCleanedSegment) 90 | 91 | ## createAndDeleteSegment { #createAndDeleteSegment } 92 | 93 | ```scala 94 | createAndDeleteSegment( 95 | newOffset: Long, 96 | segmentToDelete: LogSegment, 97 | asyncDelete: Boolean, 98 | reason: SegmentDeletionReason): LogSegment 99 | ``` 100 | 101 | `createAndDeleteSegment`...FIXME 102 | 103 | --- 104 | 105 | `createAndDeleteSegment` is used when: 106 | 107 | * `LocalLog` is requested to [roll](#roll), [truncateFullyAndStartAt](#truncateFullyAndStartAt) 108 | -------------------------------------------------------------------------------- /docs/log/LogCleanerManager.md: -------------------------------------------------------------------------------- 1 | # LogCleanerManager 2 | 3 | `LogCleanerManager` is used exclusively by [LogCleaner](LogCleaner.md#cleanerManager) to manage the state of partitions for [Log Cleanup](../log-cleanup/index.md). 4 | 5 | ## Creating Instance 6 | 7 | `LogCleanerManager` takes the following to be created: 8 | 9 | * Log directories 10 | * `TopicPartition`s and their [UnifiedLog](UnifiedLog.md)s (`Pool[TopicPartition, UnifiedLog]`) 11 | * `LogDirFailureChannel` 12 | 13 | `LogCleanerManager` is created along with a [LogCleaner](LogCleaner.md#cleanerManager). 14 | 15 | ## Performance Metrics 16 | 17 | `LogCleanerManager` is a [KafkaMetricsGroup](../metrics/KafkaMetricsGroup.md) and registers the following performance metrics in **kafka.log:type=LogCleanerManager** group. 18 | 19 | Metric Name | Description 20 | ------------|------------ 21 | max-dirty-percent | 22 | time-since-last-run-ms | 23 | uncleanable-bytes | (for every [log directory](#logDirs)) 24 | uncleanable-partitions-count | (for every [log directory](#logDirs)) 25 | 26 | ![LogCleanerManager in jconsole](../images/LogCleanerManager-jconsole.png) 27 | 28 | ## grabFilthiestCompactedLog 29 | 30 | ```scala 31 | grabFilthiestCompactedLog( 32 | time: Time, 33 | preCleanStats: PreCleanStats = new PreCleanStats()): Option[LogToClean] 34 | ``` 35 | 36 | `grabFilthiestCompactedLog`...FIXME 37 | 38 | --- 39 | 40 | `grabFilthiestCompactedLog` is used when: 41 | 42 | * `CleanerThread` is requested to [clean the filthiest log](CleanerThread.md#cleanFilthiestLog) 43 | 44 | ## isCompactAndDelete 45 | 46 | ```scala 47 | isCompactAndDelete( 48 | log: UnifiedLog): Boolean 49 | ``` 50 | 51 | `isCompactAndDelete` holds true when the given [UnifiedLog](UnifiedLog.md) is [compact](LogConfig.md#compact) and [delete](LogConfig.md#delete). 52 | 53 | --- 54 | 55 | `isCompactAndDelete` holds true only when both [compact](../log-cleanup/index.md#compact) and [delete](../log-cleanup/index.md#delete) cleanup policies are included in the [cleanup.policy](../log-cleanup/index.md#cleanup.policy) configuration property. Otherwise, `isCompactAndDelete` flag is disabled (`false`). 56 | 57 | ## Logging 58 | 59 | `LogCleanerManager` uses [kafka.log.LogCleaner](LogCleaner.md#logging) logger. 60 | -------------------------------------------------------------------------------- /docs/log/LogLoader.md: -------------------------------------------------------------------------------- 1 | # LogLoader 2 | 3 | ## Creating Instance 4 | 5 | `LogLoader` takes the following to be created: 6 | 7 | * Directory 8 | * `TopicPartition` 9 | * [LogConfig](LogConfig.md) 10 | * `Scheduler` 11 | * `Time` 12 | * `LogDirFailureChannel` 13 | * `hadCleanShutdown` flag 14 | * `LogSegments` 15 | * `logStartOffsetCheckpoint` 16 | * `recoveryPointCheckpoint` 17 | * `LeaderEpochFileCache` 18 | * `ProducerStateManager` 19 | * `numRemainingSegments` 20 | * `isRemoteLogEnabled` flag (default: `false`) 21 | 22 | `LogLoader` is created when: 23 | 24 | * `UnifiedLog` is requested to [create a UnifiedLog](UnifiedLog.md#apply) 25 | 26 | ## Loading { #load } 27 | 28 | ```scala 29 | load(): LoadedLogOffsets 30 | ``` 31 | 32 | `load`...FIXME 33 | 34 | --- 35 | 36 | `load` is used when: 37 | 38 | * `UnifiedLog` is requested to [create a UnifiedLog](UnifiedLog.md#apply) 39 | -------------------------------------------------------------------------------- /docs/log/LogSegment.md: -------------------------------------------------------------------------------- 1 | # LogSegment 2 | 3 | ## Creating Instance 4 | 5 | `LogSegment` takes the following to be created: 6 | 7 | * `FileRecords` 8 | * `OffsetIndex` 9 | * `TimeIndex` 10 | * `TransactionIndex` 11 | * baseOffset 12 | * indexIntervalBytes 13 | * rollJitterMs 14 | * `Time` 15 | 16 | `LogSegment` is created using [open](#open) utility. 17 | 18 | ## Opening LogSegment { #open } 19 | 20 | ```scala 21 | open( 22 | dir: File, 23 | baseOffset: Long, 24 | config: LogConfig, 25 | time: Time, 26 | fileAlreadyExists: Boolean = false, 27 | initFileSize: Int = 0, 28 | preallocate: Boolean = false, 29 | fileSuffix: String = ""): LogSegment 30 | ``` 31 | 32 | `open`...FIXME 33 | 34 | --- 35 | 36 | `open` is used when: 37 | 38 | * `LocalLog` is requested to [createAndDeleteSegment](LocalLog.md#createAndDeleteSegment), [roll](LocalLog.md#roll), [createNewCleanedSegment](LocalLog.md#createNewCleanedSegment) 39 | * `LogLoader` is requested to [load](LogLoader.md#load), [loadSegmentFiles](LogLoader.md#loadSegmentFiles), [recoverLog](LogLoader.md#recoverLog) 40 | 41 | ## Reading Messages { #read } 42 | 43 | ```scala 44 | read( 45 | startOffset: Long, 46 | maxSize: Int, 47 | maxPosition: Long = size, 48 | minOneMessage: Boolean = false): FetchDataInfo 49 | ``` 50 | 51 | `read`...FIXME 52 | 53 | --- 54 | 55 | `read` is used when: 56 | 57 | * `Log` is requested to [read](../Log.md#read) 58 | * `LogSegment` is requested to [readNextOffset](#readNextOffset) 59 | -------------------------------------------------------------------------------- /docs/log/UnifiedLog.md: -------------------------------------------------------------------------------- 1 | # UnifiedLog 2 | 3 | ## Creating Instance 4 | 5 | `UnifiedLog` takes the following to be created: 6 | 7 | * `logStartOffset` 8 | * [LocalLog](LocalLog.md) 9 | * `BrokerTopicStats` 10 | * `producerIdExpirationCheckIntervalMs` 11 | * `LeaderEpochFileCache` 12 | * `ProducerStateManager` 13 | * Topic ID 14 | * `keepPartitionMetadataFile` 15 | * `remoteStorageSystemEnable` (default: `false`) 16 | * `LogOffsetsListener` 17 | 18 | `UnifiedLog` is created using [apply](#apply) utility. 19 | 20 | ## Creating UnifiedLog { #apply } 21 | 22 | ```scala 23 | apply( 24 | dir: File, 25 | config: LogConfig, 26 | logStartOffset: Long, 27 | recoveryPoint: Long, 28 | scheduler: Scheduler, 29 | brokerTopicStats: BrokerTopicStats, 30 | time: Time, 31 | maxTransactionTimeoutMs: Int, 32 | producerStateManagerConfig: ProducerStateManagerConfig, 33 | producerIdExpirationCheckIntervalMs: Int, 34 | logDirFailureChannel: LogDirFailureChannel, 35 | lastShutdownClean: Boolean = true, 36 | topicId: Option[Uuid], 37 | keepPartitionMetadataFile: Boolean, 38 | numRemainingSegments: ConcurrentMap[String, Int] = new ConcurrentHashMap[String, Int], 39 | remoteStorageSystemEnable: Boolean = false, 40 | logOffsetsListener: LogOffsetsListener = LogOffsetsListener.NO_OP_OFFSETS_LISTENER): UnifiedLog 41 | ``` 42 | 43 | `apply`...FIXME 44 | 45 | --- 46 | 47 | `apply` is used when: 48 | 49 | * `LogManager` is requested to [startup](LogManager.md#startup) (and [loadLog](LogManager.md#loadLog)), [getOrCreateLog](LogManager.md#getOrCreateLog) 50 | * `KafkaMetadataLog` is requested to [create a KafkaMetadataLog](../kraft/KafkaMetadataLog.md#apply) 51 | -------------------------------------------------------------------------------- /docs/log/index.md: -------------------------------------------------------------------------------- 1 | # LogManager 2 | 3 | [LogManager](LogManager.md) 4 | -------------------------------------------------------------------------------- /docs/logging.md: -------------------------------------------------------------------------------- 1 | # Logging 2 | 3 | ## Brokers 4 | 5 | Kafka brokers use [Apache Log4j 2](https://logging.apache.org/log4j/2.x/) for logging and use `config/log4j.properties` by default. 6 | 7 | The default logging level is `INFO` with `stdout` appender. 8 | 9 | ```text 10 | log4j.rootLogger=INFO, stdout, kafkaAppender 11 | 12 | log4j.logger.kafka=INFO 13 | log4j.logger.org.apache.kafka=INFO 14 | ``` 15 | 16 | ## Tools 17 | 18 | Kafka tools (e.g. `kafka-console-producer`) use `config/tools-log4j.properties` as the logging configuration file. 19 | 20 | ## Clients 21 | 22 | ### build.sbt 23 | 24 | ```text 25 | libraryDependencies += "org.apache.kafka" % "kafka-clients" % "2.8.0" 26 | 27 | val slf4j = "1.7.32" 28 | libraryDependencies += "org.slf4j" % "slf4j-api" % slf4j 29 | libraryDependencies += "org.slf4j" % "slf4j-log4j12" % slf4j 30 | ``` 31 | 32 | ### log4j.properties 33 | 34 | In `src/main/resources/log4j.properties` use the following: 35 | 36 | ```text 37 | log4j.rootLogger=INFO, stdout 38 | 39 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 40 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 41 | log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n 42 | 43 | log4j.logger.org.apache.kafka.clients.producer.KafkaProducer=ALL 44 | ``` 45 | -------------------------------------------------------------------------------- /docs/metadata/.pages: -------------------------------------------------------------------------------- 1 | title: Metadata 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/metadata/BrokerMetadataPublisher.md: -------------------------------------------------------------------------------- 1 | # BrokerMetadataPublisher 2 | 3 | `BrokerMetadataPublisher` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/metadata/LoaderManifest.md: -------------------------------------------------------------------------------- 1 | # LoaderManifest 2 | 3 | `LoaderManifest` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/metadata/MetadataLoader.md: -------------------------------------------------------------------------------- 1 | # MetadataLoader 2 | 3 | `MetadataLoader` is built for a [SharedServer](../kraft/SharedServer.md#loader). 4 | 5 | ## Creating Instance 6 | 7 | `MetadataLoader` takes the following to be created: 8 | 9 | * `Time` 10 | * `LogContext` 11 | * Node ID (_not used_) 12 | * Thread Name Prefix 13 | * `FaultHandler` 14 | * `MetadataLoaderMetrics` 15 | * [HighWaterMarkAccessor](#highWaterMarkAccessor) 16 | 17 | `MetadataLoader` is created when: 18 | 19 | * `MetadataLoader.Builder` is requested to [build a MetadataLoader](#build) 20 | 21 | ### HighWaterMarkAccessor { #highWaterMarkAccessor } 22 | 23 | `MetadataLoader` is given a high watermark accessor (a `Supplier`) when [created](#creating-instance) (when `SharedServer` is requested to [start](../kraft/SharedServer.md#start)). 24 | 25 | The high watermark accessor is the [highWatermark](../kraft/KafkaRaftClient.md#highWatermark) from the [KafkaRaftClient](../kraft/KafkaRaftManager.md#client) of a [KafkaRaftManager](../kraft/KafkaRaftManager.md). 26 | 27 | ## Installing MetadataPublishers { #installPublishers } 28 | 29 | ```java 30 | CompletableFuture installPublishers( 31 | List newPublishers) 32 | ``` 33 | 34 | `installPublishers` adds all the [MetadataPublisher](MetadataPublisher.md)s (in the given `newPublishers`) to the [uninitializedPublishers](#uninitializedPublishers) registry followed by [scheduleInitializeNewPublishers](#scheduleInitializeNewPublishers). 35 | 36 | --- 37 | 38 | `installPublishers` is used when: 39 | 40 | * `MetadataShell` is requested to [run](../tools/kafka-metadata-shell/MetadataShell.md#run) (to register a [MetadataShellPublisher](../tools/kafka-metadata-shell/MetadataShellPublisher.md)) 41 | * `BrokerServer` is requested to [startup](../kraft/BrokerServer.md#startup) 42 | * `ControllerServer` is requested to [startup](../kraft/ControllerServer.md#startup) 43 | * `SharedServer` is requested to [start](../kraft/SharedServer.md#start) (to register a [SnapshotGenerator](SnapshotGenerator.md)) 44 | 45 | ## scheduleInitializeNewPublishers { #scheduleInitializeNewPublishers } 46 | 47 | ```java 48 | void scheduleInitializeNewPublishers( 49 | long delayNs) 50 | ``` 51 | 52 | `scheduleInitializeNewPublishers`...FIXME 53 | 54 | --- 55 | 56 | `scheduleInitializeNewPublishers` is used when: 57 | 58 | * `MetadataLoader` is requested to [initializeNewPublishers](#initializeNewPublishers), [maybePublishMetadata](#maybePublishMetadata) and [installPublishers](#installPublishers) 59 | 60 | ### initializeNewPublishers { #initializeNewPublishers } 61 | 62 | ```java 63 | void initializeNewPublishers() 64 | ``` 65 | 66 | `initializeNewPublishers`...FIXME 67 | 68 | ## maybePublishMetadata { #maybePublishMetadata } 69 | 70 | ```java 71 | void maybePublishMetadata( 72 | MetadataDelta delta, 73 | MetadataImage image, 74 | LoaderManifest manifest) 75 | ``` 76 | 77 | `maybePublishMetadata`...FIXME 78 | 79 | --- 80 | 81 | `maybePublishMetadata` is used when: 82 | 83 | * `MetadataLoader` is [created](#batchLoader) and requested to [handleLoadSnapshot](#handleLoadSnapshot) 84 | 85 | ## handleLoadSnapshot { #handleLoadSnapshot } 86 | 87 | ??? note "RaftClient.Listener" 88 | 89 | ```java 90 | void handleLoadSnapshot( 91 | SnapshotReader reader) 92 | ``` 93 | 94 | `handleLoadSnapshot` is part of the [RaftClient.Listener](../kraft/RaftClient.Listener.md#handleLoadSnapshot) abstraction. 95 | 96 | `handleLoadSnapshot`...FIXME 97 | 98 | ### loadSnapshot { #loadSnapshot } 99 | 100 | ```java 101 | SnapshotManifest loadSnapshot( 102 | MetadataDelta delta, 103 | SnapshotReader reader) 104 | ``` 105 | 106 | `loadSnapshot`...FIXME 107 | -------------------------------------------------------------------------------- /docs/metadata/MetadataLoaderMetrics.md: -------------------------------------------------------------------------------- 1 | # MetadataLoaderMetrics 2 | 3 | ## Creating Instance 4 | 5 | `MetadataLoaderMetrics` takes the following to be created: 6 | 7 | * `MetricsRegistry` 8 | * `batchProcessingTimeNsUpdater` 9 | * `batchSizesUpdater` 10 | * Last Applied `MetadataProvenance` 11 | 12 | `MetadataLoaderMetrics` is created when: 13 | 14 | * `SharedServer` is requested to [start](../kraft/SharedServer.md#start) 15 | * `MetadataLoader.Builder` is requested to `build` 16 | -------------------------------------------------------------------------------- /docs/metadata/MetadataPublisher.md: -------------------------------------------------------------------------------- 1 | # MetadataPublisher 2 | 3 | `MetadataPublisher` is an [abstraction](#contract) of [metadata publishers](#implementations). 4 | 5 | ## Contract (Subset) 6 | 7 | ### onMetadataUpdate { #onMetadataUpdate } 8 | 9 | ```java 10 | void onMetadataUpdate( 11 | MetadataDelta delta, 12 | MetadataImage newImage, 13 | LoaderManifest manifest) 14 | ``` 15 | 16 | See: 17 | 18 | * [AclPublisher](../authorization/AclPublisher.md#onMetadataUpdate) 19 | * [BrokerMetadataPublisher](BrokerMetadataPublisher.md#onMetadataUpdate) 20 | 21 | Used when: 22 | 23 | * `MetadataLoader` is requested to [initializeNewPublishers](MetadataLoader.md#initializeNewPublishers) and [maybePublishMetadata](MetadataLoader.md#maybePublishMetadata) 24 | * `BrokerMetadataPublisher` is requested to [onMetadataUpdate](BrokerMetadataPublisher.md#onMetadataUpdate) 25 | 26 | ## Implementations 27 | 28 | * [AclPublisher](../authorization/AclPublisher.md) 29 | * [BrokerMetadataPublisher](BrokerMetadataPublisher.md) 30 | * `ControllerMetadataMetricsPublisher` 31 | * `DelegationTokenPublisher` 32 | * `DynamicClientQuotaPublisher` 33 | * `DynamicConfigPublisher` 34 | * `FeaturesPublisher` 35 | * `KRaftMigrationDriver` 36 | * [MetadataShellPublisher](../tools/kafka-metadata-shell/MetadataShellPublisher.md) 37 | * `ScramPublisher` 38 | * [SnapshotGenerator](SnapshotGenerator.md) 39 | -------------------------------------------------------------------------------- /docs/metadata/QuorumController.md: -------------------------------------------------------------------------------- 1 | # QuorumController 2 | 3 | ## leaderImbalanceCheckIntervalNs { #leaderImbalanceCheckIntervalNs } 4 | 5 | `QuorumController` is given`leaderImbalanceCheckIntervalNs` when created. 6 | 7 | The interval is used to [maybeScheduleNextBalancePartitionLeaders](#maybeScheduleNextBalancePartitionLeaders). 8 | 9 | ## maybeScheduleNextBalancePartitionLeaders { #maybeScheduleNextBalancePartitionLeaders } 10 | 11 | ```java 12 | void maybeScheduleNextBalancePartitionLeaders() 13 | ``` 14 | 15 | `maybeScheduleNextBalancePartitionLeaders`...FIXME 16 | 17 | --- 18 | 19 | `maybeScheduleNextBalancePartitionLeaders` is used when: 20 | 21 | * `CompleteActivationEvent` is requested to `processBatchEndOffset` 22 | * `ControllerWriteEvent` is requested to `run` 23 | -------------------------------------------------------------------------------- /docs/metadata/SnapshotGenerator.md: -------------------------------------------------------------------------------- 1 | # SnapshotGenerator 2 | 3 | `SnapshotGenerator` is a [MetadataPublisher](MetadataPublisher.md). 4 | 5 | ## Creating Instance 6 | 7 | `SnapshotGenerator` takes the following to be created: 8 | 9 | * Node ID 10 | * `Time` 11 | * `Emitter` 12 | * `FaultHandler` 13 | * [metadata.log.max.record.bytes.between.snapshots](../KafkaConfig.md#metadata.log.max.record.bytes.between.snapshots) 14 | * [metadata.log.max.snapshot.interval.ms](../KafkaConfig.md#metadata.log.max.snapshot.interval.ms) 15 | * `disabledReason` 16 | * Thread Name Prefix 17 | 18 | `SnapshotGenerator` is created when: 19 | 20 | * `SnapshotGenerator.Builder` is requested to `build` (when `SharedServer` is requested to [start](../kraft/SharedServer.md#snapshotGenerator)) 21 | 22 | ## onMetadataUpdate { #onMetadataUpdate } 23 | 24 | ??? note "MetadataPublisher" 25 | 26 | ```java 27 | void onMetadataUpdate( 28 | MetadataDelta delta, 29 | MetadataImage newImage, 30 | LoaderManifest manifest) 31 | ``` 32 | 33 | `onMetadataUpdate` is part of the [MetadataPublisher](MetadataPublisher.md#onMetadataUpdate) abstraction. 34 | 35 | `onMetadataUpdate`...FIXME 36 | -------------------------------------------------------------------------------- /docs/metadata/SnapshotManifest.md: -------------------------------------------------------------------------------- 1 | # SnapshotManifest 2 | 3 | `SnapshotManifest` is a [LoaderManifest](LoaderManifest.md). 4 | 5 | ## Creating Instance 6 | 7 | `SnapshotManifest` takes the following to be created: 8 | 9 | * `MetadataProvenance` 10 | * `elapsedNs` 11 | 12 | `SnapshotManifest` is created when: 13 | 14 | * `MetadataLoader` is requested to [initializeNewPublishers](MetadataLoader.md#initializeNewPublishers) and [loadSnapshot](MetadataLoader.md#loadSnapshot) 15 | 16 | ## LoaderManifestType { #type } 17 | 18 | ??? note "PARENT" 19 | 20 | ```java 21 | LoaderManifestType type() 22 | ``` 23 | 24 | `type` is part of the [LoaderManifest](LoaderManifest.md#type) abstraction. 25 | 26 | `type` is `SNAPSHOT`. 27 | -------------------------------------------------------------------------------- /docs/metadata/index.md: -------------------------------------------------------------------------------- 1 | # Kafka Metadata 2 | -------------------------------------------------------------------------------- /docs/metrics/.pages: -------------------------------------------------------------------------------- 1 | title: Metrics 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/metrics/KafkaMetricsGroup.md: -------------------------------------------------------------------------------- 1 | # KafkaMetricsGroup 2 | 3 | `KafkaMetricsGroup` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/metrics/MetricConfig.md: -------------------------------------------------------------------------------- 1 | # MetricConfig 2 | 3 | ## Creating Instance 4 | 5 | `MetricConfig` takes no arguments to be created. 6 | 7 | `MetricConfig` is created when: 8 | 9 | * _many places_ (FIXME) 10 | 11 | ## RecordingLevel 12 | 13 | `MetricConfig` uses `INFO` recording level by default (when [created](#creating-instance)) that can be changed using [recordLevel](#recordLevel). 14 | 15 | ### recordLevel 16 | 17 | ```java 18 | MetricConfig recordLevel( 19 | Sensor.RecordingLevel recordingLevel) 20 | ``` 21 | 22 | `recordLevel` sets the [recordingLevel](#recordingLevel) to the given `RecordingLevel`. 23 | 24 | `recordLevel` is used when: 25 | 26 | * `KafkaAdminClient` is requested to `createInternal` 27 | * `KafkaConsumer` is requested to [buildMetrics](../clients/consumer/KafkaConsumer.md#buildMetrics) 28 | * `KafkaProducer` is [created](../clients/producer/KafkaProducer.md#metricConfig) 29 | * `KafkaStreams` ([Kafka Streams]({{ book.kafka_streams }}/KafkaStreams#getMetrics)) is requested to `getMetrics` 30 | * `StreamsMetricsImpl` ([Kafka Streams]({{ book.kafka_streams }}/metrics/StreamsMetricsImpl)) is requested to `addClientLevelImmutableMetric`, `addClientLevelMutableMetric`, `addStoreLevelMutableMetric` 31 | * `Server` utility is used to [buildMetricsConfig](../Server.md#buildMetricsConfig) 32 | -------------------------------------------------------------------------------- /docs/metrics/Metrics.md: -------------------------------------------------------------------------------- 1 | # Metrics 2 | 3 | `Metrics` is a registry of sensors and performance metrics (of Kafka brokers and clients). 4 | 5 | ## Creating Instance 6 | 7 | `Metrics` takes the following to be created: 8 | 9 | * [MetricConfig](MetricConfig.md) 10 | * `MetricsReporter`s 11 | * `Time` 12 | * `enableExpiration` flag 13 | * `MetricsContext` 14 | 15 | `Metrics` is created when: 16 | 17 | * `Server` utility is used to [buildMetrics](../Server.md#buildMetrics) 18 | * `KafkaAdminClient` utility is used to `createInternal` 19 | * `KafkaConsumer` utility is used to [buildMetrics](../clients/consumer/KafkaConsumer.md#buildMetrics) 20 | * `KafkaProducer` is [created](../clients/producer/KafkaProducer.md#metrics) 21 | * `KafkaStreams` ([Kafka Streams]({{ book.kafka_streams }}/KafkaStreams#getMetrics)) utility is used to `getMetrics` 22 | * Kafka Connect clients 23 | 24 | ## addReporter 25 | 26 | ```java 27 | void addReporter( 28 | MetricsReporter reporter) 29 | ``` 30 | 31 | `addReporter`...FIXME 32 | 33 | `addReporter` is used when: 34 | 35 | * `DynamicMetricsReporters` is requested to `createReporters` 36 | 37 | ## sensors 38 | 39 | `Metrics` defines `sensors` collection of metric `Sensor`s by name (`ConcurrentMap`). 40 | 41 | `sensors` is empty when `Metrics` is [created](#creating-instance). 42 | 43 | A new `Sensor` is added in [sensor](#sensor). 44 | 45 | ### sensor 46 | 47 | ```java 48 | Sensor sensor( 49 | String name, 50 | MetricConfig config, 51 | long inactiveSensorExpirationTimeSeconds, 52 | Sensor.RecordingLevel recordingLevel, 53 | Sensor... parents) 54 | Sensor sensor(...) // (1) 55 | ``` 56 | 57 | 1. There are others 58 | 59 | `sensor` [looks up the sensor (by name)](#getSensor) and returns it immediately if available. 60 | 61 | Otherwise, `sensor` creates a new `Sensor` and adds it to the [sensors](#sensors) registry. 62 | 63 | In the end, `sensor` prints out the following TRACE message to the logs: 64 | 65 | ```text 66 | Added sensor with name [name] 67 | ``` 68 | 69 | ### getSensor 70 | 71 | ```java 72 | Sensor getSensor( 73 | String name) 74 | ``` 75 | 76 | `getSensor` looks up the given `name` in the [sensors](#sensors) registry. 77 | 78 | ## Logging 79 | 80 | Enable `ALL` logging level for `org.apache.kafka.common.metrics.Metrics` logger to see what happens inside. 81 | 82 | Add the following line to `config/log4j.properties`: 83 | 84 | ```text 85 | log4j.logger.org.apache.kafka.common.metrics.Metrics=ALL 86 | ``` 87 | 88 | Refer to [Logging](../logging.md). 89 | -------------------------------------------------------------------------------- /docs/metrics/MetricsReporter.md: -------------------------------------------------------------------------------- 1 | # MetricsReporter 2 | 3 | `MetricsReporter` is an [extension](#contract) of the [Reconfigurable](../dynamic-configuration/Reconfigurable.md) abstraction for [reconfigurable metrics reporters](#implementations). 4 | 5 | ## Contract (Subset) 6 | 7 | ### init 8 | 9 | ```java 10 | void init( 11 | List metrics) 12 | ``` 13 | 14 | Used when: 15 | 16 | * `Metrics` is [created](Metrics.md#creating-instance) and requested to [addReporter](Metrics.md#addReporter) 17 | 18 | ## Implementations 19 | 20 | * `JmxReporter` 21 | * `PushHttpMetricsReporter` 22 | -------------------------------------------------------------------------------- /docs/metrics/Sensor.md: -------------------------------------------------------------------------------- 1 | # Sensor 2 | 3 | ## Creating Instance 4 | 5 | `Sensor` takes the following to be created: 6 | 7 | * [Metrics](Metrics.md) 8 | * Name 9 | * Parent `Sensor`s 10 | * [MetricConfig](MetricConfig.md) 11 | * `Time` 12 | * `inactiveSensorExpirationTimeSeconds` 13 | * `RecordingLevel` 14 | 15 | `Sensor` is created when: 16 | 17 | * `Metrics` is requested for a [sensor](Metrics.md#sensor) 18 | 19 | ## shouldRecord 20 | 21 | ```java 22 | boolean shouldRecord() 23 | ``` 24 | 25 | `shouldRecord` requests the [RecordingLevel](#recordingLevel) to `shouldRecord` based on the configured [RecordingLevel](MetricConfig.md#recordLevel) (in the [MetricConfig](#config)). 26 | -------------------------------------------------------------------------------- /docs/metrics/index.md: -------------------------------------------------------------------------------- 1 | # Metrics 2 | -------------------------------------------------------------------------------- /docs/overview.md: -------------------------------------------------------------------------------- 1 | # {{ book.title }} 2 | 3 | [Apache Kafka](https://kafka.apache.org/) is an open source project for a distributed publish-subscribe messaging system rethought as a **distributed commit log**. 4 | 5 | ## Messages 6 | 7 | Messages (_records_, _events_) are byte arrays (String, JSON, and Avro are among the most common formats). 8 | If a message has a key, Kafka (uses [Partitioner](clients/producer/Partitioner.md)) to make sure that all messages of the same key are in the same partition. 9 | 10 | ## Topics 11 | 12 | Kafka stores messages in topics that are partitioned and replicated across multiple brokers in a cluster. 13 | 14 | ## Kafka Clients 15 | 16 | [Producers](clients/producer/index.md) send messages to topics from which [consumers](clients/consumer/index.md) read. 17 | 18 | ## Language Agnostic 19 | 20 | Kafka clients use binary protocol to talk to a Kafka cluster. 21 | 22 | ## Consumer Groups 23 | 24 | Consumers may be grouped in a consumer group with multiple consumers. Each consumer in a consumer group will read messages from a unique subset of partitions in each topic they subscribe to. Each message is delivered to one consumer in the group, and all messages with the same key arrive to the same consumer. 25 | 26 | ## Durability 27 | 28 | Kafka does not track which messages were read by consumers. Kafka keeps all messages for a finite amount of time, and it is consumers' responsibility to track their location per topic (offsets). 29 | -------------------------------------------------------------------------------- /docs/partition-leader-election/index.md: -------------------------------------------------------------------------------- 1 | # Partition Leader Election 2 | 3 | **Partition Leader Election** is a process of electing a broker as the leader of a partition. 4 | 5 | Use [kafka-leader-election](../tools/kafka-leader-election/index.md) utility for [preferred](#preferred-partition-leader-election) or [unclean](#unclean-partition-leader-election) leader election. 6 | 7 | !!! note 8 | `kafka-preferred-replica-election.sh` tool has been deprecated since Kafka 2.4.0 (cf. [KIP-460: Admin Leader Election RPC](https://cwiki.apache.org/confluence/display/KAFKA/KIP-460%3A+Admin+Leader+Election+RPC)). 9 | 10 | Observe `state.change.logger` (default: `state-change.log`) to trace the process in the logs. 11 | 12 | Internally, Kafka controller uses [Election](../controller/Election.md) utility (and [PartitionLeaderElectionAlgorithms](../controller/PartitionLeaderElectionAlgorithms.md)) for the algorithms for partition leader election. 13 | 14 | ## Preferred Partition Leader Election 15 | 16 | **Preferred Partition Leader Election** is...FIXME 17 | 18 | ## Unclean Partition Leader Election 19 | 20 | **Unclean Partition Leader Election** allows a [non-ISR replica broker to be elected as a partition leader](../controller/PartitionLeaderElectionAlgorithms.md#offlinePartitionLeaderElection) (as the last resort since doing so may result in data loss). 21 | 22 | [unclean.leader.election.enable](../KafkaConfig.md#unclean.leader.election.enable) configuration property is used to enable it cluster-wide (for any topic) or per topic. 23 | 24 | Enable INFO logging level for [kafka.controller.KafkaController](../controller/KafkaController.md#logging) logger to observe the [process](../controller/KafkaController.md#processUncleanLeaderElectionEnable) in the logs. 25 | 26 | ## Demo 27 | 28 | [Demo: Using kafka-leader-election](../demo/partition-leader-election.md). 29 | -------------------------------------------------------------------------------- /docs/tiered-storage/.pages: -------------------------------------------------------------------------------- 1 | title: Tiered Storage 2 | nav: 3 | - index.md 4 | - ... 5 | -------------------------------------------------------------------------------- /docs/tiered-storage/RemoteLogManager.md: -------------------------------------------------------------------------------- 1 | # RemoteLogManager 2 | 3 | ## Creating Instance 4 | 5 | `RemoteLogManager` takes the following to be created: 6 | 7 | * [RemoteLogManagerConfig](RemoteLogManagerConfig.md) 8 | * Broker ID 9 | * Log Directory 10 | * Cluster ID 11 | * `Time` 12 | * Fetch Log Function (`Function>`) 13 | * `updateRemoteLogStartOffset` (`BiConsumer`) 14 | * `BrokerTopicStats` 15 | 16 | `RemoteLogManager` is created when: 17 | 18 | * `BrokerServer` is requested to [create a RemoteLogManager](../kraft/BrokerServer.md#createRemoteLogManager) 19 | * `KafkaServer` is requested to [create a RemoteLogManager](../broker/KafkaServer.md#createRemoteLogManager) 20 | -------------------------------------------------------------------------------- /docs/tiered-storage/RemoteLogManagerConfig.md: -------------------------------------------------------------------------------- 1 | # RemoteLogManagerConfig 2 | 3 | ## Creating Instance 4 | 5 | `RemoteLogManagerConfig` takes the following to be created: 6 | 7 | * [AbstractConfig](../AbstractConfig.md) 8 | 9 | `RemoteLogManagerConfig` is created when: 10 | 11 | * `KafkaConfig` is [created](../KafkaConfig.md#_remoteLogManagerConfig) 12 | 13 | ## remote.log.storage.system.enable { #remote.log.storage.system.enable } 14 | 15 | **remote.log.storage.system.enable** 16 | 17 | Enables [Tiered Storage](index.md) 18 | 19 | Default: `false` 20 | 21 | Available as [KafkaConfig.isRemoteLogStorageSystemEnabled](../KafkaConfig.md#isRemoteLogStorageSystemEnabled) 22 | 23 | Used when: 24 | 25 | * `LogManager` is [created](../log/LogManager.md#apply) 26 | * `TopicConfigHandler` is requested to [updateLogConfig](../dynamic-broker-configuration/TopicConfigHandler.md#updateLogConfig) 27 | * `ControllerConfigurationValidator` is requested to `validate` 28 | * `AdminZkClient` is requested to [validateTopicCreate](../zk/AdminZkClient.md#validateTopicCreate) and [validateTopicConfig](../zk/AdminZkClient.md#validateTopicConfig) 29 | -------------------------------------------------------------------------------- /docs/tiered-storage/index.md: -------------------------------------------------------------------------------- 1 | # Kafka Tiered Storage 2 | 3 | In tiered storage approach, a Kafka cluster is configured with two tiers of storage: local and remote. 4 | 5 | The local tier is the same as the current Kafka that uses the local disks on the Kafka brokers to store the log segments. 6 | The new remote tier uses systems (e.g., HDFS, S3) to store the completed log segments. 7 | 8 | Tiered storage is enabled using [remote.log.storage.system.enable](RemoteLogManagerConfig.md#remote.log.storage.system.enable) property. 9 | 10 | Tiered storage is not supported with multiple [log directories](../KafkaConfig.md#logDirs). 11 | 12 | ## Learn More 13 | 14 | 1. [KIP-405: Kafka Tiered Storage]({{ kafka.wiki }}/KIP-405%3A+Kafka+Tiered+Storage) 15 | -------------------------------------------------------------------------------- /docs/tools/ConsoleConsumer.md: -------------------------------------------------------------------------------- 1 | # ConsoleConsumer 2 | 3 | `ConsoleConsumer` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/tools/index.md: -------------------------------------------------------------------------------- 1 | # Tools 2 | -------------------------------------------------------------------------------- /docs/tools/kafka-acls/AclCommand.md: -------------------------------------------------------------------------------- 1 | # AclCommand 2 | 3 | `AclCommand` is an [administration command-line utility](#main) to manage ACLs in a Kafka cluster. 4 | 5 | `AclCommand` can be executed as `kafka-acls` shell script. 6 | 7 | ## SecurityDisabledException 8 | 9 | `kafka-acls.sh` requires [Authorizer](../../authorization/Authorizer.md) to be configured on a broker (when executed with `--bootstrap-server` option) or throws a `SecurityDisabledException`. 10 | 11 | ``` console 12 | $ ./bin/kafka-acls.sh --list --bootstrap-server :9092 13 | SecurityDisabledException: No Authorizer is configured on the broker 14 | ``` 15 | 16 | ## Executing Command 17 | 18 | `main` selects the [AclCommandService](AclCommandService.md): 19 | 20 | * `AdminClientService` when `--bootstrap-server` option is used 21 | * [AuthorizerService](AuthorizerService.md) with [AclAuthorizer](../../authorization/AclAuthorizer.md) otherwise 22 | 23 | In the end, `main` executes the operation: 24 | 25 | * `add` to [add ACLs](AclCommandService.md#addAcls) 26 | * `remove` to [remove ACLs](AclCommandService.md#removeAcls) 27 | * `list` to [list ACLs](AclCommandService.md#listAcls) for `--topic`, `--group` or `--cluster` resource types 28 | -------------------------------------------------------------------------------- /docs/tools/kafka-acls/AclCommandOptions.md: -------------------------------------------------------------------------------- 1 | # AclCommandOptions 2 | 3 | `AclCommandOptions` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/tools/kafka-acls/AclCommandService.md: -------------------------------------------------------------------------------- 1 | # AclCommandService 2 | 3 | `AclCommandService` is an [abstraction](#contract) of [AclCommand services](#implementations) for [AclCommand](AclCommand.md). 4 | 5 | ## Contract 6 | 7 | ### addAcls 8 | 9 | ```scala 10 | addAcls(): Unit 11 | ``` 12 | 13 | Used when: 14 | 15 | * [AclCommand](AclCommand.md) is executed (with `--add` option) 16 | 17 | ### listAcls 18 | 19 | ```scala 20 | listAcls(): Unit 21 | ``` 22 | 23 | Used when: 24 | 25 | * [AclCommand](AclCommand.md) is executed (with `--list` option) 26 | 27 | ### removeAcls 28 | 29 | ```scala 30 | removeAcls(): Unit 31 | ``` 32 | 33 | Used when: 34 | 35 | * [AclCommand](AclCommand.md) is executed (with `--remove` option) 36 | 37 | ## Implementations 38 | 39 | * [AdminClientService](AdminClientService.md) 40 | * [AuthorizerService](AuthorizerService.md) 41 | -------------------------------------------------------------------------------- /docs/tools/kafka-acls/AdminClientService.md: -------------------------------------------------------------------------------- 1 | # AdminClientService 2 | 3 | `AdminClientService` is an [AclCommandService](AclCommandService.md) that is used by [AclCommand](AclCommand.md) when executed with `--bootstrap-server` option. 4 | 5 | ## Creating Instance 6 | 7 | `AdminClientService` takes the following to be created: 8 | 9 | * [AclCommandOptions](AclCommandOptions.md) 10 | -------------------------------------------------------------------------------- /docs/tools/kafka-acls/AuthorizerService.md: -------------------------------------------------------------------------------- 1 | # AuthorizerService 2 | 3 | `AuthorizerService` is an [AclCommandService](AclCommandService.md) that is used by [AclCommand](AclCommand.md) when executed with no `--bootstrap-server` option. 4 | 5 | ## Creating Instance 6 | 7 | `AuthorizerService` takes the following to be created: 8 | 9 | * Class name of the authorizer 10 | * [AclCommandOptions](AclCommandOptions.md) 11 | -------------------------------------------------------------------------------- /docs/tools/kafka-acls/index.md: -------------------------------------------------------------------------------- 1 | # kafka-acls 2 | -------------------------------------------------------------------------------- /docs/tools/kafka-cluster/index.md: -------------------------------------------------------------------------------- 1 | # kafka-cluster 2 | 3 | `kafka-cluster.sh` utility can print out the [cluster id](#cluster-id) or [unregister a broker](#unregister). 4 | 5 | ## cluster-id 6 | 7 | The `cluster-id` command prints out the cluster id. 8 | 9 | ## unregister 10 | 11 | The `unregister` command removes the registration of a specific broker ID. 12 | -------------------------------------------------------------------------------- /docs/tools/kafka-configs/ConfigCommand.md: -------------------------------------------------------------------------------- 1 | # ConfigCommand 2 | 3 | `kafka.admin.ConfigCommand` is a [command-line utility](#main). 4 | 5 | `ConfigCommand` can be executed using [kafka-configs](index.md) shell script. 6 | 7 | ## processCommandWithZk 8 | 9 | ```scala 10 | processCommandWithZk( 11 | zkConnectString: String, 12 | opts: ConfigCommandOptions): Unit 13 | ``` 14 | 15 | `processCommandWithZk`...FIXME 16 | 17 | --- 18 | 19 | `processCommandWithZk` is used when: 20 | 21 | * `ConfigCommand` is [executed](#main) with `--zookeeper` option (deprecated) 22 | 23 | ## alterConfigWithZk 24 | 25 | ```scala 26 | alterConfigWithZk( 27 | zkClient: KafkaZkClient, 28 | opts: ConfigCommandOptions, 29 | adminZkClient: AdminZkClient): Unit 30 | ``` 31 | 32 | `alterConfigWithZk`...FIXME 33 | 34 | --- 35 | 36 | `alterConfigWithZk` is used when: 37 | 38 | * `ConfigCommand` is [executed](#processCommandWithZk) with `--alter` option (to alter the configuration for an entity) 39 | -------------------------------------------------------------------------------- /docs/tools/kafka-configs/index.md: -------------------------------------------------------------------------------- 1 | # kafka-configs 2 | 3 | `kafka-configs` utility uses [ConfigCommand](ConfigCommand.md) to...FIXME 4 | 5 | ## alterBrokerConfig 6 | 7 | ```console 8 | ./bin/kafka-configs.sh \ 9 | --bootstrap-server :9092 \ 10 | --alter \ 11 | --entity-type brokers \ 12 | --entity-name 0 \ 13 | --add-config advertised.listeners=plaintext://:9092 14 | ``` 15 | 16 | ## processBrokerConfig 17 | 18 | ```console 19 | ./bin/kafka-configs.sh \ 20 | --bootstrap-server :9092 \ 21 | --describe \ 22 | --entity-type brokers \ 23 | --entity-name 0 24 | ``` 25 | -------------------------------------------------------------------------------- /docs/tools/kafka-dump-log/index.md: -------------------------------------------------------------------------------- 1 | # kafka-dump-log Tool 2 | 3 | `kafka-dump-log.sh` utility can dump log segments of a topic (incl. cluster metadata) to the console (e.g., for debugging a seemingly corrupt log segment). 4 | 5 | `kafka-dump-log.sh` has got two new flags: [--cluster-metadata-decoder](#cluster-metadata-decoder), and [--skip-record-metadata](#skip-record-metadata). 6 | 7 | ```shell 8 | ./bin/kafka-dump-log.sh \ 9 | --cluster-metadata-decoder \ 10 | --skip-record-metadata \ 11 | --files /tmp/kraft-controller-logs/__cluster_metadata-0/00000000000000000000.log,/tmp/kraft-controller-logs/__cluster_metadata-0/00000000000000000000.index 12 | ``` 13 | 14 | ## cluster-metadata-decoder 15 | 16 | `--cluster-metadata-decoder` flag tells the `DumpLogSegments` tool to decode the records as [cluster metadata](../../kraft/index.md) records. 17 | 18 | ## skip-record-metadata 19 | 20 | `--skip-record-metadata` flag will skip printing metadata for each record. However, metadata for each record batch will still be printed when this flag is present. 21 | -------------------------------------------------------------------------------- /docs/tools/kafka-get-offsets/GetOffsetShell.md: -------------------------------------------------------------------------------- 1 | # GetOffsetShell 2 | 3 | ## Launching Application 4 | 5 | ```scala 6 | main( 7 | args: Array[String]): Unit 8 | ``` 9 | 10 | `main` [fetchOffsets](#fetchOffsets). 11 | 12 | ## fetchOffsets 13 | 14 | ```scala 15 | fetchOffsets( 16 | args: Array[String]): Unit 17 | ``` 18 | 19 | `fetchOffsets`...FIXME 20 | -------------------------------------------------------------------------------- /docs/tools/kafka-get-offsets/index.md: -------------------------------------------------------------------------------- 1 | # kafka-get-offsets 2 | 3 | `kafka-get-offsets` utility is an interactive shell for getting topic-partition offsets. 4 | 5 | ```console 6 | $ ./bin/kafka-get-offsets.sh 7 | An interactive shell for getting topic-partition offsets. 8 | ``` 9 | 10 | `kafka-get-offsets` uses [GetOffsetShell](GetOffsetShell.md) for execution. 11 | 12 | ```console 13 | $ ./bin/kafka-topics.sh \ 14 | --bootstrap-server :9092 \ 15 | --create \ 16 | --if-not-exists \ 17 | --topic demo-get-offsets \ 18 | --partitions 3 19 | ``` 20 | 21 | ```console 22 | $ ./bin/kafka-get-offsets.sh \ 23 | --bootstrap-server :9092 24 | demo-get-offsets:0:0 25 | demo-get-offsets:1:0 26 | demo-get-offsets:2:0 27 | ``` 28 | 29 | ```console 30 | $ echo zero | kcat -P -b :9092 -t demo-get-offsets -p 0 31 | ``` 32 | 33 | ```console 34 | $ ./bin/kafka-get-offsets.sh \ 35 | --bootstrap-server :9092 \ 36 | --topic demo-get-offsets 37 | demo-get-offsets:0:1 38 | demo-get-offsets:1:0 39 | demo-get-offsets:2:0 40 | ``` 41 | 42 | ```console 43 | $ ./bin/kafka-get-offsets.sh \ 44 | --bootstrap-server :9092 \ 45 | --topic-partitions demo-get-offsets:0,demo-get-offsets:2 46 | demo-get-offsets:0:1 47 | demo-get-offsets:2:0 48 | ``` 49 | 50 | ## Options 51 | 52 | ### exclude-internal-topics 53 | 54 | ### partitions 55 | 56 | ### time 57 | 58 | ### topic 59 | 60 | ### topic-partitions 61 | -------------------------------------------------------------------------------- /docs/tools/kafka-leader-election/index.md: -------------------------------------------------------------------------------- 1 | # kafka-leader-election 2 | -------------------------------------------------------------------------------- /docs/tools/kafka-metadata-shell/MetadataShell.md: -------------------------------------------------------------------------------- 1 | # MetadataShell 2 | 3 | `MetadataShell` is [launched](#main) as a command-line application using [kafka-metadata-shell.sh](index.md) shell script. 4 | 5 | ## Creating Instance 6 | 7 | `MetadataShell` takes the following to be created: 8 | 9 | * [KafkaRaftManager](#raftManager) 10 | * Snapshot Path 11 | * `FaultHandler` 12 | 13 | `MetadataShell` is created when: 14 | 15 | * `MetadataShell.Builder` is requested to `build` (a `MetadataShell`) 16 | 17 | ### KafkaRaftManager { #raftManager } 18 | 19 | `MetadataShell` is given a [KafkaRaftManager](../../kraft/KafkaRaftManager.md) when [created](#creating-instance). 20 | 21 | !!! note 22 | A `KafkaRaftManager` does not seem to be defined ever (and seems always `null`). 23 | 24 | ## Entry Point { #main } 25 | 26 | ```scala 27 | void main( 28 | String[] args) 29 | ``` 30 | 31 | `main`...FIXME 32 | 33 | --- 34 | 35 | `main` is used when: 36 | 37 | * FIXME 38 | -------------------------------------------------------------------------------- /docs/tools/kafka-metadata-shell/MetadataShellPublisher.md: -------------------------------------------------------------------------------- 1 | # MetadataShellPublisher 2 | 3 | `MetadataShellPublisher` is a [MetadataPublisher](../../metadata/MetadataPublisher.md). 4 | -------------------------------------------------------------------------------- /docs/tools/kafka-metadata-shell/index.md: -------------------------------------------------------------------------------- 1 | # kafka-metadata-shell Tool 2 | 3 | `kafka-metadata-shell.sh` utility is a Kafka metadata tool to interactively examine the metadata stored in a [KRaft cluster](../../kraft/index.md). 4 | 5 | `kafka-metadata-shell.sh` can read the metadata from a metadata snapshot on disk. 6 | 7 | !!! note "KafkaRaftManager Not Supported Yet" 8 | There is no support to specify a [KafkaRaftManager](../../kraft/KafkaRaftManager.md) on command line yet. 9 | 10 | Based on [metadata.log.max.snapshot.interval.ms](../../KafkaConfig.md#metadata.log.max.snapshot.interval.ms) and [metadata.log.max.record.bytes.between.snapshots](../../KafkaConfig.md#metadata.log.max.record.bytes.between.snapshots), KRaft controllers write metadata snapshots to cluster [metadata topic](../../kraft/index.md)'s log directory (using [SnapshotGenerator](../../metadata/SnapshotGenerator.md) and `SnapshotEmitter`). 11 | 12 | ```text 13 | INFO [SnapshotGenerator id=1] Creating new KRaft snapshot file snapshot 00000000000000061020-0000000002 because we have waited at least 60 minute(s). 14 | INFO [SnapshotEmitter id=1] Successfully wrote snapshot 00000000000000061020-0000000002 15 | ``` 16 | 17 | Use [--snapshot](#snapshot) option to load the snapshot file. 18 | 19 | ```console 20 | $ ./bin/kafka-metadata-shell.sh \ 21 | --snapshot /tmp/kraft-controller-logs/__cluster_metadata-0/00000000000000061020-0000000002.checkpoint 22 | Loading... 23 | Starting... 24 | [ Kafka Metadata Shell ] 25 | >> 26 | ``` 27 | 28 | ```text 29 | >> help 30 | Welcome to the Apache Kafka metadata shell. 31 | 32 | usage: {cat,cd,exit,find,help,history,ls,man,pwd,tree} ... 33 | 34 | positional arguments: 35 | {cat,cd,exit,find,help,history,ls,man,pwd,tree} 36 | cat Show the contents of metadata files. 37 | cd Set the current working directory. 38 | exit Exit the metadata shell. 39 | find Search for nodes in the directory hierarchy. 40 | help Display this help message. 41 | history Print command history. 42 | ls List metadata nodes. 43 | man Show the help text for a specific command. 44 | pwd Print the current working directory. 45 | tree Show the contents of metadata nodes in a tree structure. 46 | ``` 47 | 48 | ```text 49 | >> ls 50 | image local 51 | ``` 52 | 53 | The metadata tool works by replaying the log and storing the state into in-memory nodes. 54 | These nodes are presented in a fashion similar to filesystem directories. 55 | 56 | ```text 57 | >> man tree 58 | tree: Show the contents of metadata nodes in a tree structure. 59 | 60 | usage: tree targets [targets ...] 61 | 62 | positional arguments: 63 | targets The metadata nodes to display. 64 | ``` 65 | 66 | ```text 67 | >> tree local 68 | commitId: 69 | 60e845626d8a465a 70 | version: 71 | 3.6.0 72 | ``` 73 | 74 | `kafka-metadata-shell.sh` uses [MetadataShell](MetadataShell.md) as an entry point. 75 | 76 | ## Arguments 77 | 78 | ### snapshot 79 | 80 | `--snapshot` (`-s`) is a metadata snapshot file to read. 81 | -------------------------------------------------------------------------------- /docs/tools/kafka-reassign-partitions/ReassignPartitionsCommand.md: -------------------------------------------------------------------------------- 1 | # ReassignPartitionsCommand 2 | 3 | `ReassignPartitionsCommand` is a [command-line application](#main) for [kafka-reassign-partitions](index.md) to [generate](#generate), [execute](#execute) and [verify](#verify) a custom partition (re)assignment configuration (as specified using a [reassignment JSON file](#reassignment-json-file)). 4 | 5 | ## Actions 6 | 7 | ### execute 8 | 9 | Executes the reassignment as specified by the [reassignment-json-file](#reassignment-json-file) option 10 | 11 | ### generate 12 | 13 | Generates a candidate partition reassignment configuration 14 | 15 | !!! note 16 | This only generates a candidate assignment and does not execute it. 17 | 18 | ### verify 19 | 20 | Verifies if the reassignment completed as specified by the [reassignment-json-file](#reassignment-json-file). If there is a throttle engaged for the replicas specified, and the rebalance has completed, the throttle will be removed 21 | 22 | ## Options 23 | 24 | ### reassignment-json-file 25 | 26 | A JSON file with a custom partition (re)assignment configuration 27 | 28 | The format to use is as follows: 29 | 30 | ```json 31 | { 32 | "partitions": [ 33 | { 34 | "topic": "foo", 35 | "partition": 1, 36 | "replicas": [ 37 | 1, 38 | 2, 39 | 3 40 | ], 41 | "log_dirs": [ 42 | "dir1", 43 | "dir2", 44 | "dir3" 45 | ] 46 | } 47 | ], 48 | "version": 1 49 | } 50 | ``` 51 | 52 | Note that `log_dirs` is optional. When specified, its length must equal the length of the replicas list. The value in this list can be either `"any"` or the absolute path of the log directory on the broker. 53 | 54 | If absolute log directory path is specified, it is currently required that the replica has not already been created on that broker. The replica will then be created in the specified log directory on the broker later. 55 | 56 | ### replica-alter-log-dirs-throttle 57 | 58 | The movement of replicas between log directories on the same broker will be throttled to this value (bytes/sec). 59 | 60 | Default: `-1` 61 | 62 | Rerunning with this option, whilst a rebalance is in progress, will alter the throttle value. The throttle rate should be at least 1 KB/s. 63 | 64 | ### throttle 65 | 66 | The movement of partitions between brokers will be throttled to this value (bytes/sec). 67 | 68 | Default: `-1` 69 | 70 | Rerunning with this option, whilst a rebalance is in progress, will alter the throttle value. The throttle rate should be at least 1 KB/s. 71 | 72 | ### timeout 73 | 74 | The maximum time (in ms) allowed to wait for partition reassignment execution to be successfully initiated 75 | 76 | Default: `10000` 77 | 78 | ## Executing Application 79 | -------------------------------------------------------------------------------- /docs/tools/kafka-reassign-partitions/demo.md: -------------------------------------------------------------------------------- 1 | # Demo: kafka-reassign-partitions 2 | 3 | ## Set Up Kafka Cluster 4 | 5 | Start a 3-broker Kafka cluster. 6 | 7 | ```shell 8 | ./bin/kafka-server-start.sh config/server.properties \ 9 | --override broker.id=10 \ 10 | --override log.dirs=/tmp/kafka-logs-10 \ 11 | --override listeners=PLAINTEXT://:9192 12 | ``` 13 | 14 | ```shell 15 | ./bin/kafka-server-start.sh config/server.properties \ 16 | --override broker.id=20 \ 17 | --override log.dirs=/tmp/kafka-logs-20 \ 18 | --override listeners=PLAINTEXT://:9292 19 | ``` 20 | 21 | ```shell 22 | ./bin/kafka-server-start.sh config/server.properties \ 23 | --override broker.id=30 \ 24 | --override log.dirs=/tmp/kafka-logs-30 \ 25 | --override listeners=PLAINTEXT://:9392 26 | ``` 27 | 28 | ## Create Topic 29 | 30 | ```shell 31 | ./bin/kafka-topics.sh \ 32 | --bootstrap-server :9192 \ 33 | --create \ 34 | --topic demo-reassign-partitions \ 35 | --replication-factor 2 \ 36 | --partitions 1 37 | ``` 38 | 39 | ```shell 40 | ./bin/kafka-topics.sh \ 41 | --bootstrap-server :9192 \ 42 | --describe \ 43 | --topic demo-reassign-partitions 44 | ``` 45 | 46 | ```text 47 | Topic: demo-reassign-partitions TopicId: J3ginqnTTg-LjVoI20dqqw PartitionCount: 1 ReplicationFactor: 2 Configs: segment.bytes=1073741824 48 | Topic: demo-reassign-partitions Partition: 0 Leader: 10 Replicas: 10,20 Isr: 10,20 49 | ``` 50 | 51 | ## reassign-partitions.json 52 | 53 | ```json title="reassign-partitions.json" 54 | { 55 | "partitions": [ 56 | { 57 | "topic": "demo-reassign-partitions", 58 | "partition": 1, 59 | "replicas": [ 60 | 20, 61 | 30 62 | ] 63 | } 64 | ], 65 | "version": 1 66 | } 67 | ``` 68 | 69 | ## Generate Reassignment Configuration 70 | 71 | !!! FIXME "Does not seem to work" 72 | 73 | ```console 74 | $ ./bin/kafka-reassign-partitions.sh \ 75 | --bootstrap-server :9192 \ 76 | --generate \ 77 | --topics-to-move-json-file reassign-partitions.json \ 78 | --broker-list 10,20,30 79 | ``` 80 | 81 | ## Verify Reassignment Configuration 82 | 83 | ``` console 84 | $ ./bin/kafka-reassign-partitions.sh \ 85 | --bootstrap-server :9192 \ 86 | --verify \ 87 | --reassignment-json-file reassign-partitions.json 88 | ``` 89 | -------------------------------------------------------------------------------- /docs/tools/kafka-reassign-partitions/index.md: -------------------------------------------------------------------------------- 1 | # kafka-reassign-partitions 2 | 3 | `kafka-reassign-partitions` utility is used to move topic partitions between replicas (based on a [partition reassignment JSON file](ReassignPartitionsCommand.md#reassignment-json-file)). 4 | 5 | `kafka-reassign-partitions` uses [ReassignPartitionsCommand](ReassignPartitionsCommand.md) for its execution. 6 | 7 | ## Options 8 | 9 | ``` console 10 | $ ./bin/kafka-reassign-partitions.sh --help 11 | This tool helps to move topic partitions between replicas. 12 | Option Description 13 | ------ ----------- 14 | --additional Execute this reassignment in addition 15 | to any other ongoing ones. This 16 | option can also be used to change 17 | the throttle of an ongoing 18 | reassignment. 19 | ... 20 | ``` 21 | 22 | ## Demo 23 | 24 | [Demo: kafka-reassign-partitions](demo.md) 25 | -------------------------------------------------------------------------------- /docs/tools/kafka-replica-verification/ReplicaBuffer.md: -------------------------------------------------------------------------------- 1 | # ReplicaBuffer 2 | 3 | ## Creating Instance 4 | 5 | `ReplicaBuffer` takes the following to be created: 6 | 7 | * Expected Replicas per TopicPartition (`Map[TopicPartition, Int]`) 8 | * Initial offsets (`Map[TopicPartition, Long]`) 9 | * Expected number of fetchers 10 | * [Report interval](#reportInterval) 11 | 12 | `ReplicaBuffer` is created when: 13 | 14 | * `ReplicaVerificationTool` is [executed](ReplicaVerificationTool.md#main) 15 | 16 | ## Report Interval 17 | 18 | `ReplicaBuffer` is given a report interval when [created](#creating-instance). 19 | 20 | The value is `--report-interval-ms` and defaults to `30s`. 21 | 22 | The interval is used when [verifyCheckSum](#verifyCheckSum) to print out the following to the standard output: 23 | 24 | ```text 25 | [currentTimeMs]: max lag is [lag] for partition [partition] at offset [offset] among [n] partitions 26 | ``` 27 | -------------------------------------------------------------------------------- /docs/tools/kafka-replica-verification/ReplicaFetcher.md: -------------------------------------------------------------------------------- 1 | # ReplicaFetcher 2 | 3 | `ReplicaFetcher` is a thread that [ReplicaVerificationTool](ReplicaVerificationTool.md) uses for [replica verification](#doWork-verification). 4 | 5 | ## Creating Instance 6 | 7 | `ReplicaFetcher` takes the following to be created: 8 | 9 | * `ReplicaFetcher-[brokerId]` 10 | * Source broker 11 | * `TopicPartition`s 12 | * Topics IDs (`Map[String, Uuid]`) 13 | * [ReplicaBuffer](ReplicaBuffer.md) 14 | * Socket timeout (`30000`) 15 | * Socket buffer size (`256000`) 16 | * Fetch size 17 | * Max wait 18 | * Min bytes 19 | * [doVerification](#doVerification) flag 20 | * Consumer properties 21 | * Fetcher ID 22 | 23 | `ReplicaFetcher` is created when: 24 | 25 | * `ReplicaVerificationTool` command-line utility is [executed](ReplicaVerificationTool.md#main) 26 | 27 | ### doVerification 28 | 29 | `ReplicaFetcher` is given `doVerification` flag when [created](#creating-instance). 30 | 31 | `doVerification` flag is enabled for a single `ReplicaFetcher` among the replica fetcher threads. 32 | 33 | The flag is used to determine which `ReplicaFetcher` should [perform verification](#doWork-verification). 34 | 35 | ## doWork 36 | 37 | ```scala 38 | doWork(): Unit 39 | ``` 40 | 41 | `doWork` is part of the `ShutdownableThread` abstraction. 42 | 43 | --- 44 | 45 | `doWork` creates a `requestMap` with `TopicPartition`s and `PartitionData`s. 46 | 47 | `doWork` prints out the following DEBUG message to the logs: 48 | 49 | ```text 50 | Issuing fetch request 51 | ``` 52 | 53 | `doWork` sends a `Fetch` request (with the `requestMap`). 54 | 55 | With a `FetchResponse`, `doWork` [addFetchedData](ReplicaBuffer.md#addFetchedData) (to the [ReplicaBuffer](#replicaBuffer) that all `ReplicaFetcher`s append fetched data to). Otherwise, `doWork`...FIXME 56 | 57 | `doWork` decrements the [fetcherBarrier](ReplicaBuffer.md#getFetcherBarrier) latch. If it reaches 0, `doWork` prints out the following DEBUG message to the logs: 58 | 59 | ```text 60 | Done fetching 61 | ``` 62 | 63 | `doWork` waits for the other fetchers to finish and prints out the following DEBUG message to the logs: 64 | 65 | ```text 66 | Ready for verification 67 | ``` 68 | 69 | With the [doVerification](#doVerification) flag enabled, `doWork` [performs verification](#doWork-verification). 70 | 71 | `doWork` waits for the verification to be finished and prints out the following DEBUG message to the logs: 72 | 73 | ```text 74 | Done verification 75 | ``` 76 | 77 | ### Verification 78 | 79 | `doWork`...FIXME 80 | -------------------------------------------------------------------------------- /docs/tools/kafka-replica-verification/ReplicaVerificationTool.md: -------------------------------------------------------------------------------- 1 | # ReplicaVerificationTool 2 | 3 | `ReplicaVerificationTool` is a [command-line application](#main) for [kafka-replica-verification](index.md) to perform [replica verification](ReplicaFetcher.md#doWork-verification). 4 | 5 | ## Executing Application 6 | 7 | `main` prints out the following INFO message to the logs: 8 | 9 | ```text 10 | Getting topic metadata... 11 | ``` 12 | 13 | `main` [createAdminClient](#createAdminClient) (with the brokers in `--broker-list` option) for [topics metadata](#listTopicsMetadata) and [broker info](#brokerDetails). 14 | 15 | `main` creates `TopicPartitionReplica`s for the topics. 16 | 17 | `main` prints out the following DEBUG message to the logs: 18 | 19 | ```text 20 | Selected topic partitions: [topicPartitionReplicas] 21 | ``` 22 | 23 | `main` groups partitions per broker and prints out the following DEBUG message to the logs: 24 | 25 | ```text 26 | Topic partitions per broker: [brokerToTopicPartitions] 27 | ``` 28 | 29 | `main` groups partitions per replica to count the number of replicas and prints out the following DEBUG message to the logs: 30 | 31 | ```text 32 | Expected replicas per topic partition: [expectedReplicasPerTopicPartition] 33 | ``` 34 | 35 | `main` [creates a consumer config](#consumerConfig). 36 | 37 | `main` creates [ReplicaFetcher](ReplicaFetcher.md)s for every replica broker and [starts them all](ReplicaFetcher.md#doWork). 38 | 39 | `ReplicaFetcher`s run until termination (Ctrl-C). `main` prints out the following INFO message to the logs: 40 | 41 | ```text 42 | Stopping all fetchers 43 | ``` 44 | -------------------------------------------------------------------------------- /docs/tools/kafka-replica-verification/index.md: -------------------------------------------------------------------------------- 1 | # kafka-replica-verification 2 | 3 | `kafka-replica-verification` utility is used to verify replica consistency (i.e., validate that all replicas for a set of topics have the same data). 4 | 5 | `kafka-replica-verification` uses [ReplicaVerificationTool](ReplicaVerificationTool.md) with [ReplicaFetcher](ReplicaFetcher.md)s for its execution. 6 | 7 | ## Options 8 | 9 | ``` console 10 | $ ./bin/kafka-replica-verification.sh --help 11 | Validate that all replicas for a set of topics have the same data. 12 | Option Description 13 | ------ ----------- 14 | --broker-list port of the server to connect to. 16 | --fetch-size The fetch size of each request. 17 | (default: 1048576) 18 | --help Print usage information. 19 | --max-wait-ms The max amount of time each fetch 20 | request waits. (default: 1000) 21 | --report-interval-ms The reporting interval. (default: 22 | 30000) 23 | --time offsets. (default: -1) 25 | --topic-white-list instead; ignored if --topics-include 27 | specified. List of topics to verify 28 | replica consistency. Defaults to '. 29 | *' (all topics) (default: .*) 30 | --topics-include consistency. Defaults to '.*' (all 32 | topics) (default: .*) 33 | --version Print version information and exit. 34 | ``` 35 | 36 | ## Demo 37 | 38 | ``` console 39 | $ ./bin/kafka-replica-verification.sh \ 40 | --broker-list :9092 \ 41 | --report-interval-ms 5000 42 | verification process is started. 43 | max lag is 0 for partition t100-0 at offset 0 among 1 partitions 44 | ``` 45 | -------------------------------------------------------------------------------- /docs/tools/kafka-storage/StorageTool.md: -------------------------------------------------------------------------------- 1 | # StorageTool 2 | 3 | `StorageTool` (`kafka.tools.StorageTool`) is an utility to...FIXME 4 | 5 | `StorageTool` can be executed using [bin/kafka-storage.sh](index.md) script. 6 | 7 | ## random-uuid 8 | 9 | ## format 10 | 11 | `format` command [configToLogDirectories](#configToLogDirectories). 12 | 13 | ## configToLogDirectories { #configToLogDirectories } 14 | 15 | ```scala 16 | configToLogDirectories( 17 | config: KafkaConfig): Seq[String] 18 | ``` 19 | 20 | `configToLogDirectories` takes the value of the following configuration properties: 21 | 22 | * [log.dirs or log.dir](../../KafkaConfig.md#logDirs) (depending on availability) 23 | * [metadataLogDir](../../KafkaConfig.md#metadataLogDir) 24 | -------------------------------------------------------------------------------- /docs/tools/kafka-storage/index.md: -------------------------------------------------------------------------------- 1 | # kafka-storage Utility 2 | 3 | `kafka-storage` script is used to...FIXME...when setting up a Kafka cluster in [KRaft mode](../../kraft/index.md). 4 | 5 | ```shell 6 | $ ./bin/kafka-storage.sh -h 7 | usage: kafka-storage [-h] {info,format,random-uuid} ... 8 | 9 | The Kafka storage tool. 10 | 11 | positional arguments: 12 | {info,format,random-uuid} 13 | info Get information about the Kafka log directories on this node. 14 | format Format the Kafka log directories on this node. 15 | random-uuid Print a random UUID. 16 | 17 | optional arguments: 18 | -h, --help show this help message and exit 19 | ``` 20 | 21 | `kafka-storage` runs [kafka.tools.StorageTool](StorageTool.md). 22 | 23 | ## format 24 | 25 | [format](StorageTool.md#format) command is used to format the Kafka storage directories of a node (brokers and controllers). 26 | 27 | ```shell 28 | $ ./bin/kafka-storage.sh format -h 29 | usage: kafka-storage format [-h] --config CONFIG --cluster-id CLUSTER_ID [--add-scram ADD_SCRAM] [--ignore-formatted] [--release-version RELEASE_VERSION] 30 | 31 | optional arguments: 32 | -h, --help show this help message and exit 33 | --config CONFIG, -c CONFIG 34 | The Kafka configuration file to use. 35 | --cluster-id CLUSTER_ID, -t CLUSTER_ID 36 | The cluster ID to use. 37 | --add-scram ADD_SCRAM, -S ADD_SCRAM 38 | A SCRAM_CREDENTIAL to add to the __cluster_metadata log e.g. 39 | 'SCRAM-SHA-256=[name=alice,password=alice-secret]' 40 | 'SCRAM-SHA-512=[name=alice,iterations=8192,salt="N3E=",saltedpassword="YCE="]' 41 | --ignore-formatted, -g 42 | --release-version RELEASE_VERSION, -r RELEASE_VERSION 43 | A KRaft release version to use for the initial metadata version. The minimum is 3.0, the default is 3.6-IV2 44 | ``` 45 | 46 | `format` is the second command to be executed while setting up a Kafka cluster. 47 | 48 | ```shell 49 | $ ./bin/kafka-storage.sh format -t $KAFKA_CLUSTER_ID -c config/kraft/server.properties 50 | Formatting /tmp/kraft-combined-logs with metadata.version 3.6-IV2. 51 | ``` 52 | 53 | ## info 54 | 55 | The `info` command gives information about the configured storage directories. 56 | 57 | ```shell 58 | $ ./bin/kafka-storage.sh info -h 59 | usage: kafka-storage info [-h] --config CONFIG 60 | 61 | optional arguments: 62 | -h, --help show this help message and exit 63 | --config CONFIG, -c CONFIG 64 | The Kafka configuration file to use. 65 | ``` 66 | 67 | ```shell 68 | $ ./bin/kafka-storage.sh info -c config/kraft/server.properties 69 | Found log directory: 70 | /tmp/kraft-combined-logs 71 | 72 | Found metadata: {cluster.id=F9_futKUQPKBwpQddvXsDQ, node.id=1, version=1} 73 | 74 | $ tree /tmp/kraft-combined-logs 75 | /tmp/kraft-combined-logs 76 | ├── bootstrap.checkpoint 77 | └── meta.properties 78 | ``` 79 | 80 | ## random-uuid 81 | 82 | [random-uuid](StorageTool.md#random-uuid) command prints out a pseudo randomly-generated UUID of a cluster to stdout. 83 | 84 | ```shell 85 | $ ./bin/kafka-storage.sh random-uuid 86 | pnHyFfvWT6i2F2wZzSUx6A 87 | ``` 88 | 89 | `random-uuid` is the first command to be executed while setting up a Kafka cluster. 90 | 91 | ```shell 92 | KAFKA_CLUSTER_ID="$(bin/kafka-storage.sh random-uuid)" 93 | ``` 94 | -------------------------------------------------------------------------------- /docs/transactions/TransactionConfig.md: -------------------------------------------------------------------------------- 1 | # TransactionConfig 2 | 3 | `TransactionConfig` holds the values of the transactional configuration properties. 4 | 5 | ## transactional.id.expiration.ms 6 | 7 | [transactional.id.expiration.ms](../KafkaConfig.md#transactionalIdExpirationMs) 8 | 9 | Default: 7 days 10 | 11 | ## transaction.max.timeout.ms 12 | 13 | [transaction.max.timeout.ms](../KafkaConfig.md#transactionMaxTimeoutMs) 14 | 15 | Default: 15 minutes 16 | 17 | ## transaction.state.log.num.partitions 18 | 19 | [transaction.state.log.num.partitions](../KafkaConfig.md#transactionTopicPartitions) 20 | 21 | Default: 50 22 | 23 | ## transaction.state.log.replication.factor 24 | 25 | [transaction.state.log.replication.factor](../KafkaConfig.md#transactionTopicReplicationFactor) 26 | 27 | Default: 3 28 | 29 | ## transaction.state.log.segment.bytes 30 | 31 | [transaction.state.log.segment.bytes](../KafkaConfig.md#transactionTopicSegmentBytes) 32 | 33 | Default: 100 * 1024 * 1024 34 | 35 | ## transaction.state.log.load.buffer.size 36 | 37 | [transaction.state.log.load.buffer.size](../KafkaConfig.md#transactionsLoadBufferSize) 38 | 39 | Default: 5 * 1024 * 1024 40 | 41 | ## transaction.state.log.min.isr 42 | 43 | [transaction.state.log.min.isr](../KafkaConfig.md#transactionTopicMinISR) 44 | 45 | Default: 2 46 | 47 | ## transaction.abort.timed.out.transaction.cleanup.interval.ms 48 | 49 | [transaction.abort.timed.out.transaction.cleanup.interval.ms](../KafkaConfig.md#transactionAbortTimedOutTransactionCleanupIntervalMs) 50 | 51 | Default: 10 seconds 52 | 53 | ## transaction.remove.expired.transaction.cleanup.interval.ms 54 | 55 | [transaction.remove.expired.transaction.cleanup.interval.ms](../KafkaConfig.md#transactionRemoveExpiredTransactionalIdCleanupIntervalMs) 56 | 57 | Default: 1 hour 58 | 59 | ## request.timeout.ms 60 | 61 | [request.timeout.ms](../KafkaConfig.md#requestTimeoutMs) 62 | 63 | Default: 30000 64 | -------------------------------------------------------------------------------- /docs/transactions/TransactionMarkerChannelManager.md: -------------------------------------------------------------------------------- 1 | # TransactionMarkerChannelManager 2 | 3 | `TransactionMarkerChannelManager` is...FIXME 4 | -------------------------------------------------------------------------------- /docs/transactions/index.md: -------------------------------------------------------------------------------- 1 | # Transactions 2 | 3 | Apache Kafka supports transactional record delivery (and consumption if in consumer-process-produce processing mode). 4 | 5 | Every Kafka broker runs a [TransactionCoordinator](TransactionCoordinator.md) to manage (_coordinate_) transactions. 6 | 7 | ## Transactional Producer 8 | 9 | A [KafkaProducer](../clients/producer/KafkaProducer.md) is [transactional](../clients/producer/TransactionManager.md#isTransactional) when [transactional.id](../clients/producer/ProducerConfig.md#transactional.id) configuration property is specified. 10 | 11 | Any [record sending](../clients/producer/KafkaProducer.md#send) has to be after [KafkaProducer.initTransactions](../clients/producer/KafkaProducer.md#initTransactions) followed by [KafkaProducer.beginTransaction](../clients/producer/KafkaProducer.md#beginTransaction). Otherwise, the underlying [TransactionManager](../clients/producer/TransactionManager.md) is going to be in a wrong [state](../clients/producer/TransactionManager.md#states) (that will inevitably lead to exceptions). 12 | 13 | ## Transaction-Aware Consumer 14 | 15 | A [KafkaConsumer](../clients/consumer/KafkaConsumer.md) supports transactions using [isolation.level](../clients/consumer/ConsumerConfig.md#isolation.level) configuration property. 16 | 17 | ## kafka-console-consumer 18 | 19 | `kafka-console-consumer` supports `--isolation-level` option for [isolation.level](../clients/consumer/ConsumerConfig.md#isolation.level) configuration property. 20 | 21 | ## Demo 22 | 23 | [Demo: Transactional Kafka Producer](../demo/transactional-kafka-producer.md) 24 | 25 | ## Transactional Configuration Properties 26 | 27 | [TransactionConfig](TransactionConfig.md) 28 | 29 | ## Transaction Topic 30 | 31 | Kafka brokers use `__transaction_state` internal topic for managing transactions (as [records](TransactionStateManager.md#appendTransactionToLog)). 32 | 33 | `__transaction_state` is [auto-created](../DefaultAutoTopicCreationManager.md#creatableTopic) at the first transaction. 34 | 35 | The number of partitions is configured using [transaction.state.log.num.partitions](../KafkaConfig.md#transaction.state.log.num.partitions) configuration property. 36 | 37 | A transaction (record) is assigned a partition (_txn topic partition_) based on the [absolute hash code](TransactionStateManager.md#partitionFor) of the [transactional.id](../clients/producer/ProducerConfig.md#transactional.id). 38 | 39 | ## Learning Resources 40 | 41 | * [Transactions in Apache Kafka](https://www.confluent.io/blog/transactions-apache-kafka/) by Confluent 42 | -------------------------------------------------------------------------------- /docs/zk/AdminZkClient.md: -------------------------------------------------------------------------------- 1 | # AdminZkClient 2 | 3 | ## Creating Instance 4 | 5 | `AdminZkClient` takes the following to be created: 6 | 7 | * [KafkaZkClient](KafkaZkClient.md) 8 | 9 | `AdminZkClient` is created when: 10 | 11 | * `ConfigCommand` is requested to `processCommandWithZk` 12 | * `DynamicBrokerConfig` is requested to [initialize](../dynamic-broker-configuration/DynamicBrokerConfig.md#initialize) 13 | * `KafkaServer` is requested to [start up](../broker/KafkaServer.md#startup) 14 | * `ZkAdminManager` is created (`adminZkClient`) 15 | * `ZkConfigManager` is created (`adminZkClient`) 16 | * `ZkConfigRepository` is created (`adminZkClient`) 17 | 18 | ## fetchEntityConfig 19 | 20 | ```scala 21 | fetchEntityConfig( 22 | rootEntityType: String, 23 | sanitizedEntityName: String): Properties 24 | ``` 25 | 26 | `fetchEntityConfig` requests the [KafkaZkClient](#zkClient) to [getEntityConfigs](KafkaZkClient.md#getEntityConfigs). 27 | 28 | --- 29 | 30 | `fetchEntityConfig` is used when: 31 | 32 | * FIXME 33 | 34 | ## changeEntityConfig 35 | 36 | ```scala 37 | changeEntityConfig( 38 | rootEntityType: String, 39 | fullSanitizedEntityName: String, 40 | configs: Properties): Unit 41 | ``` 42 | 43 | `changeEntityConfig`...FIXME 44 | 45 | --- 46 | 47 | `changeEntityConfig` is used when: 48 | 49 | * `AdminZkClient` is requested to [changeClientIdConfig](#changeClientIdConfig), [changeUserOrUserClientIdConfig](#changeUserOrUserClientIdConfig), [changeIpConfig](#changeIpConfig), [changeTopicConfig](#changeTopicConfig), [changeBrokerConfig](#changeBrokerConfig) 50 | 51 | ## changeTopicConfig 52 | 53 | ```scala 54 | changeTopicConfig( 55 | topic: String, 56 | configs: Properties): Unit 57 | ``` 58 | 59 | `changeTopicConfig`...FIXME 60 | 61 | --- 62 | 63 | `changeTopicConfig` is used when: 64 | 65 | * `ZkAdminManager` is requested to [alterTopicConfigs](../ZkAdminManager.md#alterTopicConfigs) 66 | * `AdminZkClient` is requested to [changeConfigs](#changeConfigs) 67 | 68 | ## changeConfigs 69 | 70 | ```scala 71 | changeConfigs( 72 | entityType: String, 73 | entityName: String, 74 | configs: Properties): Unit 75 | ``` 76 | 77 | `changeConfigs`...FIXME 78 | 79 | --- 80 | 81 | `changeConfigs` is used when: 82 | 83 | * `ConfigCommand` is requested to [alterConfigWithZk](../tools/kafka-configs/ConfigCommand.md#alterConfigWithZk) 84 | * `ZkAdminManager` is requested to [alterClientQuotas](../ZkAdminManager.md#alterClientQuotas), [alterUserScramCredentials](../ZkAdminManager.md#alterUserScramCredentials) 85 | -------------------------------------------------------------------------------- /docs/zk/KafkaZkClient.md: -------------------------------------------------------------------------------- 1 | # KafkaZkClient 2 | 3 | ## getEntityConfigs 4 | 5 | ```scala 6 | getEntityConfigs( 7 | rootEntityType: String, 8 | sanitizedEntityName: String): Properties 9 | ``` 10 | 11 | `getEntityConfigs` talks to Zookeeper for `config` data in the `/config/[entityType]/[entityName]` znode. 12 | 13 | ``` console 14 | $ ./bin/zkCli.sh -server localhost:2181 ls /config 15 | [brokers, changes, clients, ips, topics, users] 16 | ``` 17 | 18 | --- 19 | 20 | `getEntityConfigs` is used when: 21 | 22 | * `AdminZkClient` is requested to [fetchEntityConfig](AdminZkClient.md#fetchEntityConfig) 23 | 24 | ## createConfigChangeNotification 25 | 26 | ```scala 27 | createConfigChangeNotification( 28 | sanitizedEntityPath: String): Unit 29 | ``` 30 | 31 | `createConfigChangeNotification`...FIXME 32 | 33 | --- 34 | 35 | `createConfigChangeNotification` is used when: 36 | 37 | * `AdminZkClient` is requested to [changeEntityConfig](AdminZkClient.md#changeEntityConfig) 38 | -------------------------------------------------------------------------------- /docs/zk/TopicPartitionStateZNode.md: -------------------------------------------------------------------------------- 1 | # TopicPartitionStateZNode 2 | -------------------------------------------------------------------------------- /graffles/ControllerEventManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/graffles/ControllerEventManager.graffle -------------------------------------------------------------------------------- /graffles/ControllerEventThread-doWork.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/graffles/ControllerEventThread-doWork.graffle -------------------------------------------------------------------------------- /graffles/GroupCoordinator-startup.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/graffles/GroupCoordinator-startup.graffle -------------------------------------------------------------------------------- /graffles/GroupMetadataManager-creating-instance.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/graffles/GroupMetadataManager-creating-instance.graffle -------------------------------------------------------------------------------- /graffles/KafkaApis.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/graffles/KafkaApis.graffle -------------------------------------------------------------------------------- /graffles/KafkaController.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/graffles/KafkaController.graffle -------------------------------------------------------------------------------- /graffles/LogManager.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/graffles/LogManager.graffle -------------------------------------------------------------------------------- /graffles/ZkPartitionStateMachine.graffle: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/japila-books/kafka-internals/7d3662058deedbeccc639803dbd41277b645271b/graffles/ZkPartitionStateMachine.graffle -------------------------------------------------------------------------------- /requirements.txt: -------------------------------------------------------------------------------- 1 | git+https://${GH_TOKEN}@github.com/squidfunk/mkdocs-material-insiders.git 2 | mkdocs-minify-plugin>=0.3.0 3 | mkdocs-git-revision-date-localized-plugin>=0.8 4 | mkdocs-git-revision-date-plugin>=0.3.1 5 | mkdocs-awesome-pages-plugin>=2.5.0 6 | mkdocs-redirects>=1.0.1 7 | mkdocs-macros-plugin>=0.5.0 8 | --------------------------------------------------------------------------------