Commit Graph

4042 Commits

Author SHA1 Message Date
Jason Gustafson bf3bfd6749 KAFKA-5747; Producer snapshot loading should cover schema errors
Author: Jason Gustafson <jason@confluent.io>

Reviewers: Apurva Mehta <apurva@confluent.io>, Ismael Juma <ismael@juma.me.uk>

Closes #3688 from hachikuji/KAFKA-5747
2017-08-21 10:22:53 -07:00
Ismael Juma 8a5a84dc67 KAFKA-5753; ShellTest.testRunProgramWithErrorReturn fails on macOS
Author: Ismael Juma <ismael@juma.me.uk>

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>

Closes #3706 from ijuma/fix-shell-test-mac
2017-08-21 17:49:11 +01:00
Ismael Juma 26c65953ed MINOR: Deprecate LogConfig.Compact
It actually refers to the `delete` cleanup policy.

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

Reviewers: Jason Gustafson <jason@confluent.io>

Closes #3634 from ijuma/fix-misleading-compact-log-config
2017-08-20 07:54:36 +01:00
Jason Gustafson 6896f1ddb7 MINOR: Ensure consumer logging has clientId/groupId context
This patch ensures that the consumer groupId and clientId are available in all log messages which makes debugging much easier when a single application has multiple consumer instances. To make this easier, I've added a new `LogContext` object which builds a log prefix similar to the broker-side `kafka.utils.Logging` mixin. Additionally this patch changes the log level for a couple minor cases:

- Consumer wakeup events are now logged at DEBUG instead of TRACE
- Heartbeat enabling/disabling is now logged at DEBUG instead of TRACE

Author: Jason Gustafson <jason@confluent.io>

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>

