This page describes a proposed Kafka Improvement Proposal (KIP) process for proposing a major change to Kafka.

To create your own KIP, click on . If you don't have permission, please send an email with your Wiki ID to dev@kafka.apache.org and request permission (http://kafka.apache.org/contact). Also add an entry to the table KIPs under discussion (for Streams API KIPs, please also add it to Kafka Streams sub page).

Purpose

We want to make Kafka a core architectural component for users. We also support a large number of integrations with other tools, systems, and clients. Keeping this kind of usage health requires a high level of compatibility between releases — core architectural elements can't break compatibility or shift functionality from release to release. As a result each new major feature or public api has to be done in a way that we can stick with it going forward.

This means when making this kind of change we need to think through what we are doing as best we can prior to release. And as we go forward we need to stick to our decisions as much as possible. All technical decisions have pros and cons so it is important we capture the thought process that lead to a decision or design to avoid flip-flopping needlessly.

Hopefully we can make these proportional in effort to their magnitude — small changes should just need a couple brief paragraphs, whereas large changes need detailed design discussions.

This process also isn't meant to discourage incompatible changes — proposing an incompatible change is totally legitimate. Sometimes we will have made a mistake and the best path forward is a clean break that cleans things up and gives us a good foundation going forward. Rather this is intended to avoid accidentally introducing half thought-out interfaces and protocols that cause needless heartburn when changed. Likewise the definition of "compatible" is itself squishy: small details like which errors are thrown when are clearly part of the contract but may need to change in some circumstances, likewise performance isn't part of the public contract but dramatic changes may break use cases. So we just need to use good judgement about how big the impact of an incompatibility will be and how big the payoff is.

What is considered a "major change" that needs a KIP?

Any of the following should be considered a major change:

What are the "public interfaces" of the project?

All of the following are public interfaces that people build around:

Not all compatibility commitments are the same. We need to spend significantly more time on log format and protocol as these break code in lots of clients, cause downtime releases, etc. Public apis are next as they cause people to rebuild code and lead to compatibility issues in large multi-dependency projects (which end up requiring multiple incompatible versions). Configuration, monitoring, and command line tools can be faster and looser — changes here will break monitoring dashboards and require a bit of care during upgrades but aren't a huge burden.

For the most part monitoring, command line tool changes, and configs are added with new features so these can be done with a single KIP.

What should be included in a KIP?

A KIP should contain the following sections:

Who should initiate the KIP?

Anyone can initiate a KIP but you shouldn't do it unless you have an intention of getting the work done to implement it (otherwise it is silly).

Process

Here is the process for making a KIP:

  1. Click . Take the next available KIP number and give your proposal a descriptive heading. e.g. "KIP 42: Allow Infinite Retention With Bounded Disk Usage".
  2. Fill in the sections as described above
  3. Start a [DISCUSS] thread on the Apache mailing list. Please ensure that the subject of the thread is of the format [DISCUSS] KIP-{your KIP number} {your KIP heading} The discussion should happen on the mailing list not on the wiki since the wiki comment system doesn't work well for larger discussions. In the process of the discussion you may update the proposal. You should let people know the changes you are making. When you feel you have a finalized proposal 
  4. Once the proposal is finalized call a [VOTE] to have the proposal adopted. These proposals are more serious than code changes and more serious even than release votes. The criteria for acceptance is lazy majority. The vote should remain open for at least 72 hours.
  5. Please update the KIP wiki page, and the index below, to reflect the current stage of the KIP after a vote. This acts as the permanent record indicating the result of the KIP (e.g., Accepted or Rejected). Also report the result of the KIP vote to the voting thread on the mailing list so the conclusion is clear.

KIP round-up

Next KIP Number: 442

Use this number as the identifier for your KIP and increment this value.

Adopted KIPs

Please insert new rows in sorted order (by KIP number).


KIP

Release

1KIP-436: Add a metric indicating start time2.3.0 (WIP)
2KIP-415: Incremental Cooperative Rebalancing in Kafka Connect2.3.0 (WIP)
3KIP-427: Add AtMinIsr topic partition category (new metric & TopicCommand option)2.3.0 (WIP)
4KIP-430 - Return Authorized Operations in Describe Responses2.3.0 (WIP)
5KIP-428: Add in-memory window store2.3.0
6KIP-425: Add some Log4J Kafka Appender Properties for Producing to Secured Brokers2.3.0
7KIP-420: Add Single Value Fetch in Session Stores2.2.0
8KIP-412: Extend Admin API to support dynamic application log levels2.3.0 (WIP)
9KIP-414: Expose Embedded ClientIds in Kafka Streams2.2.0
10KIP-402: Improve fairness in SocketServer processors2.2.0 (WIP)
11KIP-394: Require member.id for initial join group request2.2.0
12KIP-393: Time windowed serde to properly deserialize changelog input topic2.2.0
13KIP-386: Standardize on Min/Avg/Max metrics' default value2.2.0
14KIP-382: MirrorMaker 2.02.3.0 (WIP)
15KIP-380: Detect outdated control requests and bounced brokers using broker generation2.2.0
16KIP-377: TopicCommand to use AdminClient2.2.0
17KIP-376: Implement AutoClosable on appropriate classes that want to be used in a try-with-resource statement2.2.0
18KIP-374: Add '--help' option to all available Kafka CLI commands2.2.0
19KIP-372: Naming Repartition Topics for Joins and Grouping2.1.0
20KIP-371: Add a configuration to build custom SSL principal name2.2.0
21KIP 368: Allow SASL Connections to Periodically Re-Authenticate2.2.0
22KIP-367 Introduce close(Duration) to Producer and AdminClient instead of close(long, TimeUnit)2.2.0
23KIP-366: Make FunctionConversions deprecated2.1.0
24KIP-365: Materialized, Serialized, Joined, Consumed and Produced with implicit Serde2.1.0
25KIP-361: Add Consumer Configuration to Disable Auto Topic Creation2.3.0 (WIP)
26KIP-359: Verify leader epoch in produce requests2.3.0 (WIP)
27KIP-358: Migrate Streams API to Duration instead of long ms times2.1.0
28KIP-357: Add support to list ACLs per principal2.1.0
29KIP-356: Add withCachingDisabled() to StoreBuilder2.1.0
30KIP-353: Improve Kafka Streams Timestamp Synchronization2.1.0
31KIP-351: Add --under-min-isr option to describe topics command2.3.0
32KIP-346: Improve LogCleaner behavior on error2.1
33KIP-342 Add support for custom SASL extensions in OAuthBearer authentication2.1.0
34KIP-341: Update Sticky Assignor's User Data Protocol2.3.0 (WIP)
35KIP-340: Allow kafka-reassign-partitions.sh and kafka-log-dirs.sh to take admin client property file2.1.0
36KIP-339: Create a new IncrementalAlterConfigs API2.3.0 (WIP)
37KIP-338 Support to exclude the internal topics in kafka-topics.sh command2.1.0
38KIP-336: Consolidate ExtendedSerializer/Serializer and ExtendedDeserializer/Deserializer2.1.0
39KIP-332: Update AclCommand to use AdminClient API2.1.0
40KIP-331 Add default implementation to close() and configure() for Serializer, Deserializer and Serde2.3.0 (WIP)
41KIP-330: Add retentionPeriod in SessionBytesStoreSupplier2.1.0
42KIP-328: Ability to suppress updates for KTables2.1.0 (partially implemented) / 2.3.0 (WIP)
43KIP-324: Add method to get metrics() in AdminClient2.1.0
44KIP-322: Return new error code for DeleteTopics API when topic deletion disabled.2.1.0
45KIP-321: Update TopologyDescription to better represent Source and Sink Nodes2.1.0
46KIP-320: Allow fetchers to detect and handle log truncation2.1.0 (partially implemented) / 2.2.0
47KIP-319: Replace segments with segmentInterval in WindowBytesStoreSupplier2.1.0
48KIP-313: Add KStream.flatTransform and KStream.flatTransformValues 

