Release Notes - Kafka - Version 0.9.0.0
Sub-task
- [KAFKA-1316] - Refactor Sender
- [KAFKA-1328] - Add new consumer APIs
- [KAFKA-1329] - Add metadata fetch and refresh functionality to the consumer
- [KAFKA-1330] - Implement subscribe(TopicPartition...partitions) and pool(timeout) in the consumer
- [KAFKA-1331] - Add ability to commit offsets to the new consumer
- [KAFKA-1333] - Add consumer co-ordinator module to the server
- [KAFKA-1334] - Coordinator should detect consumer failures
- [KAFKA-1335] - Add rebalancing logic to the coordinator / consumer
- [KAFKA-1471] - Add Producer Unit Tests for LZ4 and LZ4HC compression
- [KAFKA-1477] - add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication
- [KAFKA-1683] - Implement a "session" concept in the socket server
- [KAFKA-1684] - Implement TLS/SSL authentication
- [KAFKA-1685] - Implement TLS/SSL tests
- [KAFKA-1686] - Implement SASL/Kerberos
- [KAFKA-1688] - Add authorization interface and naive implementation
- [KAFKA-1690] - Add SSL support to Kafka Broker, Producer and Consumer
- [KAFKA-1691] - new java consumer needs ssl support as a client
- [KAFKA-1695] - Authenticate connection to Zookeeper
- [KAFKA-1740] - Merge Offset manager into Coordinator
- [KAFKA-1750] - handle "topic not exists" scenario
- [KAFKA-1752] - add --replace-broker option
- [KAFKA-1760] - Implement new consumer client
- [KAFKA-1809] - Refactor brokers to allow listening on multiple ports and IPs
- [KAFKA-1845] - KafkaConfig should use ConfigDef
- [KAFKA-1893] - Allow regex subscriptions in the new consumer
- [KAFKA-1910] - Refactor KafkaConsumer
- [KAFKA-1914] - Count TotalProduceRequestRate and TotalFetchRequestRate in BrokerTopicMetrics
- [KAFKA-1925] - Coordinator Node Id set to INT_MAX breaks coordinator metadata updates
- [KAFKA-1928] - Move kafka.network over to using the network classes in org.apache.kafka.common.network
- [KAFKA-1943] - Producer request failure rate should not include MessageSetSizeTooLarge and MessageSizeTooLargeException
- [KAFKA-1953] - Disambiguate metrics from different purgatories
- [KAFKA-1986] - Producer request failure rate should not include InvalidMessageSizeException and OffsetOutOfRangeException
- [KAFKA-2015] - Enable ConsoleConsumer to use new consumer
- [KAFKA-2017] - Persist Coordinator State for Coordinator Failover
- [KAFKA-2018] - Add metadata to consumer registry info
- [KAFKA-2055] - ConsumerBounceTest.testSeekAndCommitWithBrokerFailures transient failure
- [KAFKA-2056] - PartitionAssignorTest.testRangePartitionAssignor transient failure
- [KAFKA-2065] - Add ControlledShutdown to org.apache.kafka.common.requests
- [KAFKA-2067] - Add LeaderAndISR request/response to org.apache.kafka.common.requests
- [KAFKA-2089] - MetadataTest transient failure
- [KAFKA-2123] - Make new consumer offset commit API use callback + future
- [KAFKA-2136] - Client side protocol changes to return quota delays
- [KAFKA-2168] - New consumer poll() can block other calls like position(), commit(), and close() indefinitely
- [KAFKA-2195] - Add versionId to AbstractRequest.getErrorResponse and AbstractRequest.getRequest
- [KAFKA-2205] - Generalize TopicConfigManager to handle multiple entity configs
- [KAFKA-2209] - Change client quotas dynamically using DynamicConfigManager
- [KAFKA-2210] - KafkaAuthorizer: Add all public entities, config changes and changes to KafkaAPI and kafkaServer to allow pluggable authorizer implementation.
- [KAFKA-2211] - KafkaAuthorizer: Add simpleACLAuthorizer implementation.
- [KAFKA-2212] - KafkaAuthorizer: Add CLI for Acl management.
- [KAFKA-2245] - Add response tests for ConsumerCoordinator
- [KAFKA-2258] - Port mirrormaker_testsuite
- [KAFKA-2271] - transient unit test failure in KafkaConfigConfigDefTest.testFromPropsToProps
- [KAFKA-2274] - Add integration test for consumer coordinator
- [KAFKA-2275] - Add a ListTopics() API to the new consumer
- [KAFKA-2301] - Deprecate ConsumerOffsetChecker
- [KAFKA-2332] - Add quota metrics to old producer and consumer
- [KAFKA-2342] - KafkaConsumer rebalance with in-flight fetch can cause invalid position
- [KAFKA-2366] - Initial patch for Copycat
- [KAFKA-2367] - Add Copycat runtime data API
- [KAFKA-2368] - Add Copycat standalone CLI
- [KAFKA-2369] - Add Copycat REST API
- [KAFKA-2371] - Add distributed coordinator implementation for Copycat
- [KAFKA-2372] - Copycat distributed config storage
- [KAFKA-2373] - Copycat distributed offset storage
- [KAFKA-2374] - Implement Copycat log/file connector
- [KAFKA-2377] - Add copycat system tests
- [KAFKA-2379] - Add Copycat documentation
- [KAFKA-2386] - Transient test failure: testGenerationIdIncrementsOnRebalance
- [KAFKA-2388] - subscribe(topic)/unsubscribe(topic) should either take a callback to allow user to handle exceptions or it should be synchronous.
- [KAFKA-2389] - CommitType seems not necessary in commit().
- [KAFKA-2397] - leave group request
- [KAFKA-2400] - Expose heartbeat frequency in new consumer configuration
- [KAFKA-2401] - Fix transient failure of ProducerSendTest.testCloseWithZeroTimeoutFromSenderThread()
- [KAFKA-2403] - Expose offset commit metadata in new consumer
- [KAFKA-2409] - Have KafkaConsumer.committed() return null when there is no committed offset
- [KAFKA-2411] - remove usage of BlockingChannel in the broker
- [KAFKA-2415] - Transient failure in LogRecoveryTest.testHWCheckpointWithFailuresMultipleLogSegments
- [KAFKA-2417] - Ducktape tests for SSL/TLS
- [KAFKA-2431] - Test SSL/TLS impact on performance
- [KAFKA-2439] - Add MirrorMakerService to ducktape system tests
- [KAFKA-2440] - Use `NetworkClient` instead of `SimpleConsumer` to fetch data from replica
- [KAFKA-2441] - SSL/TLS in official docs
- [KAFKA-2452] - enable new consumer in mirror maker
- [KAFKA-2453] - enable new consumer in EndToEndLatency
- [KAFKA-2456] - Disable SSLv3 for ssl.enabledprotocols config on client & broker side
- [KAFKA-2464] - Client-side assignment and group generalization
- [KAFKA-2474] - Add caching for converted Copycat schemas in JSONConverter
- [KAFKA-2475] - Reduce copycat configs to only specify a converter or serializer, not both
- [KAFKA-2476] - Define logical types for Copycat data API
- [KAFKA-2480] - Handle non-CopycatExceptions from SinkTasks
- [KAFKA-2481] - Allow copycat sinks to request periodic invocation of put even if no new data is available
- [KAFKA-2482] - Allow copycat sink tasks to pause/resume consumption of specific topic partitions
- [KAFKA-2484] - Add schema projection utilities
- [KAFKA-2486] - New consumer performance
- [KAFKA-2487] - change kafka.examples.Consumer to use the new java consumer
- [KAFKA-2490] - support new consumer in ConsumerGroupCommand
- [KAFKA-2491] - Update ErrorMapping with New Consumer Errors
- [KAFKA-2515] - handle oversized messages properly in new consumer
- [KAFKA-2527] - System Test for Quotas in Ducktape
- [KAFKA-2532] - Remove Consumer from rebalance callback arguments
- [KAFKA-2558] - ServerShutdownTest is failing intermittently
- [KAFKA-2574] - Add ducktape based ssl test for KafkaLog4jAppender
- [KAFKA-2579] - Unauthorized clients should not be able to join groups
- [KAFKA-2581] - Run some existing ducktape tests with SSL-enabled clients and brokers
- [KAFKA-2582] - ConsumerMetdata authorization error not returned to user
- [KAFKA-2587] - Transient test failure: `SimpleAclAuthorizerTest`
- [KAFKA-2598] - Add Test with authorizer for producer and consumer
- [KAFKA-2604] - Remove `completeAll` and improve timeout passed to `Selector.poll` from `NetworkClient.poll`
- [KAFKA-2613] - Consider capping `maxParallelForks` for Jenkins builds
- [KAFKA-2622] - Add Time logical type for Copycat
- [KAFKA-2626] - Null offsets in copycat causes exception in OffsetStorageWriter
- [KAFKA-2632] - Move fetchable check from fetchedRecords to fetch response handler
- [KAFKA-2639] - Refactoring of ZkUtils
- [KAFKA-2640] - Add tests for ZK authentication
- [KAFKA-2641] - Upgrade path for ZK authentication
- [KAFKA-2644] - Run relevant ducktape tests with SASL_PLAINTEXT and SASL_SSL
- [KAFKA-2675] - SASL/Kerberos follow-up
- [KAFKA-2681] - SASL authentication in official docs
- [KAFKA-2682] - Authorization section in official docs
- [KAFKA-2690] - Protect passwords from logging
- [KAFKA-2691] - Improve handling of authorization failure during metadata refresh
- [KAFKA-2697] - add leave group logic to the consumer
- [KAFKA-2711] - SaslClientAuthenticator no longer needs KerberosNameParser in constructor
- [KAFKA-2713] - Copycat worker should not call connector's/task's start methods in the control thread
- [KAFKA-2718] - Reuse of temporary directories leading to transient unit test failures
- [KAFKA-2724] - Document ZooKeeper authentication
- [KAFKA-2745] - Update JavaDoc for the new / updated APIs
- [KAFKA-2765] - Connectors and tasks should have versions that can be reported when they are instantiated
- [KAFKA-2774] - Rename Copycat -> Kafka Connect
Bug
- [KAFKA-328] - Write unit test for kafka server startup and shutdown API
- [KAFKA-742] - Existing directories under the Kafka data directory without any data cause process to not start
- [KAFKA-766] - Isr shrink/expand check is fragile
- [KAFKA-824] - java.lang.NullPointerException in commitOffsets
- [KAFKA-972] - MetadataRequest returns stale list of brokers
- [KAFKA-1057] - Trim whitespaces from user specified configs
- [KAFKA-1070] - Auto-assign node id
- [KAFKA-1082] - zkclient dies after UnknownHostException in zk reconnect
- [KAFKA-1109] - Need to fix GC log configuration code, not able to override KAFKA_GC_LOG_OPTS
- [KAFKA-1230] - shell script files under bin don't work with cygwin (bash on windows)
- [KAFKA-1282] - Disconnect idle socket connection in Selector
- [KAFKA-1367] - Broker topic metadata not kept in sync with ZooKeeper
- [KAFKA-1374] - LogCleaner (compaction) does not support compressed topics
- [KAFKA-1387] - Kafka getting stuck creating ephemeral node it has already created when two zookeeper sessions are established in a very short period of time
- [KAFKA-1465] - kafka-reassign-partitions.sh fails when topic name contains dash/hyphen
- [KAFKA-1481] - Stop using dashes AND underscores as separators in MBean names
- [KAFKA-1517] - Messages is a required argument to Producer Performance Test
- [KAFKA-1581] - Log cleaner should have an option to ignore messages without keys
- [KAFKA-1583] - Kafka API Refactoring
- [KAFKA-1634] - Improve semantics of timestamp in OffsetCommitRequests and update documentation
- [KAFKA-1641] - Log cleaner exits if last cleaned offset is lower than earliest offset
- [KAFKA-1648] - Round robin consumer balance throws an NPE when there are no topics
- [KAFKA-1667] - topic-level configuration not validated
- [KAFKA-1668] - TopicCommand doesn't warn if --topic argument doesn't match any topics
- [KAFKA-1679] - JmxTool outputs nothing if any mbean attributes can't be retrieved
- [KAFKA-1697] - remove code related to ack>1 on the broker
- [KAFKA-1698] - Validator.ensureValid() only validates default config value
- [KAFKA-1700] - examples directory - README and shell scripts are out of date
- [KAFKA-1702] - Messages silently Lost by producer
- [KAFKA-1711] - WARN Property topic is not valid when running console producer
- [KAFKA-1724] - Errors after reboot in single node setup
- [KAFKA-1727] - Fix comment about message format
- [KAFKA-1733] - Producer.send will block indeterminately when broker is unavailable.
- [KAFKA-1739] - Remove testComplexCompressDecompress in MessageCompressionTest
- [KAFKA-1746] - System tests don't handle errors well
- [KAFKA-1747] - TestcaseEnv improperly shares state between instances
- [KAFKA-1757] - Can not delete Topic index on Windows
- [KAFKA-1758] - corrupt recovery file prevents startup
- [KAFKA-1759] - transient unit test failure in PartitionAssignorTest
- [KAFKA-1762] - Update max-inflight-request doc string
- [KAFKA-1782] - Junit3 Misusage
- [KAFKA-1788] - producer record can stay in RecordAccumulator forever if leader is no available
- [KAFKA-1790] - Remote controlled shutdown was removed
- [KAFKA-1798] - ConfigDef.parseType() should throw exception on invalid boolean value
- [KAFKA-1803] - UncleanLeaderElectionEnableProp in LogConfig should be of boolean
- [KAFKA-1804] - Kafka network thread lacks top exception handler
- [KAFKA-1812] - Allow IpV6 in configuration with parseCsvMap
- [KAFKA-1813] - Build fails for scala 2.9.2
- [KAFKA-1815] - ServerShutdownTest fails in trunk.
- [KAFKA-1816] - Topic configs reset after partition increase
- [KAFKA-1824] - in ConsoleProducer - properties key.separator and parse.key no longer work
- [KAFKA-1836] - metadata.fetch.timeout.ms set to zero blocks forever
- [KAFKA-1848] - Checking shutdown during each iteration of ZookeeperConsumerConnector
- [KAFKA-1849] - Utils.readBytes() should support null ByteBuffer
- [KAFKA-1852] - OffsetCommitRequest can commit offset on unknown topic
- [KAFKA-1855] - Topic unusable after unsuccessful UpdateMetadataRequest
- [KAFKA-1865] - Add a flush() call to the new producer API
- [KAFKA-1866] - LogStartOffset gauge throws exceptions after log.delete()
- [KAFKA-1867] - liveBroker list not updated on a cluster with no topics
- [KAFKA-1877] - Expose version via JMX for 'new' producer
- [KAFKA-1878] - ProducerFailureHandlingTest.testCannotSendToInternalTopic fails with TimeoutException while trying to fetch metadata for topic
- [KAFKA-1881] - transient unit test failure in testDeleteTopicWithCleaner due to OOME
- [KAFKA-1883] - NullPointerException in RequestSendThread
- [KAFKA-1884] - Print metadata response errors
- [KAFKA-1890] - Fix bug preventing Mirror Maker from successful rebalance.
- [KAFKA-1891] - MirrorMaker hides consumer exception - making troubleshooting challenging
- [KAFKA-1896] - Record size funcition of record in mirror maker hit NPE when the message value is null.
- [KAFKA-1901] - Move Kafka version to be generated in code by build (instead of in manifest)
- [KAFKA-1913] - App hungs when calls producer.send to wrong IP of Kafka broker
- [KAFKA-1940] - Initial checkout and build failing
- [KAFKA-1947] - can't explicitly set replica-assignment when add partitions
- [KAFKA-1948] - kafka.api.consumerTests are hanging
- [KAFKA-1959] - Class CommitThread overwrite group of Thread class causing compile errors
- [KAFKA-1960] - .gitignore does not exclude test generated files and folders.
- [KAFKA-1964] - testPartitionReassignmentCallback hangs occasionally
- [KAFKA-1969] - NPE in unit test for new consumer
- [KAFKA-1973] - Remove the accidentally created LogCleanerManager.scala.orig
- [KAFKA-1975] - testGroupConsumption occasionally hang
- [KAFKA-1988] - org.apache.kafka.common.utils.Utils.abs method returns wrong value for negative numbers.
- [KAFKA-1992] - Following KAFKA-1697, checkEnoughReplicasReachOffset doesn't need to get requiredAcks
- [KAFKA-1999] - Fix failing unit-test: kafka.api.ProducerFailureHandlingTest > testNotEnoughReplicasAfterBrokerShutdown
- [KAFKA-2001] - OffsetCommitTest hang during setup
- [KAFKA-2006] - switch the broker server over to the new java protocol definitions
- [KAFKA-2009] - Fix UncheckedOffset.removeOffset synchronization and trace logging issue in mirror maker
- [KAFKA-2010] - unit tests sometimes are slow during shutdown
- [KAFKA-2012] - Broker should automatically handle corrupt index files
- [KAFKA-2026] - Logging of unused options always shows null for the value and is misleading if the option is used by serializers
- [KAFKA-2032] - ConsumerConfig doesn't validate partition.assignment.strategy values
- [KAFKA-2033] - Small typo in documentation
- [KAFKA-2042] - New producer metadata update always get all topics.
- [KAFKA-2043] - CompressionType is passed in each RecordAccumulator append
- [KAFKA-2088] - kafka-console-consumer.sh should not create zookeeper path when no brokers found and chroot was set in zookeeper.connect
- [KAFKA-2099] - BrokerEndPoint file, methods and object names should match
- [KAFKA-2101] - Metric metadata-age is reset on a failed update
- [KAFKA-2104] - testDuplicateListeners() has a typo
- [KAFKA-2112] - make overflowWheel volatile
- [KAFKA-2113] - TestPurgatoryPerformance does not compile using IBM JDK
- [KAFKA-2114] - Unable to change min.insync.replicas default
- [KAFKA-2115] - Error updating metrics in RequestChannel
- [KAFKA-2117] - OffsetManager uses incorrect field for metadata
- [KAFKA-2118] - Cleaner cannot clean after shutdown during replaceSegments
- [KAFKA-2121] - prevent potential resource leak in KafkaProducer and KafkaConsumer
- [KAFKA-2122] - Remove controller.message.queue.size Config
- [KAFKA-2126] - New consumer does not correctly configure deserializers
- [KAFKA-2130] - Resource leakage in AppInfo.scala during initialization
- [KAFKA-2147] - Unbalanced replication can cause extreme purgatory growth
- [KAFKA-2154] - MetadataResponse is Empty on a Fresh Cluster
- [KAFKA-2161] - Fix a few copyrights
- [KAFKA-2163] - Offsets manager cache should prevent stale-offset-cleanup while an offset load is in progress; otherwise we can lose consumer offsets
- [KAFKA-2164] - ReplicaFetcherThread: suspicious log message on reset offset
- [KAFKA-2169] - Upgrade to zkclient-0.5
- [KAFKA-2182] - zkClient dies if there is any exception while reconnecting
- [KAFKA-2189] - Snappy compression of message batches less efficient in 0.8.2.1
- [KAFKA-2198] - kafka-topics.sh exits with 0 status on failures
- [KAFKA-2202] - ConsumerPerformance reports a throughput much higher than the actual one
- [KAFKA-2203] - Get gradle build to work with Java 8
- [KAFKA-2207] - The testCannotSendToInternalTopic test method in ProducerFailureHandlingTest fails consistently with the following exception:
- [KAFKA-2226] - NullPointerException in TestPurgatoryPerformance
- [KAFKA-2232] - make MockProducer generic
- [KAFKA-2233] - Log deletion is not removing log metrics
- [KAFKA-2234] - Partition reassignment of a nonexistent topic prevents future reassignments
- [KAFKA-2235] - LogCleaner offset map overflow
- [KAFKA-2241] - AbstractFetcherThread.shutdown() should not block on ReadableByteChannel.read(buffer)
- [KAFKA-2248] - Use Apache Rat to enforce copyright headers
- [KAFKA-2249] - KafkaConfig does not preserve original Properties
- [KAFKA-2250] - ConcurrentModificationException in metrics reporting
- [KAFKA-2251] - "Connection reset by peer" IOExceptions should not be logged as ERROR
- [KAFKA-2252] - Socket connection closing is logged, but not corresponding opening of socket
- [KAFKA-2253] - Deadlock in delayed operation purgatory
- [KAFKA-2255] - Missing documentation for max.in.flight.requests.per.connection
- [KAFKA-2262] - LogSegmentSize validation should be consistent
- [KAFKA-2264] - SESSION_TIMEOUT_MS_CONFIG in ConsumerConfig should be int
- [KAFKA-2266] - Client Selector can drop idle connections without notifying NetworkClient
- [KAFKA-2270] - incorrect package name in unit tests
- [KAFKA-2272] - listeners endpoint parsing fails if the hostname has capital letter
- [KAFKA-2276] - Initial patch for KIP-25
- [KAFKA-2281] - org.apache.kafka.clients.producer.internals.ErrorLoggingCallback holds unnecessary byte[] value
- [KAFKA-2288] - Follow-up to KAFKA-2249 - reduce logging and testing
- [KAFKA-2290] - OffsetIndex should open RandomAccessFile consistently
- [KAFKA-2291] - Documentation Error
- [KAFKA-2295] - Dynamically loaded classes (encoders, etc.) may not be found by Kafka Producer
- [KAFKA-2300] - Error in controller log when broker tries to rejoin cluster
- [KAFKA-2304] - Support enabling JMX in Kafka Vagrantfile
- [KAFKA-2306] - New producer should emit metrics for buffer exhaustion
- [KAFKA-2308] - New producer + Snappy face un-compression errors after broker restart
- [KAFKA-2313] - javadoc fix for KafkaConsumer deserialization
- [KAFKA-2316] - Drop java 1.6 support
- [KAFKA-2317] - De-register isrChangeNotificationListener on controller resignation
- [KAFKA-2327] - broker doesn't start if config defines advertised.host but not advertised.port
- [KAFKA-2330] - Vagrantfile sets global configs instead of per-provider override configs
- [KAFKA-2335] - Javadoc for Consumer says that it's thread-safe
- [KAFKA-2336] - Changing offsets.topic.num.partitions after the offset topic is created breaks consumer group partition assignment
- [KAFKA-2337] - Verify that metric names will not collide when creating new topics
- [KAFKA-2338] - Warn users if they change max.message.bytes that they also need to update broker and consumer settings
- [KAFKA-2345] - Attempt to delete a topic already marked for deletion throws ZkNodeExistsException
- [KAFKA-2347] - Add setConsumerRebalanceListener method to ZookeeperConsuemrConnector java api.
- [KAFKA-2353] - SocketServer.Processor should catch exception and close the socket properly in configureNewConnections.
- [KAFKA-2357] - Update zookeeper.connect description in Kafka documentation
- [KAFKA-2362] - kafka-consumer-perf-test.sh got a negative result
- [KAFKA-2381] - Possible ConcurrentModificationException while unsubscribing from a topic in new consumer
- [KAFKA-2382] - KafkaConsumer seek methods should throw an exception when called for partitions not assigned to this consumer instance
- [KAFKA-2393] - Correctly Handle InvalidTopicException in KafkaApis.getTopicMetadata()
- [KAFKA-2405] - KafkaHealthCheck kills the JVM in handleSessionEstablishmentError
- [KAFKA-2406] - ISR propagation should be throttled to avoid overwhelming controller.
- [KAFKA-2407] - Only create a log directory when it will be used
- [KAFKA-2408] - (new) system tests: ConsoleConsumerService occasionally fails to register consumed message
- [KAFKA-2412] - Documentation bug: Add information for key.serializer and value.serializer to New Producer Config sections
- [KAFKA-2413] - New consumer's subscribe(Topic...) api fails if called more than once
- [KAFKA-2428] - Add sanity test in kafkaConsumer for the timeouts. This is a followup ticket for Kafka-2120
- [KAFKA-2436] - log.retention.hours should be honored by LogManager
- [KAFKA-2437] - Controller does not handle zk node deletion correctly.
- [KAFKA-2438] - add maxParallelForks to build.gradle to speedup tests
- [KAFKA-2449] - Update mirror maker (MirrorMaker) docs
- [KAFKA-2450] - Kafka 0.8.2.1 kafka-console-consumer.sh broken
- [KAFKA-2454] - Dead lock between delete log segment and shutting down.
- [KAFKA-2457] - StackOverflowError during builds
- [KAFKA-2459] - Connection backoff/blackout period should start when a connection is disconnected, not when the connection attempt was initiated
- [KAFKA-2461] - request logger no longer logs extra information in debug mode
- [KAFKA-2466] - ConsoleConsumer throws ConcurrentModificationException on termination
- [KAFKA-2467] - ConsoleConsumer regressions
- [KAFKA-2468] - SIGINT during Kafka server startup can leave server deadlocked
- [KAFKA-2469] - System test console consumer logs should write all messages to debug logger
- [KAFKA-2470] - kafka-producer-perf-test.sh can't configure all to request-num-acks
- [KAFKA-2472] - Fix kafka ssl configs to not throw warnings
- [KAFKA-2477] - Replicas spuriously deleting all segments in partition
- [KAFKA-2489] - System tests: update benchmark tests to run with new and old consumer
- [KAFKA-2504] - Stop logging WARN when client disconnects
- [KAFKA-2516] - Rename o.a.k.client.tools to o.a.k.tools
- [KAFKA-2517] - Performance Regression post SSL implementation
- [KAFKA-2518] - Update NOTICE file
- [KAFKA-2519] - NetworkClient.close should remove node from inFlightRequests
- [KAFKA-2533] - Create a member Metadata.Listener inside KafkaConsumer
- [KAFKA-2534] - SSLTransportLayer does not handle buffer overflow correctly
- [KAFKA-2536] - topics tool should allow users to alter topic configuration
- [KAFKA-2538] - Compilation in trunk is failing due to https://github.com/apache/kafka/commit/845514d62329be8382e6d02b8041fc858718d534
- [KAFKA-2548] - kafka-merge-pr tool fails to update JIRA with fix version 0.9.0.0
- [KAFKA-2554] - change 0.8.3 to 0.9.0 in ApiVersion
- [KAFKA-2555] - Infinite recursive function call occurs when ConsumerRebalanceCallback.onPartitionRevoked() calls commitSync()
- [KAFKA-2557] - Separate RebalanceInProgress from IllegalGeneration Error Code
- [KAFKA-2563] - Version number major.major.minor.fix in kafka-merge-pr.py
- [KAFKA-2564] - SSL: Received fatal alert: handshake_failure occurs sporadically
- [KAFKA-2567] - throttle-time shouldn't be NaN
- [KAFKA-2571] - KafkaLog4jAppender dies while specifying "acks" config
- [KAFKA-2573] - Mirror maker system test hangs and eventually fails
- [KAFKA-2576] - ConsumerPerformance hangs when SSL enabled for Multi-Partition Topic
- [KAFKA-2585] - ConsoleConsumer should not hang infinitely upon exception
- [KAFKA-2596] - Coordinator should return illegal generation for commits from unknown groups with non-negative generation
- [KAFKA-2599] - Metadata#getClusterForCurrentTopics can throw NPE even with null checking
- [KAFKA-2614] - No more clients can connect after `TooManyConnectionsException` threshold (max.connections.per.ip) is reached
- [KAFKA-2618] - Disable SSL renegotiation for 0.9.0.0
- [KAFKA-2621] - nextOffsetMetadata should be changed after rolling a new log segment
- [KAFKA-2624] - Truncate warn message logged after truncating partitions
- [KAFKA-2628] - KafkaOffsetBackingStoreTest.testGetSet transient test failure
- [KAFKA-2633] - Default logging from tools to Stderr
- [KAFKA-2646] - Re-enable altering topic config in the topics command to maintain backwards compatibility
- [KAFKA-2648] - Coordinator should not allow empty groupIds
- [KAFKA-2650] - Change ConfigCommand --deleted-config option to align with TopicCommand
- [KAFKA-2656] - Default SSL keystore and truststore config are unusable
- [KAFKA-2660] - Correct cleanableRatio calculation
- [KAFKA-2663] - Add quota-delay time to request processing time break-up
- [KAFKA-2664] - Adding a new metric with several pre-existing metrics is very expensive
- [KAFKA-2665] - Docs: Images that are part of the documentation are not part of the code github
- [KAFKA-2666] - Docs: Automatically generate documentation from config classes
- [KAFKA-2669] - Fix LogCleanerIntegrationTest
- [KAFKA-2674] - ConsumerRebalanceListener.onPartitionsRevoked() is not called on consumer close
- [KAFKA-2677] - Coordinator disconnects not propagated to new consumer
- [KAFKA-2678] - partition level lag metrics can be negative
- [KAFKA-2680] - Zookeeper SASL check prevents any SASL code being run with IBM JDK
- [KAFKA-2683] - Ensure wakeup exceptions are propagated to user in new consumer
- [KAFKA-2686] - unsubscribe() call leaves KafkaConsumer in invalid state for manual topic-partition assignment
- [KAFKA-2688] - Avoid forcing reload of `Configuration`
- [KAFKA-2702] - ConfigDef toHtmlTable() sorts in a way that is a bit confusing
- [KAFKA-2716] - Make Kafka core not depend on log4j-appender
- [KAFKA-2719] - Kafka classpath has grown too large and breaks some system tests
- [KAFKA-2721] - Avoid handling duplicate LeaderAndISR requests
- [KAFKA-2722] - Improve ISR change propagation
- [KAFKA-2726] - ntpdate causes vagrant provision to fail if ntp running
- [KAFKA-2730] - partition-reassignment tool stops working due to error in registerMetric
- [KAFKA-2734] - kafka-console-consumer throws NoSuchElementException on not specifying topic
- [KAFKA-2735] - BrokerEndPoint should support uppercase hostnames
- [KAFKA-2736] - ZkClient doesn't handle SaslAuthenticated
- [KAFKA-2738] - Can't set SSL as inter-broker-protocol by rolling restart of brokers
- [KAFKA-2741] - Source/SinkTaskContext should be interfaces and implementations kept in runtime jar
- [KAFKA-2742] - SourceTaskOffsetCommitter does not properly remove commit tasks when they are already in progress
- [KAFKA-2743] - Forwarding task reconfigurations in Copycat can deadlock with rebalances and has no backoff
- [KAFKA-2744] - WorkerSourceTask commits offsets too early when stopping
- [KAFKA-2747] - Message loss if mirror maker is killed with hard kill and then restarted
- [KAFKA-2748] - SinkTasks do not handle rebalances and offset commit properly
- [KAFKA-2755] - NPE thrown while handling DescribeGroup request for non-existent consumer group
- [KAFKA-2756] - Replication Broken between Kafka 0.8.2.1 and 0.9 - NetworkClient.java uses wrong protocol version
- [KAFKA-2760] - Clean up interface of AdminClient.describeConsumerGroup(groupId
- [KAFKA-2764] - Copycat APIs should use Map<String, String> instead of Properties
- [KAFKA-2766] - Make the Java Producer as the default producer in tooling.
- [KAFKA-2768] - New-consumer sends invalid describeGroupResponse while restabilizing
- [KAFKA-2770] - Race condition causes Mirror Maker to hang during shutdown (new consumer)
- [KAFKA-2773] - Vagrant provision fails if num_brokers or num_zookeepers is nonzero
- [KAFKA-2775] - Copycat exceptions should be in api package so they can be caught by user code without any dependencies other than api
- [KAFKA-2776] - JsonConverter uses wrong key to look up schema conversion cache size configuration
- [KAFKA-2778] - Use zero loss settings for producer in Kafka Connect
- [KAFKA-2779] - Kafka SSL transport layer leaks file descriptors
- [KAFKA-2781] - Signing jars shouldn't be required for install task
- [KAFKA-2782] - Incorrect assertion in KafkaBasedLogTest.testSendAndReadToEnd
- [KAFKA-2785] - Copycat jars not included in release tar gz
- [KAFKA-2786] - Only invoke SinkTask onPartitionsRevoked and commitOffsets after task has fully started
- [KAFKA-2788] - allow comma when specifying principals in AclCommand
- [KAFKA-2792] - KafkaConsumer.close() can block unnecessarily due to leave group waiting for a reply
- [KAFKA-2793] - ConsoleConsumer crashes with new consumer when using keys because of incorrect deserializer
- [KAFKA-2795] - potential NPE in GroupMetadataManager
- [KAFKA-2797] - Release artifact expects a git repository for the release audit tool (RAT)
- [KAFKA-2798] - Kafka Connect distributed configs can conflict with producer/consumer configs, making it impossible to control them independently
- [KAFKA-2799] - WakupException thrown in the followup poll() could lead to data loss
- [KAFKA-2801] - Data read from network not processed by SSL transport layer
- [KAFKA-2805] - RecordAccumulator request timeout not enforced when all brokers are gone
- [KAFKA-2807] - Movement of throughput throttler to common broke upgrade tests
- [KAFKA-2813] - selector doesn't close socket connection on non-IOExceptions
- [KAFKA-2814] - Kafka Connect system tests using REST interface fail on AWS
- [KAFKA-2817] - Closing an unconnected SslTransport get an invalid close state exception
- [KAFKA-2819] - ConsumerConsume throw unexpected exception during clean shutdown
- [KAFKA-2821] - Deadlock in group metadata persistence callback
- [KAFKA-2822] - DescribeConsumerGroup now returns empty list for non-existent group, it used to throw IllegalArgumentException
- [KAFKA-2833] - OffsetsMessageFormatter hits unexpected exception
- [KAFKA-2838] - allow comma when specifying superusers
- [KAFKA-2841] - Group metadata cache loading is not safe when reloading a partition
- [KAFKA-2847] - remove principal.builder.class from client configs
- [KAFKA-2848] - Use withClientSslSupport/withClientSaslSupport in Kafka Connect
- [KAFKA-2852] - Kafka Authroizer CLI should use consistent way to specify multiple values for all config options.
- [KAFKA-2859] - Deadlock in WorkerSourceTask
- [KAFKA-2867] - Missing synchronization and improperly handled InterruptException in WorkerSourceTask
- [KAFKA-2869] - host used by Authorizer should be IP address not hostname/IP
Improvement
- [KAFKA-1173] - Using Vagrant to get up and running with Apache Kafka
- [KAFKA-1461] - Replica fetcher thread does not implement any back-off behavior
- [KAFKA-1546] - Automate replica lag tuning
- [KAFKA-1644] - Inherit FetchResponse from RequestOrResponse
- [KAFKA-1646] - Improve consumer read performance for Windows
- [KAFKA-1650] - Mirror Maker could lose data on unclean shutdown.
- [KAFKA-1651] - Removed some extra whitespace in KafkaServer.scala
- [KAFKA-1654] - Provide a way to override server configuration from command line
- [KAFKA-1660] - Ability to call close() with a timeout on the Java Kafka Producer.
- [KAFKA-1801] - Remove non-functional variable definition in log4j.properties
- [KAFKA-1807] - Improve accuracy of ProducerPerformance target throughput
- [KAFKA-1818] - Code cleanup in ReplicationUtils including unit test
- [KAFKA-1822] - Add "echo" request
- [KAFKA-1835] - Kafka new producer needs options to make blocking behavior explicit
- [KAFKA-1854] - Allow the JIRA username and password to be prompted during patch submission
- [KAFKA-1885] - Allow test methods in "core" to be individually run from outside of the IDE
- [KAFKA-1888] - Add a "rolling upgrade" system test
- [KAFKA-1915] - Integrate checkstyle for java code
- [KAFKA-1926] - Replace kafka.utils.Utils with o.a.k.common.utils.Utils
- [KAFKA-1938] - [doc] Quick start example should reference appropriate Kafka version
- [KAFKA-1957] - code doc typo
- [KAFKA-1965] - Leaner DelayedItem
- [KAFKA-1982] - change kafka.examples.Producer to use the new java producer
- [KAFKA-1989] - New purgatory design
- [KAFKA-1990] - Add unlimited time-based log retention
- [KAFKA-1994] - Evaluate performance effect of chroot check on Topic creation
- [KAFKA-1997] - Refactor Mirror Maker
- [KAFKA-2005] - Generate html report for system tests
- [KAFKA-2016] - RollingBounceTest takes long
- [KAFKA-2039] - Update Scala to 2.10.5 and 2.11.6
- [KAFKA-2044] - Support requests and responses from o.a.k.common in KafkaApis
- [KAFKA-2090] - Remove duplicate check to metadataFetchInProgress
- [KAFKA-2096] - Enable keepalive socket option for broker to prevent socket leak
- [KAFKA-2109] - Support retries in KafkaLog4jAppender
- [KAFKA-2110] - Doc: Clarify that max number of consumer instances is per consumer group
- [KAFKA-2119] - ConsumerRecord key() and value() methods should not have throws Exception
- [KAFKA-2128] - kafka.Kafka should return non-zero exit code when caught exception.
- [KAFKA-2131] - Update new producer javadocs with correct documentation links
- [KAFKA-2132] - Move Log4J appender to a separate module
- [KAFKA-2185] - Update to Gradle 2.4
- [KAFKA-2265] - creating a topic with large number of partitions takes a long time
- [KAFKA-2278] - JmxTool should support querying all objects when object-name is omitted
- [KAFKA-2279] - add a main method in LogConfig to generate topic level configs in html
- [KAFKA-2312] - Use AtomicLong opposed to AtomicReference to store currentThread in consumer
- [KAFKA-2314] - Proper help message for MirrorMaker's `message.handler` property
- [KAFKA-2321] - Introduce CONTRIBUTING.md
- [KAFKA-2323] - Simplify ScalaTest dependency versions
- [KAFKA-2324] - Update to Scala 2.11.7
- [KAFKA-2328] - merge-kafka-pr.py script should not leave user in a detached branch
- [KAFKA-2344] - kafka-merge-pr improvements
- [KAFKA-2350] - Add KafkaConsumer pause capability
- [KAFKA-2384] - Override commit message title in kafka-merge-pr.py
- [KAFKA-2419] - Allow certain Sensors to be garbage collected after inactivity
- [KAFKA-2425] - Migrate website from SVN to Git
- [KAFKA-2429] - Add annotations to mark classes as stable/unstable
- [KAFKA-2430] - Listing of PR commits in commit message should be optional
- [KAFKA-2433] - Remove documentation on dead configuration item: replica.lag.max.messages
- [KAFKA-2447] - Add capability to KafkaLog4jAppender to be able to use SSL
- [KAFKA-2460] - Fix capitalization in SSL classes
- [KAFKA-2485] - Allow producer performance to take properties from a file via --producer.config command line parameter
- [KAFKA-2514] - change default JVM options in kafka-run-class.sh
- [KAFKA-2562] - check Kafka scripts for 0.9.0.0
- [KAFKA-2570] - New consumer should commit before every rebalance when auto-commit is enabled
- [KAFKA-2586] - Enable SSL for inter-broker communication in SSL tests
- [KAFKA-2597] - Add Eclipse directories to .gitignore
- [KAFKA-2645] - Document potentially breaking changes in the release notes for 0.9.0
- [KAFKA-2671] - Enable starting Kafka server with a Properties object
- [KAFKA-2687] - Add support for ListGroups and DescribeGroup APIs
- [KAFKA-2689] - Expose select gauges and metrics programmatically (not just through JMX)
- [KAFKA-2715] - Remove the previous system test folder
- [KAFKA-2723] - Standardize new consumer exceptions
- [KAFKA-2740] - Convert Windows bin scripts from CRLF to LF line encodings
- [KAFKA-2753] - Improve SyncGroup error handling in AbstractCoordinator
- [KAFKA-2767] - Upgrade ZkClient version to 0.7
- [KAFKA-2790] - Kafka 0.9.0 doc improvement
- [KAFKA-2791] - remove deprecated producer properties from console-producer
- [KAFKA-2809] - Improve documentation linking
- [KAFKA-2831] - kafka-consumer-groups requires zookeeper url when using the new-consumer option
- [KAFKA-2863] - Authorizer should provide lifecycle (shutdown) methods
New Feature
- [KAFKA-1499] - Broker-side compression configuration
- [KAFKA-1785] - Consumer offset checker should show the offset manager and offsets partition
- [KAFKA-2120] - Add a request timeout to NetworkClient
- [KAFKA-2187] - Introduce merge-kafka-pr.py script
Task
- [KAFKA-863] - System Test - update 0.7 version of kafka-run-class.sh for Migration Tool test cases
- [KAFKA-1661] - Move MockConsumer and MockProducer from src/main to src/test
- [KAFKA-1856] - Add PreCommit Patch Testing
- [KAFKA-1949] - Update 0.8.3.0 docs with upgrade process
- [KAFKA-2348] - Drop support for Scala 2.9
- [KAFKA-2349] - `contributing` website page should link to "Contributing Code Changes" wiki page
- [KAFKA-2364] - Improve documentation for contributing to docs
- [KAFKA-2443] - Expose windowSize on Rate
- [KAFKA-2492] - Upgrade zkclient dependency to 0.6
- [KAFKA-2502] - Quotas documentation for 0.8.3
- [KAFKA-2746] - Add support for using ConsumerGroupCommand on secure install
- [KAFKA-2783] - Drop outdated hadoop contrib modules
Test
- [KAFKA-2013] - benchmark test for the purgatory
- [KAFKA-2340] - Add additional unit tests for new consumer Fetcher
- [KAFKA-2355] - Add an unit test to validate the deletion of a partition marked as deleted
- [KAFKA-2531] - Add Ducktape based tests for KafkaLog4jAppender
- [KAFKA-2603] - Add timeout to ConsoleConsumer running with new consumer
- [KAFKA-2705] - Remove static JAAS config file for ZK auth tests
- [KAFKA-2714] - Add integration tests for exceptional cases in Fetching for new consumer
- [KAFKA-2737] - Integration tests for round-robin assignment
- [KAFKA-2769] - Integration tests for multi-consumer assignment including session timeouts
Wish