Commit Graph

2699 Commits

Author SHA1 Message Date
Jaikiran Pai 6fb25f080f MINOR: Update the README.md to include a note about GRADLE_USER_HOME
Trying to build the source and publish it to internal Maven repo, I ran into an issue that I explain in the mailing list discussion here https://www.mail-archive.com/devkafka.apache.org/msg56359.html.

The commit here updates the README.md to make a note that the GRADLE_USER_HOME environment variable plays a role in deciding which file to add the maven configs to.

Author: Jaikiran Pai <jaikiran.pai@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #1837 from jaikiran/readme-update-grade-user-home
2016-09-17 23:01:32 +01:00
Grant Henke b5bc1a1caa KAFKA-4157; Transient system test failure in replica_verification_test.test_replica_lags
…t.test_replica_lags

Author: Grant Henke <granthenke@gmail.com>

Reviewers: Ashish Singh <asingh@cloudera.com>, Ismael Juma <ismael@juma.me.uk>

Closes #1849 from granthenke/replica-verification-fix
2016-09-17 22:47:56 +01:00
Matthias J. Sax b8683026ea HOTFIX: changed quickstart donwload from 0.10.0.0 to 0.10.0.1
Author: Matthias J. Sax <matthias@confluent.io>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #1869 from mjsax/hotfix-doc
2016-09-17 22:45:29 +01:00
Eno Thereska 932e4d7ba7 HOTFIX: Increase timeout for bounce test
Author: Eno Thereska <eno.thereska@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #1874 from enothereska/hotfix-bounce-test
2016-09-17 22:43:43 +01:00
Rajini Sivaram 69356fbc6e KAFKA-3492; Secure quotas for authenticated users
Implementation and tests for secure quotas at <user> and <user, client-id> levels as described in KIP-55. Also adds dynamic default quotas for <client-id>, <user> and <user-client-id>. For each client connection, the most specific quota matching the connection is used, with user quota taking precedence over client-id quota.

Author: Rajini Sivaram <rajinisivaram@googlemail.com>

Reviewers: Jun Rao <junrao@gmail.com>