2.2.0 (partially implemented)

/ 2.3.0 (WIP)

49KIP-312 Add Overloaded StreamsBuilder Build Method to Accept java.util.Properties2.1.0
50KIP-308: Support dynamic update of max.connections.per.ip/max.connections.per.ip.overrides2.1.0
51KIP-306: Configuration for Delaying Response to Failed Authentication2.1.0
52KIP-305: Add Connect primitive number converters2.0.0
53KIP-303: Add Dynamic Routing in Streams Sink2.0.0
54KIP-302 - Enable Kafka clients to use all DNS resolved IP addresses2.1.0
55KIP-298: Error Handling in Connect2.0.0
56KIP-297: Externalizing Secrets for Connect Configurations2.0.0
57KIP-295 Add Streams Configuration Allowing for Optional Topology Optimization2.0.0
58KIP-294 - Enable TLS hostname verification by default2.0.0
59KIP-292: Add transformValues() method to KTable2.0.0
60KIP-290: Support for Prefixed ACLs2.0.0
61KIP-289: Improve the default group id behavior in KafkaConsumer2.2.0
62KIP-285: Connect Rest Extension Plugin2.0.0
63KIP-284: Set default retention ms for Streams repartition topics to Long.MAX_VALUE2.0.0
64KIP-283: Efficient Memory Usage for Down-Conversion2.0.0
65KIP-282: Add the listener name to the authentication context2.0.0
66KIP-281: ConsumerPerformance: Increase Polling Loop Timeout and Make It Reachable by the End User2.0.0
67KIP-279: Fix log divergence between leader and follower after fast leader fail over2.0.0
68KIP-278 - Add version option to Kafka's commands2.0.0
69KIP-277 - Fine Grained ACL for CreateTopics API2.0.0
70KIP-276 - Add StreamsConfig prefix for different consumers2.0.0
71KIP-274: Kafka Streams Skipped Records Metrics2.0.0
72KIP-272: Add API version tag to broker's RequestsPerSec metric2.0.0
73KIP-270 - A Scala Wrapper Library for Kafka Streams2.0.0
74KIP-268: Simplify Kafka Streams Rebalance Metadata Upgrade2.0.0
75KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils2.0.0
76KIP-266: Fix consumer indefinite blocking behavior2.0.0
77KIP-265: Make Windowed Serde to public APIs2.0.0
78KIP-261: Add Single Value Fetch in Window Stores2.0.0
79KIP-258: Allow to Store Record Timestamps in RocksDB2.3.0
80KIP-257 - Configurable Quota Management2.0.0
81KIP-255: OAuth Authentication via SASL/OAUTHBEARER2.0.0
82KIP-251: Allow timestamp manipulation in Processor API2.0.0
83KIP-249: Add Delegation Token Operations to KafkaAdminClient2.0.0
84KIP-247: Add public test utils for Kafka Streams1.1.0
85KIP-245: Use Properties instead of StreamsConfig in KafkaStreams constructor2.0.0
86KIP-244: Add Record Header support to Kafka Streams Processor API2.0.0
87KIP-243: Make ProducerConfig and ConsumerConfig constructors public1.1.0
88KIP-239 Add queryableStoreName() to GlobalKTable1.1.0
89KIP-238: Expose Kafka cluster ID in Connect REST API1.1.0
90KIP-237: More Controller Health Metrics2.0.0
91KIP-235: Add DNS alias support for secured connection2.1.0
92KIP-233: Simplify StreamsBuilder#addGlobalStore1.1.0
93KIP-231: Improve the Required ACL of ListGroups API2.1.0
94KIP-229: DeleteGroups API1.1.0
95KIP-227 - Introduce Incremental FetchRequests to Increase Partition Scalability1.1.0
96KIP-226 - Dynamic Broker Configuration1.1.0
97KIP-225 - Use tags for consumer “records.lag” metrics1.1.0
98KIP-224: Add configuration parameter `retries` to Streams API1.1.0
99KIP-223 - Add per-topic min lead and per-partition lead metrics to KafkaConsumer2.0.0
100KIP-222 - Add Consumer Group operations to Admin API2.0.0
101KIP-220: Add AdminClient into Kafka Streams' ClientSupplier1.1.0
102KIP-219 - Improve quota communication2.0.0
103KIP-218: Make KafkaFuture.Function java 8 lambda compatible1.1.0
104KIP-215: Add topic regex support for Connect sinks1.1.0
105KIP-214: Add zookeeper.max.in.flight.requests config to the broker1.1.0
106KIP-213 Support non-key joining in KTable2.3.0 (WIP)
107KIP-212: Enforce set of legal characters for connector names1.1.0
108KIP-211: Revise Expiration Semantics of Consumer Group Offsets2.1.0
109KIP-210 - Provide for custom error handling when Kafka Streams fails to produce1.1.0
110KIP-208: Add SSL support to Kafka Connect REST interface1.1.0
111KIP-207: Offsets returned by ListOffsetsResponse should be monotonically increasing even during a partition leader change2.2.0
112KIP-206: Add support for UUID serialization and deserialization2.1.0
113KIP-205: Add all() and range() API to ReadOnlyWindowStore1.1.0
114KIP-204 : Adding records deletion operation to the new Admin Client API1.1.0
115KIP-203: Add toLowerCase support to sasl.kerberos.principal.to.local rule 1.1.0
116KIP-202: Move merge() from StreamsBuilder to KStream1.0.0
117KIP-198: Remove ZK dependency from Streams Reset Tool1.0.0
118KIP-197 Connect REST API should include the connector type when describing a connector1.0.0
119KIP-196: Add metrics to Kafka Connect framework1.0.0
120KIP-195: AdminClient.createPartitions1.0.0
121KIP-192 : Provide cleaner semantics when idempotence is enabled1.0.0
122KIP-191: KafkaConsumer.subscribe() overload that takes just Pattern1.0.0
123KIP-190: Handle client-ids consistently between clients and brokers1.0.0
124KIP-189: Improve principal builder interface and add support for SASL1.0.0
125KIP-188 - Add new metrics to support health checks1.0.0
126KIP-187 - Add cumulative count metric for all Kafka rate metrics1.0.0
127KIP-186: Increase offsets retention default to 7 days2.0.0
128KIP-183 - Change PreferredReplicaLeaderElectionCommand to use AdminClient2.2.0
129KIP-182: Reduce Streams DSL overloads and allow easier use of custom storage engines1.0.0
130KIP-180: Add a broker metric specifying the number of consumer group rebalances in progress1.1.0
131KIP-177: Consumer perf tool should count rebalance time1.0.0
132KIP-176: Remove deprecated new-consumer option for tools2.0.0
133KIP-175: Additional '--describe' views for ConsumerGroupCommand1.1.0
134KIP-174 - Deprecate and remove internal converter configs in WorkerConfig2.0.0
135KIP-173: Add prefix to StreamsConfig to enable setting default internal topic configs1.0.0
136KIP-171 - Extend Consumer Group Reset Offset for Stream Application1.1.0
137KIP-168: Add GlobalTopicCount and GlobalPartitionCount metric per cluster1.0.0
138KIP-167: Add interface for the state store restoration process1.0.0
139