Closes #3676 from hachikuji/log-consumer-wakeups
2017-08-19 11:17:02 -07:00
Ismael Juma ed96523a2c KAFKA-4501; Java 9 compilation and runtime fixes
Compilation error fixes:
- Avoid ambiguity error when appending to Properties in Scala
code (https://github.com/scala/bug/issues/10418)
- Use position() and limit() to fix ambiguity issue (
https://github.com/scala/bug/issues/10418#issuecomment-316364778)
- Disable findBugs if Java 9 is used (
https://github.com/findbugsproject/findbugs/issues/105)

Compilation warning fixes:
- Avoid deprecated Class.newInstance in Utils.newInstance
- Silence a few Java 9 deprecation warnings
- var -> val and unused fixes

Runtime error fixes:
- Introduce Base64 class that works in Java 7 and Java 9

Also:
- Set --release option if building with Java 9

Note that tests involving EasyMock (https://github.com/easymock/easymock/issues/193)
or PowerMock (https://github.com/powermock/powermock/issues/783)
will fail as neither supports Java 9 currently.

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

Reviewers: Jason Gustafson <jason@confluent.io>

Closes #3647 from ijuma/kafka-4501-support-java-9
2017-08-19 08:55:29 +01:00
Eno Thereska 3e22c1c04a KAFKA-5725; More failure testing
Author: Eno Thereska <eno.thereska@gmail.com>

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

Closes #3656 from enothereska/minor-add-more-tests
2017-08-18 18:13:02 +01:00
Damian Guy 75c78e9692 KAFKA-5668; fetch across stores in CompositeReadOnlyWindowStore & CompositeReadOnlySessionStore
Fix range queries in `CompositeReadOnlyWindowStore` and `CompositeReadOnlySessionStore` to fetch across all stores (was previously just looking in the first store)

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

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #3685 from dguy/kafka-5668
2017-08-18 17:59:33 +01:00
huxihx 3457c4761a KAFKA-5745; makeLeader should invoke `convertHWToLocalOffsetMetadata` before marking it as leader
Author: huxihx <huxi_2b@hotmail.com>

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

Closes #3682 from huxihx/KAFKA-5745
2017-08-18 15:58:59 +01:00
Colin P. Mccabe a4720b25a3 KAFKA-5744; ShellTest: add tests for attempting to run nonexistent program, error return
Author: Colin P. Mccabe <cmccabe@confluent.io>

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

Closes #3679 from cmccabe/KAFKA-5744
2017-08-18 11:38:45 +01:00
Xavier Léauté 520e651d53 KAFKA-5742: support ZK chroot in system tests
Author: Xavier Léauté <xavier@confluent.io>

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

Closes #3677 from xvrl/support-zk-chroot-in-tests
2017-08-17 15:14:32 -07:00
Jason Gustafson 334a30bcff KAFKA-5730; Consumer should invoke async commit callback before sync commit returns
Author: Jason Gustafson <jason@confluent.io>

Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <me@ewencp.org>

Closes #3666 from hachikuji/KAFKA-5730
2017-08-17 15:04:03 -07:00
Bill Bejeck efe4f6540a KAFKA-5733: RocksDB bulk load with lower number of levels
This is to complete Bill's PR #3664 on KAFKA-5733, incorporating the suggestion in https://github.com/facebook/rocksdb/issues/2734.

Some minor changes: move `open = true` in `openDB`.

Author: Bill Bejeck <bill@confluent.io>
Author: Guozhang Wang <wangguoz@gmail.com>

Reviewers: Guozhang Wang <wangguoz@gmail.com>, Damian Guy <damian.guy@gmail.com>

Closes #3681 from guozhangwang/K5733-rocksdb-bulk-load
2017-08-17 14:02:10 -07:00
Konstantine Karantasis 72eacbea5b KAFKA-5567: Connect sink worker should commit offsets of original topic partitions
Author: Konstantine Karantasis <konstantine@confluent.io>

Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #3499 from kkonstantine/KAFKA-5567-With-transformations-that-mutate-the-topic-partition-committing-offsets-should-to-refer-to-the-original-topic-partition
2017-08-16 14:43:29 -07:00
Colin P. Mccabe bf34a5b740 KAFKA-5737; KafkaAdminClient thread should be daemon
Author: Colin P. Mccabe <cmccabe@confluent.io>

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

Closes #3674 from cmccabe/KAFKA-5737
2017-08-16 13:22:44 +01:00
Randall Hauch 3b1cea60e9 KAFKA-5731; Corrected how the sink task worker updates the last committed offsets
Prior to this change, it was possible for the synchronous consumer commit request to be handled before previously-submitted asynchronous commit requests. If that happened, the out-of-order handlers improperly set the last committed offsets, which then became inconsistent with the offsets the connector task is working with.

This change ensures that the last committed offsets are updated only for the most recent commit request, even if the consumer reorders the calls to the callbacks.

Author: Randall Hauch <rhauch@gmail.com>

Reviewers: Jason Gustafson <jason@confluent.io>

Closes #3662 from rhauch/kafka-5731
2017-08-15 14:16:00 -07:00
Jason Gustafson 869ef5579f HOTFIX: ConsoleConsumer using wrong old consumer config value for auto.offset.reset
Author: Jason Gustafson <jason@confluent.io>

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

Closes #3665 from hachikuji/hotfix-auto-reset-console-consumer
2017-08-14 13:52:02 -07:00
Andrey Dyachkov c0f7a77058 KAFKA-4643; Improve test coverage of StreamsKafkaClient
The commit brings improved test coverage for StreamsKafkaClientTest.java

Author: Andrey Dyachkov <andrey.dyachkov@zalando.de>

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

Closes #3663 from adyach/kafka-4643
2017-08-14 18:24:43 +01:00
Rajini Sivaram 21ea4b1d2a KAFKA-5595; Ensure client connection ids are not reused too quickly
When there are broker delays that cause a response to take longer
than `connections.max.idle.ms`, connections may be closed by the
broker (as well as by the client) before the response is processed.
If the port is reused, broker may send the outstanding response to
a new connection with the reused port. The new connection will end
up with correlation id mismatch, requiring process restart. This
is also a security exposure since clients receive response
intended for the wrong connection.

Author: Rajini Sivaram <rajinisivaram@googlemail.com>

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

Closes #3530 from rajinisivaram/KAFKA-5595
2017-08-14 17:07:50 +01:00
Jason Gustafson 8265a43897 MINOR: Safer handling of requests prior to SASL authentication
This implements two improvements for request handling prior to SASL authentication:

1. Only parse request types that are allowed prior to authentication.
2. Limit the maximum request size (the default is 100Mb).

Author: Jason Gustafson <jason@confluent.io>

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Ismael Juma <ismael@juma.me.uk>

Closes #3558 from hachikuji/minor-restrict-presasl-request-parsing
2017-08-14 14:05:40 +01:00
Damian Guy e19c37e591 KAFKA-5673; refactor KeyValueStore hierarchy to make MeteredKeyValueStore outermost
refactor StateStoreSuppliers such that a `MeteredKeyValueStore`  is the outermost store.

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

Reviewers: Eno Thereska <eno.thereska@gmail.com>, Guozhang Wang <wangguoz@gmail.com>

Closes #3592 from dguy/key-value-store-refactor
2017-08-14 10:02:32 +01:00
Ismael Juma f4dc5ac15c MINOR: AdminClient should register with `AppInfoParser`
Also make "created" message more consistent across clients.

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

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Jason Gustafson <jason@confluent.io>

Closes #3658 from ijuma/admin-client-should-register-with-app-info-parser
2017-08-14 00:11:58 +01:00
Guozhang Wang b6102cf4b2 HOTFIX: missing imports and version on web docs 2017-08-11 13:53:47 -07:00
Guozhang Wang d2774e302f KAFKA-5727: Add Streams quickstart tutorial as an archetype project
0. Minor fixes on the existing examples to merge all on a single input topic; also do not use `common.utils.Exit` as it is for internal usage only.

1. Add the archetype project for the quickstart. Steps to try it out:

  a. `mvn install` on the quickstart directory.
  b. `mvn archetype:generate \
-DarchetypeGroupId=org.apache.kafka \
-DarchetypeArtifactId=streams-quickstart-java \
-DarchetypeVersion=1.0.0-SNAPSHOT \
-DgroupId=streams-quickstart \
-DartifactId=streams-quickstart \
-Dversion=0.1 \
-Dpackage=StreamsQuickstart \
-DinteractiveMode=false` at any directory to create the project.
  c. build the streams jar with version `1.0.0-SNAPSHOT` to local maven repository with `./gradlew installAll`; `cd streams-quickstart; mvn clean package`
  d. create the input / output topics, start the console producer and consumer.
  e. start the program: `mvn exec:java -Dexec.mainClass=StreamsQuickstart.Pipe/LineSplit/WordCount`.
  f. type data on console producer and observe data on console consumer.

Author: Guozhang Wang <wangguoz@gmail.com>

Reviewers: Damian Guy <damian.guy@gmail.com>, Bill Bejeck <bbejeck@gmail.com>, Ewen Cheslack-Postava <me@ewencp.org>, Eno Thereska <eno.thereska@gmail.com>

Closes #3630 from guozhangwang/KMinor-streams-quickstart-tutorial
2017-08-11 12:19:28 -07:00
Damian Guy 3e69ce8015 KAFKA-5702; extract refactor StreamThread
Extracted `TaskManager` to handle all task related activities.
Make `StandbyTaskCreator`, `TaskCreator`, and `RebalanceListener` static classes so they must define their dependencies and can be testing independently of `StreamThread`
Added interfaces between `StreamPartitionAssignor` & `StreamThread` to reduce coupling.

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

Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, Eno Thereska <eno.thereska@gmail.com>

Closes #3624 from dguy/stream-thread-refactor
2017-08-11 12:14:01 +01:00
Bill Bejeck b2b529522d MINOR: First cut at porting State Store docs to AK
Author: Bill Bejeck <bill@confluent.io>

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

Closes #3629 from bbejeck/docs-updates-for-kip-167
2017-08-11 09:35:26 +01:00
Bill Bejeck 8ffb1a1fed KAFKA-5711: batch restore should handle deletes
Author: Bill Bejeck <bill@confluent.io>

Reviewers: Damian Guy <damian.guy@gmail.com>, Guozhang Wang <wangguoz@gmail.com>

Closes #3644 from bbejeck/KAFKA-5711_bulk_restore_should_handle_deletes
2017-08-10 11:31:55 -07:00
Jason Gustafson 017a21c604 MINOR: Remove unneeded error handlers in deprecated request objects
These handlers were previously used on the broker to handle uncaught exceptions, but now the broker users the new Java request objects exclusively.

Author: Jason Gustafson <jason@confluent.io>

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

Closes #3646 from hachikuji/remove-old-request-error-handlers
2017-08-10 11:07:53 -07:00
Xavier Léauté 75b0f30c44 KAFKA-5077; fix GC logging arguments for Java 9
Author: Xavier Léauté <xavier@confluent.io>

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

Closes #2863 from xvrl/fix-jdk9-gc-logs
2017-08-10 02:06:53 +01:00
Jason Gustafson 23df2199d7 MINOR: Add missing deprecations on old request objects
Author: Jason Gustafson <jason@confluent.io>

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

Closes #3651 from hachikuji/add-missing-request-deprecations
2017-08-10 01:31:41 +01:00
Evgeny Veretennikov d2a2b80cc6 KAFKA-5507; Check if classpath is empty in kafka-run-class.sh
Author: Evgeny Veretennikov <evg.veretennikov@gmail.com>

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

Closes #3421 from evis/kafka-run-class-check-classpath
2017-08-09 23:58:25 +01:00
Ismael Juma 69d3d4aad6 KAFKA-5470; Replace -XX:+DisableExplicitGC with -XX:+ExplicitGCInvokesConcurrent in kafka-run-class
This is important because Bits.reserveMemory calls System.gc() hoping to free native
memory in order to avoid throwing an OutOfMemoryException. This call is currently
a no-op due to -XX:+DisableExplicitGC.

It's worth mentioning that -XX:MaxDirectMemorySize can be used to increase the
amount of native memory available for allocation of direct byte buffers.

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

Reviewers: Apurva Mehta <apurva@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>, Jason Gustafson <jason@confluent.io>

Closes #3371 from ijuma/kafka-5470-explicit-gc-invokes-concurrent
2017-08-09 23:38:57 +01:00
Eno Thereska 889da45dd0 MINOR: Improve instructions for running system tests with docker
Author: Eno Thereska <eno.thereska@gmail.com>

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

Closes #3649 from enothereska/minor-docker-docs
2017-08-09 23:35:13 +01:00
Soenke Liebau 1ad74f5b76 KAFKA-5629; ConsoleConsumer should respect auto.offset.reset if specified on the command line
when "auto.offset.reset" property is specified on the command line but overridden by the code during startup. Currently the ConsoleConsumer silently overrides that setting, which can create confusing behavior.

Author: Soenke Liebau <soenke.liebau@opencore.com>

Reviewers: Jason Gustafson <jason@confluent.io>

Closes #3566 from soenkeliebau/KAFKA-5629
2017-08-09 13:22:05 -07:00
Damian Guy c35c479813 KAFKA-5717; InMemoryKeyValueStore should delete keys with null values during restore
Fixed a bug in the InMemoryKeyValueStore restoration where a key with a `null` value is written in to the map rather than being deleted.

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

Reviewers: Bill Bejeck <bbejeck@gmail.com>, Guozhang Wang <wangguoz@gmail.com>

Closes #3650 from dguy/kafka-5717
2017-08-09 20:03:28 +01:00
Ewen Cheslack-Postava a593db6a2b MINOR: Standardize logging of Worker-level messages from Tasks and Connectors
This ensures all logs have the connector/task ID, whether tasks are source or sink, and formats them consistently.

Author: Ewen Cheslack-Postava <me@ewencp.org>

Reviewers: Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Jason Gustafson <jason@confluent.io>

Closes #3639 from ewencp/standardize-connector-task-logging
2017-08-09 09:07:27 -07:00
Randall Hauch 1a653c813c KAFKA-5704: Corrected Connect distributed startup behavior to allow older brokers to auto-create topics
When a Connect distributed worker starts up talking with broker versions 0.10.1.0 and later, it will use the AdminClient to look for the internal topics and attempt to create them if they are missing. Although the AdminClient was added in 0.11.0.0, the AdminClient uses APIs to create topics that existed in 0.10.1.0 and later. This feature works as expected when Connect uses a broker version 0.10.1.0 or later.

However, when a Connect distributed worker starts up using a broker older than 0.10.1.0, the AdminClient is not able to find the required APIs and thus will throw an UnsupportedVersionException. Unfortunately, this exception is not caught and instead causes the Connect worker to fail even when the topics already exist.

This change handles the UnsupportedVersionException by logging a debug message and doing nothing. The existing producer logic will get information about the topics, which will cause the broker to create them if they don’t exist and broker auto-creation of topics is enabled. This is the same behavior that existed prior to 0.11.0.0, and so this change restores that behavior for brokers older than 0.10.1.0.

This change also adds a system test that verifies Connect works with a variety of brokers and is able to run source and sink connectors. The test verifies that Connect can read from the internal topics when the connectors are restarted.

Author: Randall Hauch <rhauch@gmail.com>

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

Closes #3641 from rhauch/kafka-5704
2017-08-08 20:20:41 -07:00
Xavier Léauté b8cf976865 MINOR: support retrieving cluster_id in system tests
ewencp would be great to cherry-pick this back into 0.11.x if possible

Author: Xavier Léauté <xavier@confluent.io>

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

Closes #3645 from xvrl/system-test-cluster-id
2017-08-08 19:58:47 -07:00
Ismael Juma fb8edbdc63 MINOR: Update dependencies for 1.0.0 release
Notable updates:

1. Gradle 4.1 includes a number of performance and
CLI improvements as well as initial Java 9 support.

2. Scala 2.12.3 has substantial compilation time
improvements.

3. lz4-java 1.4 allows us to remove a workaround in
KafkaLZ4BlockInputStream (not done in this PR).

4. snappy-java 1.1.4 improved performance of compression (5%)
and decompression (20%). There was a slight increase in the
compressed size in one of our tests.

Not updated:

1. PowerMock due to a couple of regressions. I investigated one of them
and filed https://github.com/powermock/powermock/issues/828.

2. Jackson, which will be done via #3631.

3. Rocksdb, which will be done via #3519.

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

Reviewers: Jason Gustafson <jason@confluent.io>

Closes #3619 from ijuma/update-deps-for-1.0.0
2017-08-09 01:11:39 +01:00
huxihx 649276abb2 KAFKA-2360; Extract producer-specific configs out of the common PerfConfig
Separate `batch.size`, `message-size` and `compression-code` from PerfConfig to a newly-created ProducerPerfConfig in order to hide them in ConsumerPerf tool.

Author: huxihx <huxi_2b@hotmail.com>

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

Closes #3613 from huxihx/KAFKA-2360
2017-08-08 23:22:57 +01:00
Ismael Juma 98d4a48338 MINOR: Remove unused GroupState.state field
This field doesn't seem to be used and the value for
`AwaitingSync` seems to be wrong (it seems like it
should have been `2` instead of `5`).

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

Reviewers: Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>

Closes #3572 from ijuma/remove-unused-group-state-field
2017-08-08 23:15:57 +01:00
Colin P. Mccabe c9ffab1622 KAFKA-5658; Fix AdminClient request timeout handling bug resulting in continual BrokerNotAvailableExceptions
The AdminClient does not properly clear calls from the callsInFlight structure.
Later, in an effort to clear the lingering call objects, it closes the connection
they are associated with. This disrupts new incoming calls, which then get
BrokerNotAvailableException.

This patch fixes this bug by properly removing completed calls from the
callsInFlight structure. It also adds the Call#aborted flag, which
ensures that we throw the right exception (TimeoutException instead of
DisconnectException) and only abort a connection once -- even if there
is a similar bug in the future which causes old Call objects to linger.

Author: Colin P. Mccabe <cmccabe@confluent.io>

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

Closes #3584 from cmccabe/KAFKA-5658
2017-08-08 09:38:22 +01:00
Bill Bejeck 57770dd235 KAFKA-5701: fix flaky RocksDBStore unit test
1. Remove separate thread from test failing periodically due to race condition.
2. Remove anonymous `AbstractNotifyingBatchingRestoreCallback` declare as concrete inner class `RocksDBBatchingRestoreCallback` and set as package private variable.  Class is static so it has to initialize it's dependency on `RocksDBStore`

Author: Bill Bejeck <bill@confluent.io>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #3640 from bbejeck/KAFKA-5701_fix_flaky_unit_test
2017-08-07 15:20:24 -07:00
Bill Bejeck 1f0ad0121e HOTFIX: fix for standby tasks using batching restore
Author: Bill Bejeck <bill@confluent.io>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #3625 from bbejeck/HOTFIX_need_to_correct_stanby_task_restoration_to_use_new_restore_api
2017-08-07 09:54:43 -07:00
ppatierno f15cdc73dd KAFKA-5516: Formatting verifiable producer/consumer output in a similar fashion
Author: ppatierno <ppatierno@live.com>
Author: Paolo Patierno <ppatierno@live.com>

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

Closes #3434 from ppatierno/verifiable-consumer-producer
2017-08-07 08:10:42 -07:00
Damian Guy 2010aa067f MINOR: add memory management section to streams docs
Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Guozhang Wang <wangguoz@gmail.com>, Eno Thereska <eno.thereska@gmail.com>

Closes #3604 from dguy/memory-management-docs
2017-08-07 10:17:17 +01:00
huxihx 1cd86284e8 KAFKA-5700; Producer should not drop header information when splitting batches
Producer should not drop header information when splitting batches.  This PR also corrects a minor typo in Sender.java, where `spitting and retrying` should be `splitting and retrying`.

Author: huxihx <huxi_2b@hotmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jiangjie Qin <becket.qin@gmail.com>

Closes #3620 from huxihx/KAFKA-5700
2017-08-06 22:25:52 -07:00
Paolo Patierno 1d291a4219 KAFKA-5643: Using _DUCKTAPE_OPTIONS has no effect on executing tests
Added handling of _DUCKTAPE_OPTIONS (mainly for enabling debugging)

Author: Paolo Patierno <ppatierno@live.com>

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

Closes #3578 from ppatierno/kafka-5643
2017-08-06 21:50:43 -07:00
Colin P. Mccabe d637c134c8 KAFKA-5602; ducker-ak: support --custom-ducktape
Support a --custom-ducktape flag which allows developers to install
their own versions of ducktape into Docker images.  This is helpful for
ducktape development.

Author: Colin P. Mccabe <cmccabe@confluent.io>

Reviewers: Ewen Cheslack-Postava <me@ewencp.org>, Ismael Juma <ismael@juma.me.uk>

Closes #3539 from cmccabe/KAFKA-5602
2017-08-05 09:04:05 +01:00
Jason Gustafson 70796c3ca3 MINOR: Fix error response handler for controlled shutdown v0
Author: Jason Gustafson <jason@confluent.io>

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

Closes #3627 from hachikuji/minor-fix-controlled-shutdown-error-response
2017-08-04 13:08:12 -07:00
Ewen Cheslack-Postava 22611aca9b KAFKA-5535: Handle null values in ExtractField
Author: Ewen Cheslack-Postava <me@ewencp.org>

Reviewers: Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Jason Gustafson <jason@confluent.io>

Closes #3559 from ewencp/kafka-5535-extract-field-null
2017-08-04 10:25:21 -07:00