Closes #1753 from rajinisivaram/KAFKA-3492
2016-09-17 10:06:05 -07:00
Sumit Arrawatia ecc1fb10fa KAFKA-4093; Cluster Id (KIP-78)
This PR implements  KIP-78:Cluster Identifiers [(link)](https://cwiki.apache.org/confluence/display/KAFKA/KIP-78%3A+Cluster+Id#KIP-78:ClusterId-Overview) and includes the following changes:

1. Changes to broker code
	- generate cluster id and store it in Zookeeper
	- update protocol to add cluster id to metadata request and response
	- add ClusterResourceListener interface, ClusterResource class and ClusterMetadataListeners utility class
	- send ClusterResource events to the metric reporters
2. Changes to client code
	- update Cluster and Metadata code to support cluster id
	- update clients for sending ClusterResource events to interceptors, (de)serializers and metric reporters
3. Integration tests for interceptors, (de)serializers and metric reporters for clients and for protocol changes and metric reporters for broker.
4. System tests for upgrading from previous versions.

Author: Sumit Arrawatia <sumit.arrawatia@gmail.com>
Author: Ismael Juma <ismael@juma.me.uk>

Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>

Closes #1830 from arrawatia/kip-78
2016-09-17 07:53:25 +01:00
Shikhar Bhushan d7bffebca0 KAFKA-4173; SchemaProjector should successfully project missing Struct field when target field is optional
Author: Shikhar Bhushan <shikhar@confluent.io>

Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Jason Gustafson <jason@confluent.io>

Closes #1865 from shikhar/kafka-4173
2016-09-16 15:54:33 -07:00
Randall Hauch 567cc3d787 KAFKA-4183; Corrected Kafka Connect's JSON Converter to properly convert from null to logical values
The `JsonConverter` class has `LogicalTypeConverter` implementations for Date, Time, Timestamp, and Decimal, but these implementations fail when the input literal value (deserialized from the message) is null.

Test cases were added to check for these cases, and these failed before the `LogicalTypeConverter` implementations were fixed to consider whether the schema has a default value or is optional, similarly to how the `JsonToConnectTypeConverter` implementations do this. Once the fixes were made, the new tests pass.

Author: Randall Hauch <rhauch@gmail.com>

Reviewers: Shikhar Bhushan <shikhar@confluent.io>, Jason Gustafson <jason@confluent.io>

Closes #1867 from rhauch/kafka-4183
2016-09-16 14:55:46 -07:00
Damian Guy 86aa0eb0f2 KAFKA-3776: Unify store and downstream caching in streams
This is joint work between dguy and enothereska. The work implements KIP-63. Overview of main changes:

- New byte-based cache that acts as a buffer for any persistent store and for forwarding changes downstream.
- Forwarding record path changes: previously a record in a task completed end-to-end. Now it may be buffered in a processor node while other records complete in the task.
- Cleanup and state stores and decoupling of cache from state store and forwarding.
- More than 80 new unit and integration tests.

Author: Damian Guy <damian.guy@gmail.com>
Author: Eno Thereska <eno.thereska@gmail.com>

Reviewers: Matthias J. Sax, Guozhang Wang

Closes #1752 from enothereska/KAFKA-3776-poc
2016-09-16 09:58:36 -07:00
Ben Stopford 143a33bc50 KAFKA-1464; Add a throttling option to the Kafka replication
This applies to Replication Quotas
based on KIP-73 [(link)](https://cwiki.apache.org/confluence/display/KAFKA/KIP-73+Replication+Quotas) originally motivated by KAFKA-1464.

System Tests Run: https://jenkins.confluent.io/job/system-test-kafka-branch-builder/544/

**This first PR demonstrates the approach**.

**_Overview of Change_**
The guts of this change are relatively small. Throttling occurs on both leader and follower sides. A single class tracks the throttled throughput in and out of each broker (**_ReplicationQuotaManager_**).

On the follower side, the Follower Throttled Rate is calculated as fetch responses arrive. Then, before the next fetch request is sent, we check to see if the quota is violated, removing throttled partitions from the request if it is. This is all encapsulated in a few lines of code in the **_ReplicaFetcherThread_**. There is existing code to handle temporal back off, if the request ends up being empty.

On the leader side it's a little more complex. When a fetch request arrives in the leader, it is built, partition by partition, in **_ReplicaManager.readFromLocalLog_**. As we put each partition into the fetch response, we check if the total size fits in the current quota. If the quota is exceeded, the partition will not be added to the fetch response. Importantly, we don't increase the quota at this point, we just check to see if the bytes will fit.

Now, if there aren't enough bytes to send the response immediately, which is common if we're catching up and throttled, then the request will be put in purgatory. I've added some simple code to **_DelayedFetch_** to handle throttled partitions (throttled partitions are checked against the quota, rather than the messages available in the log).

When the delayed fetch completes, and exits purgatory, _**ReplicaManager.readFromLocalLog**_ will be called again. This is why _**ReplicaManager.readFromLocalLog**_ does not actually increase the quota, it just checks whether enough bytes are available for a partition.

Finally, when there are enough bytes to be sent, or the delayed fetch times out, the response will be sent. Before it is sent the throttled-outbound-rate is increased, based on the size of throttled partitions being sent. This is at the end of _**KafkaApis.handleFetchRequest**_, exactly where client quotas are recorded.

There is an acceptance test which asserts the whole throttling process stabilises on the desired value. This covers a number of use cases including many-to-many replication. See **_ReplicationQuotaTest_**.

Note:
It should be noted that this protocol can over-request. The request is built, based on the quota at time t1 (_ReplicaManager.readFromLocalLog_). The bytes in the response are recorded at time t2 (end of _KafkaApis.handleFetchRequest_), where t2 > t1. For this reason I originally included an OverRequestedRate as a JMX metric, but testing has not seen revealed any obvious issue. Over-requesting is quickly compensated by subsequent requests, stabilising close to the quota value.

_**Main stuff left to do:**_
- The fetch size is currently unbounded. This will be addressed in KIP-74, but we need to ensure this ensures requests don’t go beyond the throttle window.
- There are two failures showing up in the system tests on this branch:  StreamsSmokeTest.test_streams (which looks like it fails regularly) and OffsetValidationTest.test_broker_rolling_bounce (which I need to look into)

_**Stuff left to do that could be deferred:**_
- Add the extra metrics specified in the KIP.
- There are no system tests.
- There is no validation for the cluster size / throttle combination that could lead to ISR dropouts

Author: Ben Stopford <benstopford@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>, Apurva Mehta <apurva@confluent.io>, Jun Rao <junrao@gmail.com>

Closes #1776 from benstopford/rep-quotas-v2
2016-09-15 22:25:56 -07:00
Bill Bejeck d436661024 KAFKA-4131; Multiple Regex KStream-Consumers cause Null pointer exception
Fix for bug outlined in KAFKA-4131

Author: bbejeck <bbejeck@gmail.com>

Reviewers: Damian Guy, Guozhang Wang

Closes #1843 from bbejeck/KAFKA-4131_mulitple_regex_consumers_cause_npe
2016-09-15 17:08:00 -07:00
Damian Guy f3bb10413d HOTFIX: fix KafkaStreams SmokeTest
Set the NUM_STREAM_THREADS_CONFIG = 1 in SmokeTestClient as we get locking issues when we have NUM_STREAM_THREADS_CONFIG > 1 and we have Standby Tasks, i.e., replicas. This is because the Standby Tasks can be assigned to the same KafkaStreams instance as the active task, hence the directory is locked

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Eno Thereska, Guozhang Wang

Closes #1861 from dguy/fix-smoketest
2016-09-15 08:57:48 -07:00
Jason Gustafson f197ad4997 KAFKA-4160: Ensure rebalance listener not called with coordinator lock
Author: Jason Gustafson <jason@confluent.io>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #1855 from hachikuji/KAFKA-4160
2016-09-14 22:31:52 -07:00
Jason Gustafson 084a19e9ac KAFKA-4172; Ensure fetch responses contain the requested partitions
Author: Jason Gustafson <jason@confluent.io>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #1857 from hachikuji/KAFKA-4172
2016-09-14 18:04:58 -07:00
David Chen c1bce2d757 KAFKA-4162: Fixed typo "rebalance"
Author: David Chen <mvjome@gmail.com>

Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>

Closes #1853 from mvj3/KAFKA-4162
2016-09-14 10:38:40 -07:00
Ismael Juma f2cd170102 MINOR: Give a name to the coordinator heartbeat thread
Followed the same naming pattern as the producer sender thread.

Author: Ismael Juma <ismael@juma.me.uk>

Reviewers: Jason Gustafson

Closes #1854 from ijuma/heartbeat-thread-name
2016-09-14 09:16:29 -07:00
Dong Lin 41e676d295 KAFKA-4158; Reset quota to default value if quota override is deleted
Author: Dong Lin <lindong28@gmail.com>

Reviewers: Joel Koshy <jjkoshy.w@gmail.com>, Jiangjie Qin <becket.qin@gmail.com>

Closes #1851 from lindong28/KAFKA-4158
2016-09-13 17:33:54 -07:00
Tim Brooks 2a660f13d2 KAFKA-2311; Make KafkaConsumer's ensureNotClosed method thread-safe
Here is the patch on github ijuma.

Acquiring the consumer lock (the single thread access controls) requires that the consumer be open. I changed the closed variable to be volatile so that another thread's writes will visible to the reading thread.

Additionally, there was an additional check if the consumer was closed after the lock was acquired. This check is no longer necessary.

This is my original work and I license it to the project under the project's open source license.

Author: Tim Brooks <tim@uncontended.net>

Reviewers: Jason Gustafson <jason@confluent.io>

Closes #1637 from tbrooks8/KAFKA-2311
2016-09-12 20:28:01 -07:00
Damian Guy a7ab9cb83d MINOR: catch InvalidStateStoreException in QueryableStateIntegrationTest
A couple of the tests may transiently fail in QueryableStateIntegrationTest as they are not catching InvalidStateStoreException. This exception is expected during rebalance.

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Eno Thereska, Guozhang Wang

Closes #1840 from dguy/minor-fix
2016-09-11 21:00:32 -07:00
Eric Wasserman 5f040cd77f KAFKA-1981; Make log compaction point configurable
Now uses LogSegment.largestTimestamp to determine age of segment's messages.

Author: Eric Wasserman <eric.wasserman@gmail.com>

Reviewers: Jun Rao <junrao@gmail.com>

Closes #1794 from ewasserman/feat-1981
2016-09-11 20:45:05 -05:00
Jason Gustafson 1933f12a53 KAFKA-4147; Fix transient failure in ConsumerCoordinatorTest.testAutoCommitDynamicAssignment
Author: Jason Gustafson <jason@confluent.io>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #1841 from hachikuji/KAFKA-4147
2016-09-11 08:46:20 +01:00
Vahid Hashemian ac99a3c86d KAFKA-4145; Avoid redundant integration testing in ProducerSendTests
Author: Vahid Hashemian <vahidhashemian@us.ibm.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #1842 from vahidhashemian/KAFKA-4145
2016-09-10 08:16:23 +01:00
Jason Gustafson 4af50bb860 KAFKA-3807; Fix transient test failure caused by race on future completion
Author: Jason Gustafson <jason@confluent.io>

Reviewers: Dan Norwood <norwood@confluent.io>, Ismael Juma <ismael@juma.me.uk>

Closes #1821 from hachikuji/KAFKA-3807
2016-09-09 21:47:03 +01:00
Eno Thereska 3759d7f764 HOTFIX: Temporarily ignoring this test until fixed
Author: Eno Thereska <eno.thereska@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #1838 from enothereska/hotfix-ignore-smoke-test
2016-09-09 15:22:29 +01:00
Vahid Hashemian 54767bbba5 KAFKA-4033; Revise partition assignment semantics on consumer subscription changes (KIP-70)
This PR changes topic subscription semantics so a change in subscription does not immediately cause a rebalance.
Instead, the next poll or the next scheduled metadata refresh will update the assigned partitions.

Author: Vahid Hashemian <vahidhashemian@us.ibm.com>

Reviewers: Jason Gustafson

Closes #1726 from vahidhashemian/KAFKA-4033
2016-09-08 19:56:53 -07:00
Damian Guy eb59c8124e MINOR: fix transient QueryableStateIntegration test failure
The verification in verifyGreaterOrEqual was incorrect. It was failing when a new key was found.
Set the TimeWindow to a large value so all windowed results fall in a single window

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #1833 from dguy/minor-test-fix
2016-09-08 10:53:29 -07:00
Rajini Sivaram 6cf2cb6f29 KAFKA-4139; Reset findCoordinatorFuture when brokers are unavailable
Author: Rajini Sivaram <rajinisivaram@googlemail.com>

Reviewers: Ismael Juma, Jason Gustafson

Closes #1831 from rajinisivaram/KAFKA-4139
2016-09-08 09:42:22 -07:00
Damian Guy 69ebf6f7be KAFKA-3595: window stores use compact,delete config for changelogs
changelogs of window stores now configure cleanup.policy=compact,delete with retention.ms set to window maintainMs + StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG
StoreChangeLogger produces messages with context.timestamp().

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Eno Thereska, Guozhang Wang

Closes #1792 from dguy/kafka-3595
2016-09-07 18:02:24 -07:00
Damian Guy eba0ede878 KAFKA-4123: Queryable State returning null for key before all stores in instance have been initialized
Mark the store as open after the DB has been restored from the changelog.
Only add the store to the map in ProcessorStateManager post restore.
Make RocksDBWindowStore.Segment override openDB(..) as it needs to mark the Segment as open.
Throw InvalidStateStoreException if any stores in a KafkaStreams instance are not available.

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Eno Thereska, Guozhang Wang

Closes #1824 from dguy/kafka-4123
2016-09-07 17:42:35 -07:00
Mathieu Fenniak ca3f3a492c MINOR: Document that Connect topics should use compaction
Update documentation for Kafka Connect distributed’s config.storage.topic, offset.storage.topic, and status.storage.topic configuration values to indicate that all three should refer to compacted topics.

Author: Mathieu Fenniak <mathieu.fenniak@replicon.com>

Reviewers: Jason Gustafson

Closes #1832 from mfenniak/kafka-connect-topic-docs
2016-09-07 17:34:24 -07:00
Michael G. Noll 3a244b7837 MINOR: Fixes javadoc of Windows, fixes typo in parameter name of KGroupedTable
Author: Michael G. Noll <michael@confluent.io>

Reviewers: Damian Guy, Ismael Juma, Guozhang Wang

Closes #1823 from miguno/trunk-windowed-javadocs
2016-09-07 15:27:58 -07:00
Dustin Cote 85c441023b KAFKA-3129; Console producer issue when request-required-acks=0
change console producer default acks to 1, update acks docs.  Also added the -1 config to the acks docs since that question comes up often.  ijuma and vahidhashemian, does this look reasonable to you?

Author: Dustin Cote <dustin@confluent.io>

Reviewers: Vahid Hashemian <vahidhashemian@us.ibm.com>, Ismael Juma <ismael@juma.me.uk>

Closes #1795 from cotedm/KAFKA-3129
2016-09-07 16:07:52 +01:00
Matthias J. Sax 2586226a9a KAFKA-4058: Failure in org.apache.kafka.streams.integration.ResetIntegrationTest.testReprocessingFromScratchAfterReset
- use AdminTool to check for active consumer group

Author: Matthias J. Sax <matthias@confluent.io>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #1767 from mjsax/kafka-4058-trunk
2016-09-06 23:02:41 -07:00
Jaikiran Pai 16469d7f94 MINOR: Reduce the log level when the peer isn't authenticated but is using SSL
The commit here changes the log level of a log message from WARN to DEBUG.
As noted in the mail discussion here https://www.mail-archive.com/devkafka.apache.org/msg56035.html,
in a pretty straightforward/typical and valid setup, the broker logs get
flooded with the following message:

[2016-09-02 08:07:13,773] WARN SSL peer is not authenticated, returning ANONYMOUS instead (org.apache.kafka.common.network.SslTransportLayer)

Author: Jaikiran Pai <jaikiran.pai@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #1825 from jaikiran/ssl-log-level
2016-09-07 03:42:57 +01:00
Ismael Juma d2acd676c3 MINOR: More graceful handling of buffers that are too small in Record's `isValid` and `ensureValid`
Also add tests and make `Crc32.update` perform the same argument checks as
`java.util.zip.CRC32`.

Author: Ismael Juma <ismael@juma.me.uk>

Reviewers: Gwen Shapira

Closes #1672 from ijuma/record-is-valid-should-be-more-robust
2016-09-06 17:34:46 -07:00
Tao Xiao 286411cbb1 KAFKA-4129; Processor throw exception when getting channel remote address after closing the channel
Get channel remote address before calling ```channel.close```

Author: Tao Xiao <xiaotao183@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #1826 from xiaotao183/KAFKA-4129
2016-09-07 01:17:29 +01:00
Dana Powers 8248caf1ca MINOR: Include TopicPartition in warning when log cleaner resets dirty offset
Typically this error condition is caused by topic-level configuration issues, so it is useful to include which topic partition was reset for operator use when debugging the root cause.

Author: Dana Powers <dana.powers@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #1801 from dpkp/log_topic_partition_reset_dirty_offset
2016-09-07 01:15:55 +01:00
Matthias J. Sax de1b853c3e MINOR: changes embedded broker time to MockTime
Author: Matthias J. Sax <matthias@confluent.io>

Reviewers: Damian Guy, Ismael Juma, Guozhang Wang

Closes #1808 from mjsax/mockTime
2016-09-06 15:35:12 -07:00
Bill Bejeck ed639e8263 KAFKA-4023: Add thread id and task id for logging prefix in Streams
Author: bbejeck <bbejeck@gmail.com>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #1803 from bbejeck/KAFKA-4023_add_thread_id_prefix_for_logging
2016-09-06 11:38:53 -07:00
Eno Thereska a960faf5f4 KAFKA-4105: Queryable state tests
Author: Eno Thereska <eno.thereska@gmail.com>

Reviewers: Damian Guy, Guozhang Wang

Closes #1806 from enothereska/queryable-state-tests
2016-09-04 21:49:48 -07:00
Jiangjie Qin af9fc503de KAFKA-4099; Fix the potential frequent log rolling
Author: Jiangjie Qin <becket.qin@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>

Closes #1809 from becketqin/KAFKA-4099
2016-09-02 12:49:34 -07:00
Damian Guy 8f3462552f KAFKA-4104: Queryable state metadata is sometimes invalid
If the thread or process is not the coordinator the Cluster instance in StreamPartitionAssignor will always be null. This builds an instance of the Cluster with the metadata associated with the Assignment

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #1804 from dguy/kafka-4104
2016-09-01 21:21:42 -07:00
Damian Guy 268cff7049 KAFKA-4112: Remove alpha quality label from Kafka Streams in docs
Rephrase 'alpha quality' wording in Streams section of api.html.
Couple of other minor fixes in streams.html

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Guozhang Wang, Ismael Juma, Michael G. Noll

Closes #1811 from dguy/kstreams-312
2016-09-01 12:21:55 -07:00
Rajini Sivaram 3a161db571 KAFKA-4077: Backdate system test certificates to cope with clock skew
Author: Rajini Sivaram <rajinisivaram@googlemail.com>

Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>

Closes #1810 from rajinisivaram/KAFKA-4077
2016-09-01 10:05:36 -07:00
Jason Gustafson c3b64cc8f1 KAFKA-4103: Fix regression in DumpLogSegments offsets decoder
Author: Jason Gustafson <jason@confluent.io>

Reviewers: Ismael Juma <ismael@juma.me.uk>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #1807 from hachikuji/KAFKA-4103
2016-08-31 13:39:59 -07:00
ybyzek bc397db0e1 MINOR: onCompletion: metadata valid only if no exception
Modifies example in doc change

Author: ybyzek <ybyzek@users.noreply.github.com>

Reviewers: Guozhang Wang, Ismael Juma

Closes #1805 from ybyzek/onComplete_doc
2016-08-31 12:08:59 -07:00
Rajini Sivaram 2e731a9ee0 KAFKA-3799: Enable SSL endpoint validation in system tests
Generate certificates with hostname in SubjectAlternativeName and enable hostname validation.

Author: Rajini Sivaram <rajinisivaram@googlemail.com>

Reviewers: Sriharsha Chintalapani <harsha@hortonworks.com>, Ismael Juma <ismael@juma.me.uk>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #1483 from rajinisivaram/KAFKA-3799
2016-08-31 09:14:59 -07:00
Dustin Cote 2f20a39873 KAFKA-4062: Make --print-data-log implicit if --offsets-decoder is enabled for DumpLogSegments
set print-data-log option when offset-decoder is set.  hachikuji we had talked about this one before, does this change look ok to you?

Author: Dustin Cote <dustin@confluent.io>

Reviewers: Jason Gustafson <jason@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #1797 from cotedm/KAFKA-4062
2016-08-30 21:31:09 -07:00
Shikhar Bhushan b91eeac943 KAFKA-4100: Ensure 'fields' and 'fieldsByName' are not null for Struct schemas
Author: Shikhar Bhushan <shikhar@confluent.io>

Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>

Closes #1800 from shikhar/kafka-4100
2016-08-29 19:08:52 -07:00
Jason Gustafson 9aaeb33c16 KAFKA-4098: NetworkClient should not intercept user metdata requests on disconnect
Author: Jason Gustafson <jason@confluent.io>

Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>

Closes #1798 from hachikuji/KAFKA-4098
2016-08-29 13:13:08 -07:00