KIP-164 - Add UnderMinIsrPartitionCount and per-partition UnderMinIsr metrics

1.0.0
140

KIP-163: Lower the Minimum Required ACL Permission of OffsetFetch

1.0.0
141KIP-162: Enable topic deletion by default1.0.0
142KIP-161: streams deserialization exception handlers1.0.0
143KIP-160: Augment KStream.print(), KStream.writeAsText() to allow users pass in extra parameters in the printed string1.0.0
144KIP-157 - Add consumer config options to streams reset tool1.0.0
145KIP-156 Add option "dry run" to Streams application reset tool0.11.0.0
146KIP-155 - Add range scan for windowed state stores0.11.0.0
147KIP-154 Add Kafka Connect configuration properties for creating internal topics0.11.0.0
148KIP-153: Include only client traffic in BytesOutPerSec metric0.11.0.0
149KIP-152 - Improve diagnostics for SASL authentication failures1.0.0
150KIP-151 Expose Connector type in REST API0.11.0.0
151KIP-150 - Kafka-Streams Cogroup2.3.0 (WIP)
152KIP-149: Enabling key access in ValueTransformer, ValueMapper, and ValueJoiner1.1.0 (partially implemented)
(WIP for 2.3.0)
153KIP-146 - Classloading Isolation in Connect

0.11.0.0

154KIP-145 - Expose Record Headers in Kafka Connect1.1.0
155KIP-144: Exponential backoff for broker reconnect attempts0.11.0.0
156KIP-143: Controller Health Metrics0.11.0.0
157KIP-140: Add administrative RPCs for adding, deleting, and listing ACLs0.11.0.0
158KIP-138: Change punctuate semantics1.0.0
159KIP-137: Enhance TopicCommand --describe to show topics marked for deletion0.11.0.0
160KIP-136: Add Listener name to SelectorMetrics tags0.11.0.0
161KIP-134: Delay initial consumer group rebalance0.11.0.0
162KIP-133: Describe and Alter Configs Admin APIs0.11.0.0
163KIP-130: Expose states of active tasks to KafkaStreams public API1.0.0
164KIP-129: Streams Exactly-Once Semantics0.11.0.0
165KIP-128: Add ByteArrayConverter for Kafka Connect0.11.0.0
166KIP-126 - Allow KafkaProducer to split and resend oversized batches.0.11.0.0
167KIP-124 - Request rate quotas0.11.0.0
168KIP-123: Allow per stream/table timestamp extractor0.11.0.0
169KIP-122: Add Reset Consumer Group Offsets tooling0.11.0.0
170KIP-121: Add KStream peek method

0.11.0.0

171KIP-120: Cleanup Kafka Streams builder API1.0.0
172KIP-119: Drop Support for Scala 2.10 in Kafka 0.110.11.0.0
173KIP-118: Drop Support for Java 72.0.0
174KIP-117: Add a public AdminClient API for Kafka admin operations0.11.0.0
175KIP-115: Enforce offsets.topic.replication.factor upon __consumer_offsets auto topic creation0.11.0.0
176KIP-114: KTable state stores and improved semantics0.11.0.0
177KIP-113: Support replicas movement between log directories1.1.0
178KIP-112: Handle disk failure for JBOD1.0.0
179KIP-110: Add Codec for ZStandard Compression2.1.0
180KIP-109: Old Consumer Deprecation0.11.0.0
181KIP-108: Create Topic Policy0.10.2.0
182KIP-107: Add deleteRecordsBefore() API in AdminClient0.11.0.0
183

KIP-106 - Change Default unclean.leader.election.enabled from True to False

0.11.0.0
184KIP-105: Addition of Record Level for Sensors0.10.2.0
185KIP-104: Granular Sensors for Streams 0.10.2.0
186KIP-103: Separation of Internal and External traffic0.10.2.0
187KIP-102 - Add close with timeout for consumers0.10.2.0
188KIP-101 - Alter Replication Protocol to use Leader Epoch rather than High Watermark for Truncation0.11.0.0
189KIP-100 - Relax Type constraints in Kafka Streams API0.10.2.0
190KIP-99: Add Global Tables to Kafka Streams0.10.2.0
191KIP-98 - Exactly Once Delivery and Transactional Messaging0.11.0.0
192KIP-97: Improved Kafka Client RPC Compatibility Policy0.10.2.0
193KIP-96 - Add per partition metrics for in-sync and assigned replica count0.10.2.0
194KIP-94 Session Windows0.10.2.0
195KIP-93: Improve invalid timestamp handling in Kafka Streams0.10.2.0
196KIP-92 - Add per partition lag metrics to KafkaConsumer0.10.2.0
197KIP-91 Provide Intuitive User Timeouts in The Producer2.1.0
198KIP-90 - Remove zkClient dependency from Streams0.10.2.0
199KIP-89: Allow sink connectors to decouple flush and offset commit0.10.2.0
200KIP-88: OffsetFetch Protocol Update0.10.2.0
201KIP-86: Configurable SASL callback handlers2.0.0
202KIP-85: Dynamic JAAS configuration for Kafka clients0.10.2.0
203KIP-84: Support SASL SCRAM mechanisms0.10.2.0
204KIP-82 - Add Record Headers0.11.0.0
205KIP-81: Bound Fetch memory usage in the consumer2.2.0 (WIP)
206KIP-79 - ListOffsetRequest/ListOffsetResponse v1 and add timestamp search methods to the new consumer0.10.1.0
207KIP-78: Cluster Id0.10.1.0
208KIP-77: Improve Kafka Streams Join Semantics0.10.2.0
209KIP-75 - Add per-connector Converters0.10.1.0
210KIP-74: Add Fetch Response Size Limit in Bytes0.10.1.0
211KIP-73: Replication Quotas0.10.1.0
212KIP-72: Allow putting a bound on memory consumed by Incoming request 1.0.0
213KIP-71: Enable log compaction and deletion to co-exist0.10.1.0
214KIP-70: Revise Partition Assignment Semantics on New Consumer's Subscription Change0.10.1.0
215KIP-67: Queryable state for Kafka Streams0.10.1.0
216KIP-66: Single Message Transforms for Kafka Connect0.10.2.0 / 0.11.0.0
217KIP-65: Expose timestamps to Connect0.10.1.0
218KIP-63: Unify store and downstream caching in streams0.10.1.0
219KIP-62: Allow consumer to send heartbeats from a background thread0.10.1.0
220KIP-60 - Make Java client classloading more flexible0.10.1.0
221KIP-58 - Make Log Compaction Point Configurable0.10.1.0
222KIP-57 - Interoperable LZ4 Framing0.10.0.0
223KIP-56: Allow cross origin HTTP requests on all HTTP methods0.10.0.0
224KIP-55: Secure Quotas for Authenticated Users0.10.1.0
225KIP-54: Sticky Partition Assignment Strategy0.11.0.0
226KIP-52: Connector Control APIs0.10.0.0
227KIP-51 - List Connectors REST API0.10.0.0
228KIP-50 - Move Authorizer to o.a.k.common package0.10.1.0
229KIP-48 Delegation token support for Kafka1.1.0
230KIP-45 - Standardize all client sequence interaction on j.u.Collection.0.10.0.0
231KIP-43: Kafka SASL enhancements0.10.0.0
232KIP-42: Add Producer and Consumer Interceptors0.10.0.0
233KIP-41: Consumer Max Records0.10.0.0
234KIP-40: ListGroups and DescribeGroup0.9.0.0
235KIP-38: ZooKeeper Authentication0.9.0.0
236KIP-36 - Rack aware replica assignment0.10.0.0
237KIP-35 - Retrieving protocol version0.10.0.0
238KIP-33 - Add a time based log index0.10.1.0
239KIP-32 - Add timestamps to Kafka message0.10.0.0
240KIP-31 - Move to relative offsets in compressed message sets0.10.0.0
241KIP-28 - Add a processor client0.10.0.0
242KIP-26 - Add Kafka Connect framework for data import/export0.9.0.0
243KIP-25 - System test improvements0.9.0.0
244KIP-22 - Expose a Partitioner interface in the new producer0.9.0.0
245KIP-21 - Dynamic Configuration0.9.0.0 (WIP)
246KIP-20 Enable log preallocate to improve consume performance under windows and some old Linux file system0.9.0.0
247KIP-19 - Add a request timeout to NetworkClient0.9.0.0
248KIP-16 - Automated Replica Lag Tuning0.9.0.0
249KIP-15 - Add a close method with a timeout in the producer0.9.0.0
250KIP-13 - Quota Design0.9.0.0
251KIP-12 - Kafka Sasl/Kerberos and SSL implementation0.9.0.0
252KIP-11 - Kafka Authorizer design0.9.0.0
253KIP-8 - Add a flush method to the producer API0.9.0.0
254KIP-4 - Metadata Protocol Changes0.10.0.0
255KIP-4 - Command line and centralized administrative operations0.9.0.0, 0.10.0.0, 0.10.1.0
256KIP-3 - Mirror Maker Enhancement0.9.0.0
257KIP-2 - Refactor brokers to allow listening on multiple ports and IPs0.9.0.0
258KIP-1 - Remove support of request.required.acks0.9.0.0


KIPs under discussion

Please insert new rows in sorted order (by KIP number).

KIPComment
KIP-59: Proposal for a kafka broker commandSent emails to Dev discussion group. Work tracked under KAFKA-3663.
KIP-125: ZookeeperConsumerConnector to KafkaConsumer Migration and Rollback
KIP-131 - Add access to OffsetStorageReader from SourceConnector
KIP-135 : Send of null key to a compacted topic should throw non-retriable error back to user
KIP 141 - ProducerRecord & SourceRecord: Add timestamp constructors
KIP-142: Add ListTopicsRequest to efficiently list all the topics in a cluster
KIP-148: Add a connect timeout for client
KIP-158: Kafka Connect should allow source connectors to set topic-specific settings for new topics
KIP-159: Introducing Rich functions to Streams
KIP-166 - Add a tool to make amounts of replicas and leaders on brokers balanced
KIP-169 - Lag-Aware Partition Assignment Strategy
KIP-178: Size-based log directory selection strategy


KIP-185: Make exactly once in order delivery the default producer setting


KIP-193: Add SchemaBuilder.from(Schema)
KIP-199: Add Kafka Connect offset tool
KIP-201: Rationalising Policy interfaces
KIP-209: Connection String Support
KIP-216: IQ should throw different exceptions for different errors
KIP-217: Expose a timeout to allow an expired ZK session to be re-created
KIP-221: Enhance KStream with Connecting Topic Creation and Repartition HintDiscussion
KIP-228 Negative record timestamp supportVoting in progress
KIP-234: add support for getting topic defaults from AdminClient
KIP-236: Interruptible Partition Reassignment Discussion
KIP-240: AdminClient.listReassignments() AdminClient.describeReassignments()
KIP-242: Mask password in Kafka Connect Rest API response
KIP-250 Add Support for Quorum-based Producer Acknowledgment
KIP-252 - Extend ACLs to allow filtering based on ip ranges and subnets
KIP-253: Support in-order message delivery with partition expansion Discussion
KIP-254: JsonConverter Exception Handeling
KIP-259: Improve Streams DSL Timestamp Propagation Semantics
KIP-260: add primary join operation for Stream-Stream join (WIP)Draft
KIP-264: Add a consumer metric to record raw fetch sizeVoting in progress
KIP-271: Add NetworkClient redirectorDiscussion
KIP-273: Kafka to support using ETCD beside ZookeeperDiscussion
KIP-275 - Indicate "isClosing" in the SinkTaskContextVoting in progress
KIP-280: Enhanced log compactionDiscussion

KIP-291: Separating controller connections and requests from the data plane

Accepted
KIP-293 Add new metrics for consumer/replication fetch requestsVoting in progress
KIP-296: Add connector level configurability for producer/consumer client configs Discussion
KIP-300: Add Windowed KTable API in StreamsBuilderDiscussion
KIP-301: Schema Inferencing for JsonConverterDiscussion
KIP-304: Connect runtime mode improvements for container platformsDiscussion
KIP-307: Allow to define custom processor names with KStreams DSLDiscussion
KIP-314: KTable to GlobalKTable Bi-directional JoinDiscussion
KIP-315: Stream Join Sticky AssignorDiscussion
KIP-316: Command-line overrides for ConnectDistributed worker propertiesDiscussion
KIP-317: Add transparent data encryption functionalityDiscussion
KIP-320: Allow fetchers to detect and handle log truncationDiscussion
KIP-325: Extend Consumer Group Command to Show Beginning OffsetsVoting in Progress
KIP-326: Schedulable KTable as Graph sourceDiscussion
KIP-333: Add faster mode of rebalancingDiscussion
KIP-334 - Include partitions in exceptions raised during consumer record deserialization/validation
KIP-335: Consider configurations for KafkaStreams

Discussion

KIP-345: Introduce static membership protocol to reduce consumer rebalancesAccepted
KIP-347: Enable batching in FindCoordinatorRequestDiscussion
KIP-348 Eliminate null from SourceTask#poll()
KIP-350: Allow kafka-topics.sh to take brokerid as parameter to show partitions associated with it
KIP-354: Add a Maximum Log Compaction LagAccepted
KIP-356: Add KafkaConsumer fetch-error-rate and fetch-error-total metricsDiscussion
KIP-360: Improve handling of unknown producerDiscussion
KIP-362: Support Dynamic Session WindowDiscussion
KIP-363: Allow performance tools to print final results to output fileDiscussion
KIP-369: Alternative Partitioner to Support "Always Round-Robin" SelectionDiscussion
KIP-370: Remove Orphan PartitionsDiscussion
KIP-373: Allow users to create delegation tokens for other usersDiscussion
KIP-375: Kafka Clients - make Metadata#TOPIC_EXPIRY_MS configurableDiscussion
KIP-378: Enable Dependency Injection for Kafka Streams handlersDiscussion

KIP-379: Multiple Consumer Group Management

Accepted
KIP-381: Connect: Tell about records that had their offsets flushed in callback

Voting in progress (restarted 18th January 2019, due to no votes in first attempt)

KIP-383:  Pluggable interface for SSL FactoryVoting in progress
KIP-384: Add config for incompatible changes to persistent metadataDiscussion
KIP-385: Avoid throwing away prefetched dataDiscussion
KIP-387: Fair Message Consumption Across Partitions in KafkaConsumer

Discussion

KIP-388: Add observer interface to record request and responseDiscussion
KIP-389: Introduce a configurable consumer group size limitAccepted
KIP-390: Allow fine-grained configuration for compressionDiscussion, JIRA exists with pull-request
KIP-391: Allow Producing with Offsets for Cluster ReplicationDiscussion, JIRA exists with pull-request
KIP-392: Allow consumers to fetch from closest replicaDiscussion
KIP-395: Encypt-then-MAC Delegation token metadata

KIP-396: Add Commit/List Offsets Operations to AdminClient

Discussion
KIP-397: Add methods to override fetch offsets based on timestampDiscussion
KIP-398: Support reading trust store from classpathDiscussion
KIP-399: Extend ProductionExceptionHandler to cover serialization exceptionsDiscussion
KIP-400: Improve exit status in case of errors in ConsoleProducerDiscussion
KIP-401: TransformerSupplier/ProcessorSupplier EnhancementsDiscussion
KIP-403: Increase ProducerPerformance precision by using nanoTimeDraft
KIP-405: Kafka Tiered Storage

Discussion

KIP-406: GlobalStreamThread should honor custom offset policy.Discussion
KIP-407: Kafka Connect support override worker kafka api configuration with connector configuration that post by rest api
KIP-408: Add asynchronous processing to Kafka StreamsDiscussion

KIP-409: Allow creating under-replicated topics and partitions


KIP-410: Add metric for request handler thread pool utilization by request type

Discussion

KIP-411: Make default Kafka Connect worker task client IDs distinct

Discussion
KIP-416: Notify SourceTask of ACK'd offsets, metadataDiscussion
KIP-417: Allow JmxTool to connect to a secured RMI portDiscussion, JIRA exists with pull-request
KIP-418: A method-chaining way to branch KStreamDiscussion, JIRA exists with pull-request
KIP-419: Safely notify Kafka Connect SourceTask is stoppedDiscussion
KIP-421: Support resolving externalized secrets in AbstractConfigDiscussion
KIP-422: Add support for user/client configuration in the Kafka Admin ClientDiscussion, JIRA exists with pull-request
KIP-423: Add JoinReason to Consumer Join Group Protocol

Discussion

KIP-424: Allow suppression of intermediate events based on wall clock timeDiscussion
KIP-426: Persist Broker Id to ZookeeperDiscussion
KIP-429: Kafka Consumer Incremental Rebalance ProtocolDiscussion
KIP-431: Support of printing additional ConsumerRecord fields in DefaultMessageFormatterDiscussion, JIRA exists with pull-request

KIP-432: Additional Broker-Side Opt-In for Default, Unsecure SASL/OAUTHBEARER Implementation

Discussion
KIP-434: Add Replica Fetcher and Log Cleaner Count MetricsDiscussion
KIP-435: Incremental Partition ReassignmentDiscussion
KIP-437: Custom replacement for MaskField SMTDiscussion, JIRA exists with pull-request
KIP-438: Expose task, connector IDs in Connect APIDiscussion
KIP-439: Deprecate Interface WindowStoreIteratorDiscussion
KIP-440: Extend Connect Converter to support headersDiscussion


Dormant/inactive KIPs

Please insert new rows in sorted order (by KIP number).

KIPComment
KIP-6 - New reassignment partition logic for rebalancingNeeds more detail
KIP-10 - Running Producer, Consumers and Brokers on Mesos
KIP-14 - Tools standardization
KIP-17 - Add HighwaterMarkOffset to OffsetFetchResponse
KIP-23 - Add JSON/CSV output and looping options to ConsumerGroupCommand
KIP-27 - Conditional Publish
KIP-30 - Allow for brokers to have plug-able consensus and meta data storage sub systems
KIP-37 - Add Namespaces to Kafka
KIP-39: Pinning controller to broker
KIP-44 - Allow Kafka to have a customized security protocol
KIP-46 - Self Healing
KIP-47 - Add timestamp-based log deletion policy
KIP-49 - Fair Partition Assignment Strategy
KIP-53 - Add custom policies for reconnect attempts to NetworkdClient
KIP-59: Proposal for a kafka broker command
KIP-64 -Allow underlying distributed filesystem to take over replication depending on configuration
KIP-68 Add a consumed log retention before log retention
KIP-69 - Kafka Schema RegistryDraft
KIP-76 Enable getting password from executable rather than passing as plaintext in config files
KIP-87 - Add Compaction Tombstone FlagPossibly displaced by KIP-82
KIP-95: Incremental Batch Processing for Kafka Streams
KIP-269: Substitution Within Configuration Values
KIP-349: Priorities for Source Topics

Functionality can be realized with existing API although at a lower level. 

Discarded KIPs

Please insert new rows in sorted order (by KIP number).

KIPComment
KIP-5 - Broker Configuration ManagementSuperseded by KIP-21
KIP-7 - Security - IP Filtering
KIP-9 - SSL SupportAdopted via KAFKA-1690 but not via this KIP process
KIP-18 - JBOD SupportSuperseded by KIP-112 and KIP-113 
KIP-24 - Remove ISR information from TopicMetadataRequest and add broker level metadata request
KIP-29 - Add an IsrPropagateIntervalMs configuration to KafkaConfigNo longer need after KAFKA-2722

KIP-34 Add Partitioner Change Listener to Partitioner Interface for Multiple Use Case

Per feedback it is better do this parition() method and avoid Thread Coordination etc.
KIP-80: Kafka Rest Server
KIP-83 - Allow multiple SASL authenticated Java clients in a single JVM processAfter KIP-85, no interface changes, reduced to KAFKA-4180
KIP-111: Kafka should preserve the Principal generated by the PrincipalBuilder while processing the request received on socket channel, on the broker.Covered by KIP-189
KIP-116: Add State Store Checkpoint Interval ConfigurationCurrently not needed as checkpointing can be done on commit interval.
KIP-127: Pluggable JAAS LoginModule configuration for SSLSimilar functionality can be implemented using a custom PrincipalBuilder.
KIP-132: Augment KStream.print to allow extra parameters in the printed string

Duplicated by KIP-160

KIP-147: Add missing type parameters to StateStoreSupplier factories and KGroupedStream/Table methodsCovered by KIP-182
KIP-165: Extend Interactive Queries for return latest update timestamp per keyCovered by KIP-258
KIP-170: Enhanced TopicCreatePolicy and introduction of TopicDeletePolicySuperseded by KIP-201: Rationalising Policy interfaces
KIP 172: Add regular-expression topic support for sink connectorSuperseded by KIP-215: Add topic regex support for Connect sinks
KIP-179: Change ReassignPartitionsCommand to use AdminClient
KIP-184: Rename LogCleaner and related classes to LogCompactorA lot of configuration change will cause more trouble and doesn't seems to be worth it
KIP 230: Name Windowing JoinsCovered by KIP-372: Naming Repartition Topics for Joins and Grouping
KIP-232: Detect outdated metadata using per-partition leaderEpoch fieldMerged into KIP-320: Allow fetchers to detect and handle log truncation
KIP-246: Connect producers and consumers should inherit worker configsWithdrawn because proposal could not be made backward compatible with existing behavior
KIP-248 - Create New ConfigCommand That Uses The New AdminClientDiscarded by author as it's partly covered in KIP-339 and KIP-422
KIP-262: Metadata should include number of state stores for taskRejected since metadata is no longer required.
KIP-263: Allow broker to skip sanity check of inactive segments on broker startupWithdrawn because the solution we agreed on does not require interface change
KIP-286: producer.send() should not block on metadata updateWithdrawn because the benefit of not having to wait for metadata is probably not worth the complexity added in producer.
KIP-288: [DISCARDED] Consumer.poll() timeout semantic change and new waitForAssignment methodDiscarded in deference to KIP-266
KIP-310: Add a Kafka Source Connector to Kafka ConnectWithdrawn in favor of KIP-382.
KIP-327: Add describe all topics API to AdminClientWithdrawn in-favor of filter support in Metadata API and KIP-142
KIP-344: The auto-generated client id should be passed to MetricsReporterBug fix so KIP not needed
KIP-364: Remove implicit Materialized, Consumed and ProducedSubsumed by KIP-365
KIP-404: Add Kafka Connect configuration parameter for disabling WADL output on OPTIONS request

Discarded, reported as a bug by

KIP Discussion Recordings

Date (link to recording)Summary
2018-04-09
  • KIP-253 - partition expansion: We discussed a few things. (1) Is it useful to backfill a compacted topic? The main use case is to rebuild the application states. If the new partition has the existing data, rebuilding the state can be done easily by reading from a single partition. Otherwise, an application has to read both the child and the parent partition to rebuild the state. This is possible, but can be complicated. Jan will do an exercise to see how complicated this is. (2) What's the best way to add the backfilling support if we want to do it? We can do this on the server side or on the client side. The former potentially makes the coordination easier. The latter potentially reduces the memory footprint on the server for reshuffling. We need to think through how to support EOS message format and how to throttle the process to avoid overwhelming the cluster. (3) Linear hashing vs doubling partitions. It seems that Linear hashing is more general. (4) Partition splitting in Kinesis. This is done differently since it doesn't allow customized partitioning. It doesn't support compacted topic either. (5) Sticky partition assignment. It could be useful to support a partition assignment strategy where the child partition is assigned together with the parent partition to a consumer instance so that the local state doesn't have to be moved immediately. (6) Consumer callback on partition splitting. This could still be useful if the states are maintained globally.
2017-01-07
  • KIP-112 - Handle disk failure for JBOD: We discussed whether we need to support JBOD directly in Kafka or just rely on the 1 disk per broker model. The general consensus is that direct JBOD support in Kafka is needed. There is some concern on the complexity added to Kafka. So, we have to be careful with the implementation details. We discussed how directory failure should be detected, where the failure state is kept, and whether the state should be reset on broker restart. There is a bit confusing on what's written in the wiki. Dong is going to clarify the proposal based on the feedback and we will follow up on the details in the mailing list.
2016-10-19
  • KIP-82 - add record header: We agreed that there are use cases for third-party vendors building tools around Kafka. We haven't reached the conclusion whether the added complexity justifies the use cases. We will follow up on the mailing list with use cases, container format people have been using, and details on the proposal.
2016-09-13
  • KIP-54 (Sticky Partition Assignment): aims to minimise partition movement so that resource reinitialisation (e.g. caches) is minimised. It is partially sticky and partially fair. Some concerns around the fact that user code for partitionsRevoked and partitionsAssigned would have to be changed to work correctly with this assignment strategy. Good: more complex usage of an assigner that takes advantage of the user data field. Vahid will start the vote.

  • KIP-72 (Allow Sizing Incoming Request Queue in Bytes): large requests can kill the broker, no control over how much memory is allocated. Client quotas don't help as damage may already have been done by the time they kick in. There was a discussion on whether it was worth it to avoid the immediate return from select when there was no memory available in the pool. Radai will update the KIP to describe this aspect in more detail as well as the config validation that is performed.

  • KIP-79 (ListOffsetRequest/ListOffsetResponse v1 and add timestamp search methods to the new consumer): we discussed the option of passing multiple timestamps for the same partition in the same request. Becket thinks it's a rare use case and not worth supporting. Gwen said that it would be nice to have, but not essential. We talked about validation of duplicate topics. Becket will check the approach taken by the create topics request and evaluate if it can be adopted here too. PR will be available today and Jason will evaluate if it's feasible to include it in the next release once it's available.

2016-08-30
  • KIP48 (delegation tokens): Harsha will update the wiki with more details on how to use delegation tokens and how to configure it.
  • KIP-78 (cluster id): There was discussion on adding human readable tags later. No major concerns.
2016-08-23
  • time-based release: No one seems to have objections. Ismael will follow up with a release wiki.
  • KIP-4: We discussed having separate ACL requests of add and delete. No one seems to object to it. We discussed the admin client. Grant will send a PR. We discussed how KStream can use the ACL api.  It seems that we will need some kind of regex or namespace support in ACL to make the authorization convenient in KStream.
  • KIP-50: There is some discussion for further changes in the PR. Ashish will reply to the KIP email thread with the recommended changes. Ashish/Grant plan to look into whether it's possible to make the authorizer api change backward compatible. However, it seems that people are in general ok with a non-compatible api change.
  • KIP-74: No objections on the current proposal.
  • Java 7 support timeline: The consensus is to defer dropping the Java 7 support until the next major release (which will be next year). Ismael will follow up on the email thread.
  • KIP-48 delegation token : Ashish will ping Harsh to see if this is still active.
  • Some of the KIPs have been idle. Grant will send a proposal on tagging them properly (e.g., blocked, inactive, no resource, etc).
2016-05-24
  • KIP-58 - Make Log Compaction Point Configurable: We want to start with just a time-based configuration since there is no good usage for byte-based or message-based configuration. Eric will change the KIP and start the vote.
  • KIP-4 - Admin api: Grant will pick up the work. Initially, he plans to route the write requests from the admin clients to the controller directly to avoid having the broker forward the requests to the controller.
  • KIP-48 - Delegation tokens: Two of the remaining issues are (1) how to store the delegation tokens and (2) how token expiration works. Since Parth wasn't able to attend the meeting. We will follow up in the mailing list.
2016-04-05
  • KIP-4: There is a slight debate on the metadata request schema, as well as the internal ZK based implementation, which we will wait for Jun to comment on the mailing list thread.
  • KIP-52: We decided to start a voting process for this.
  • KIP-35: Decided on renaming ApiVersionQuery api to ApiVersion. Consensus on using the api in java client to only check for availability of current versions. ApiVersion api's versions will not be deprecated. Update KIP-35 wiki will be updated with latest info and vote thread will be initiated.
2016-03-15
  • KIP-33 - Add a time based log index to Kafka: We decided NOT to include this in 0.10.0 since the changes may have performance risks.
  • KIP-45 - Standardize all client sequence interaction on j.u.Collection: There is no consensus in the discussion. We will just put it to vote.
  • KIP-35 - Retrieving protocol version: This gets the longest discussion. There is still no consensus. Magnus thinks the current proposal of maintaining a global protocol version won't work and will try to submit a new proposal.
  • KIP-43 - Kafka SASL enhancements: Rajini will modify the KIP to only support native SASL mechanisms and leave the changes to Login and CallbackHandler to KIP-44 instead.
2016-02-23
  • KIP-33 and KIP-47: No issues. Will start the voting thread.
  • KIP-43: We discussed whether there is a need to support multiple SASL mechanisms at the same time and what's the best way to implement this. Will discuss this in more details in the email thread.
  • KIP-4: Grant gave a comprehensive summary of the current state. We have gaps on how to make the admin request block on the broker, how to integrate admin requests with ACL (especially with respect to client config changes for throttling and ACL changes), how to do the alter topic request properly. Grant will update the KIP with an interim plan and a long term plan.
  • KIP-43: We briefly discussed on to support multiple sasl mechanisms on the broker. Harsha will follow up with more details on the email thread.
  • Everyone seems to be in favor of making the next major release 0.10.0, instead of 0.9.1.
2016-01-26
  • KIP-42: We agreed to leave the broker side interceptor for another KIP. On the client side, people favor the 2nd option in Anna's proposal. Anna will update the wiki accordingly.
  • KIP-43: We discussed whether there is a need to support multiple SASL mechanisms at the same time and what's the best way to implement this. Will discuss this in more details in the email thread.
  • Jiangjie brought up an issue related to KIP-32 (adding timestamp field in the message). The issue is that currently there is no convenient way for the consumer to tell whether the timestamp in a message is the create time or the server time. He and Guozhang propose to use a bit in the message attribute to do that. Jiangjie will describe the proposal in the email thread.
2016-01-12
  • KIP-41: Discussed whether the issue of long processing time between poll calls is a common issue and whether we should revisit the poll api. Also discussed whether the number of records returned in poll calls can be made more dynamic. In the end, we feel that just adding a config that controls the number records returned in poll() is the simplest approach at this moment.
  • KIP-36: Need to look into how to change the broker JSON representation in ZK w/o breaking rolling upgrades. Otherwise, ready for voting.
2015-10-20
  • KIP-38: No concerns with this KIP. Flavio will initiate the voting on this.
  • KIP-37: There are questions on how ACL, configurations, etc will work, and whether we should support "move" or not. We will discuss the details more in the mailing list.
  • KIP-32/KIP-33: Jiangjie raised some concerns on the approach that Jay proposed. Guozhang and Jay will follow up on the mailing list.
2015-10-13
  • 0.9.0 release: We discussed if KAFKA-2397 should be a blocker in 0.9.0. Jason and Guozhang will follow up on the jira.
  • KIP-32 and KIP-33: We discussed Jay's alternative proposal of just keeping CreateTime in the message and having a config to control how far off the CreateTime can be from the broker time. We will think a bit more on this and Jiangjie will update the KIP wiki.
  • KIP-36: We discussed an alternative approach of introducing a new broker property to designate the rack. It's simpler and potentially can work in the case when the broker to rack mapping is maintaining externally. We need to make sure that we have an upgrade plan for this change. Allen will update the KIP wiki
2015-10-06
  • We only had the time to go through KIP-35. The consensus is that we will add a BrokerProtocolRequest that returns the supported versions for every type of requests. It's up to the client to decide how to use this. Magnus will update the KIP wiki with more details.
2015-09-22
  • KIP-31: Need to figure out how to evolve inter.broker.protocol.version with multiple protocol changes within the same release, mostly for people who are deploying from trunk. Becket will update the wiki.
  • KIP-32/KIP-33: Having both CreateTime and LogAppendTime per message adds significant overtime. There are a couple of possibilities to improve this. Becket will follow up on this.
  • LinkedIn has been testing SSL in MirrorMaker (SSL is only enabled in the producer). So far, MirrorMaker can keep up with the load. LinkedIn folks will share some of the performance results.
2015-09-14
  • KIP-28: Discussed the improved proposal including 2 layers of API (the higher layer is for streaming DSL), and stream time vs processor time. Ready for review.
  • KIP-31, KIP-32: (1) Discussed whether the timestamp should be from the client or the broker. (2) Discussed the migration path and whether this requires all consumers to upgrade before the new message format can be used. (3) Since this is too big a change, it will NOT be included in 0.9.0 release. Becket will update the wiki.
2015-08-18
  • client-side assignment strategy: We discussed concerns about rebalancing time due to metadata inconsistency, especially when lots of topics are subscribed. Will discuss a bit more on the mailing list. 
  • CopyCat data api: The discussions are in KAFKA-2367 for people who are interested.
  • 0.8.2.2: We want to make this a low risk bug fix release since 0.8.3 is coming. So, will only include a small number of critical and small fixes.
  • 0.8.3: The main features will be security and the new consumer. We will be cutting a release branch when the major pieces for these new features have been committed.
2015-08-11