Commit Graph

101 Commits

Author SHA1 Message Date
Manikumar Reddy 488ea4b9fd KAFKA-5647; Use KafkaZkClient in ReassignPartitionsCommand and PreferredReplicaLeaderElectionCommand
*  Use KafkaZkClient in ReassignPartitionsCommand
*  Use KafkaZkClient in PreferredReplicaLeaderElectionCommand
*  Updated test classes to use new methods
*  All existing tests should pass

Author: Manikumar Reddy <manikumar.reddy@gmail.com>

Reviewers: Jun Rao <junrao@gmail.com>

Closes #4260 from omkreddy/KAFKA-5647-ADMINCOMMANDS
2017-12-20 12:19:36 -08:00
Matthias J. Sax 234ec8a8af KAFKA-4857: Replace StreamsKafkaClient with AdminClient in Kafka Streams
Author: Matthias J. Sax <matthias@confluent.io>

Reviewers: Ismael Juma <ismael@juma.me.uk>, Bill Bejeck <bbejeck@gmail.com>, Guozhang Wang <wangguoz@gmail.com>

Closes #4242 from mjsax/kafka-4857-admit-client
2017-12-07 16:16:54 -08:00
Jorge Quilcate Otoya 30f08d158a KAFKA-5520: KIP-171; Extend Consumer Group Reset Offset for Stream Application
KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-171+-+Extend+Consumer+Group+Reset+Offset+for+Stream+Application

Merge changes from KIP-198

Ref: https://github.com/apache/kafka/pull/3831

Author: Jorge Quilcate Otoya <quilcate.jorge@gmail.com>
Author: Ismael Juma <ismael@juma.me.uk>
Author: Matthias J. Sax <matthias@confluent.io>
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Author: Guozhang Wang <wangguoz@gmail.com>
Author: Apurva Mehta <apurva@confluent.io>
Author: Rajini Sivaram <rajinisivaram@googlemail.com>
Author: Jason Gustafson <jason@confluent.io>
Author: Vahid Hashemian <vahidhashemian@us.ibm.com>
Author: Bill Bejeck <bill@confluent.io>
Author: Dong Lin <lindong28@gmail.com>
Author: Soenke Liebau <soenke.liebau@opencore.com>
Author: Colin P. Mccabe <cmccabe@confluent.io>
Author: Damian Guy <damian.guy@gmail.com>
Author: Xavier Léauté <xl+github@xvrl.net>
Author: Maytee Chinavanichkit <maytee.chinavanichkit@linecorp.com>
Author: Joel Hamill <git config --global user.email>
Author: Paolo Patierno <ppatierno@live.com>
Author: siva santhalingam <siva.santhalingam@gmail.com>
Author: Tommy Becker <tobecker@tivo.com>
Author: Mickael Maison <mickael.maison@gmail.com>
Author: Onur Karaman <okaraman@linkedin.com>
Author: tedyu <yuzhihong@gmail.com>
Author: Xin Li <Xin.Li@trivago.com>
Author: Magnus Edenhill <magnus@edenhill.se>
Author: Manjula K <manjula@kafka-summit.org>
Author: Hugo Louro <hmclouro@gmail.com>
Author: Jeff Widman <jeff@jeffwidman.com>
Author: bartdevylder <bartdevylder@gmail.com>
Author: Ewen Cheslack-Postava <me@ewencp.org>
Author: Jacek Laskowski <jacek@japila.pl>
Author: Tom Bentley <tbentley@redhat.com>
Author: Konstantine Karantasis <konstantine@confluent.io>

Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>

Closes #4159 from jeqo/feature/kip-171
2017-12-06 11:38:38 -08:00
Colin P. Mccabe d9cbc6b1a2 KAFKA-5811; Add Kibosh integration for Trogdor and Ducktape
For ducktape: add Kibosh to the testing Dockerfile.
Create files_unreadable_fault_spec.py.

For trogdor: create FilesUnreadableFaultSpec.java.
Add a unit test of using the Kibosh service.

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

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>

Closes #4195 from cmccabe/KAFKA-5811
2017-11-16 17:59:24 +00:00
Colin P. Mccabe 4fac83ba1f KAFKA-6060; Add workload generation capabilities to Trogdor
Previously, Trogdor only handled "Faults."  Now, Trogdor can handle
"Tasks" which may be either faults, or workloads to execute in the
background.

The Agent and Coordinator have been refactored from a
mutexes-and-condition-variables paradigm into a message passing
paradigm.  No locks are necessary, because only one thread can access
the task state or worker state.  This makes them a lot easier to reason
about.

The MockTime class can now handle mocking deferred message passing
(adding a message to an ExecutorService with a delay).  I added a
MockTimeTest.

MiniTrogdorCluster now starts up Agent and Coordinator classes in
paralle in order to minimize junit test time.

RPC messages now inherit from a common Message.java class.  This class
handles implementing serialization, equals, hashCode, etc.

Remove FaultSet, since it is no longer necessary.

Previously, if CoordinatorClient or AgentClient hit a networking
problem, they would throw an exception.  They now retry several times
before giving up.  Additionally, the REST RPCs to the Coordinator and
Agent have been changed to be idempotent.  If a response is lost, and
the request is resent, no harm will be done.

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

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

Closes #4073 from cmccabe/KAFKA-6060
2017-11-03 09:37:29 +00:00
Randall Hauch 11afff0990 KAFKA-5990: Enable generation of metrics docs for Connect (KIP-196)
A new mechanism was added recently to the Metrics framework to make it easier to generate the documentation. It uses a registry with a MetricsNameTemplate for each metric, and then those templates are used when creating the actual metrics. The metrics framework provides utilities that can generate the HTML documentation from the registry of templates.

This change moves the recently-added Connect metrics over to use these templates and to then generate the metric documentation for Connect.

This PR is based upon #3975 and can be rebased once that has been merged.

Author: Randall Hauch <rhauch@gmail.com>

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

Closes #3987 from rhauch/kafka-5990
2017-10-04 11:05:50 -07:00
Rajini Sivaram 021d8a8e96 KAFKA-5746; Add new metrics to support health checks (KIP-188)
Adds new metrics to support health checks:
1. Error rates for each request type, per-error code
2. Request size and temporary memory size
3. Message conversion rate and time
4. Successful and failed authentication rates
5. ZooKeeper latency and status
6. Client version

Author: Rajini Sivaram <rajinisivaram@googlemail.com>

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

Closes #3705 from rajinisivaram/KAFKA-5746-new-metrics
2017-09-28 21:58:59 +01:00
Bill Bejeck 271f6b5aec KAFKA-5862: Remove ZK dependency from Streams reset tool, Part I
Author: Bill Bejeck <bill@confluent.io>
Author: bbejeck <bbejeck@gmail.com>

Reviewers: Matthias J. Sax <matthias@confluent.io>, Ted Yu <yuzhihong@gmail.com>, Guozhang Wang <wangguoz@gmail.com>

Closes #3927 from bbejeck/KAFKA-5862_remove_zk_dependency_from_streams_reset_tool
2017-09-23 12:05:16 +08:00
Rajini Sivaram 96ba21e0df KAFKA-5947; Handle authentication failure in admin client, txn producer
1. Raise AuthenticationException for authentication failures in admin client
2. Handle AuthenticationException as a fatal error for transactional producer
3. Add comments to authentication exceptions

Author: Rajini Sivaram <rajinisivaram@googlemail.com>

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

Closes #3928 from rajinisivaram/KAFKA-5947-auth-failure
2017-09-21 13:58:43 +01:00
Jason Gustafson 0cf7708007 MINOR: Move request/response schemas to the corresponding object representation
This refactor achieves the following:

1. Breaks up the increasingly unmanageable `Protocol` class and moves schemas closer to their actual usage.
2. Removes the need for redundant field identifiers maintained separately in `Protocol` and the respective request/response objects.
3. Provides a better mechanism for sharing common fields between different schemas (e.g. topics, partitions, error codes, etc.).
4. Adds convenience helpers to `Struct` for common patterns (such as setting a field only if it exists).

Author: Jason Gustafson <jason@confluent.io>

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

Closes #3813 from hachikuji/protocol-schema-refactor
2017-09-19 05:12:55 +01:00
Rajini Sivaram 8fca432223 KAFKA-4764; Wrap SASL tokens in Kafka headers to improve diagnostics (KIP-152)
SASL handshake protocol changes from KIP-152.

Author: Rajini Sivaram <rajinisivaram@googlemail.com>

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

Closes #3708 from rajinisivaram/KAFKA-4764-SASL-diagnostics
2017-09-15 17:16:29 +01:00
Jason Gustafson 3b5d88febb KAFKA-5783; Add KafkaPrincipalBuilder with support for SASL (KIP-189)
Author: Jason Gustafson <jason@confluent.io>

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

Closes #3795 from hachikuji/KAFKA-5783
2017-09-14 10:16:00 +01:00
James Cheng 2fb5664bf4 KAFKA-5597: Autogenerate producer sender metrics
Subtask of https://issues.apache.org/jira/browse/KAFKA-3480

The changes are very similar to what was done for the consumer in https://issues.apache.org/jira/browse/KAFKA-5191 (pull request https://github.com/apache/kafka/pull/2993)

Author: James Cheng <jylcheng@yahoo.com>

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

Closes #3535 from wushujames/producer_sender_metrics_docs

Fix one minor naming bug
2017-09-05 17:38:58 -07:00
Colin P. Mccabe 0772fde562 KAFKA-5776; Add the Trogdor fault injection daemon
Author: Colin P. Mccabe <cmccabe@confluent.io>

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

Closes #3699 from cmccabe/trogdor-review
2017-08-25 12:29:40 -07:00
huxihx 607c3c21f6 KAFKA-5755; KafkaProducer should be refactored to use LogContext
With LogContext, each producer log item is automatically prefixed with client id and transactional id.

Author: huxihx <huxi_2b@hotmail.com>

Reviewers: Jason Gustafson <jason@confluent.io>

Closes #3703 from huxihx/KAFKA-5755
2017-08-25 10:42:40 -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
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
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
radai-rosenblatt 47ee8e954d KAFKA-4602; KIP-72 - Allow putting a bound on memory consumed by Incoming requests
this is the initial implementation.

Author: radai-rosenblatt <radai.rosenblatt@gmail.com>

Reviewers: Ewen Cheslack-Postava <me@ewencp.org>, Ismael Juma <ismael@juma.me.uk>, Rajini Sivaram <rajinisivaram@googlemail.com>, Jun Rao <junrao@gmail.com>

Closes #2330 from radai-rosenblatt/broker-memory-pool-with-muting
2017-07-26 08:19:56 +02:00
Matthias J. Sax b04bed022a KAFKA-3856; Cleanup Kafka Stream builder API (KIP-120)
Author: Matthias J. Sax <matthias@confluent.io>

Reviewers: Damian Guy <damian.guy@gmail.com>, Bill Bejeck <bbejeck@gmail.com>

Closes #2301 from mjsax/kafka-3856-topology-builder-API
2017-07-20 18:47:47 +01:00
Ismael Juma a293e1dc0c MINOR: Adjust checkstyle suppression paths to work on Windows
Use the file name whenever possible and replace / with [/\\]
when it's not.

Also remove unnecessary suppresions.

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

Reviewers: Vahid Hashemian <vahidhashemian@us.ibm.com>, Jason Gustafson <jason@confluent.io>

Closes #3431 from ijuma/fix-checkstyle-suppressions-on-windows
2017-06-28 01:47:00 +01:00
Eno Thereska 55a90938a1 MINOR: add Yahoo benchmark to nightly runs
Author: Eno Thereska <eno.thereska@gmail.com>

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

Closes #3289 from enothereska/yahoo-benchmark
2017-06-21 11:46:59 +01:00
Ismael Juma b20d9333be KAFKA-5274: AdminClient Javadoc improvements
Publish Javadoc for common.annotation package, which contains
InterfaceStability.

Finally, mark AdminClient classes with `Evolving` instead of `Unstable`.

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

Reviewers: Colin Mccabe, Gwen Shapira

Closes #3316 from ijuma/kafka-5274-admin-client-javadoc
2017-06-14 08:57:49 -07:00
Matthias J. Sax ba07d828c5 KAFKA-5362: Add EOS system tests for Streams API
Author: Matthias J. Sax <matthias@confluent.io>

Reviewers: Damian Guy <damian.guy@gmail.com>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>

Closes #3201 from mjsax/kafka-5362-add-eos-system-tests-for-streams-api
2017-06-08 14:08:54 -07:00
Ismael Juma 39eb31feae MINOR: Optimise performance of `Topic.validate()`
I included a JMH benchmark and the results follow. The
implementation in this PR takes no more than 1/10th
of the time when compared to trunk. I also included
results for an alternative implementation that is a little
slower than the one in the PR.

Trunk:
```text
TopicBenchmark.testValidate                                topic  avgt   15  134.107 ±  3.956  ns/op
TopicBenchmark.testValidate                    longer-topic-name  avgt   15  316.241 ± 13.379  ns/op
TopicBenchmark.testValidate  very-long-topic-name_with_more_text  avgt   15  636.026 ± 30.272  ns/op
```

Implementation in the PR:
```text
TopicBenchmark.testValidate                                topic  avgt   15  13.153 ± 0.383  ns/op
TopicBenchmark.testValidate                    longer-topic-name  avgt   15  26.139 ± 0.896  ns/op
TopicBenchmark.testValidate  very-long-topic-name.with_more_text  avgt   15  44.829 ± 1.390  ns/op
```

Alternative implementation where boolean validChar = Character.isLetterOrDigit(c) || c == '.' || c == '_' || c == '-';
```text
TopicBenchmark.testValidate                                topic  avgt   15  18.883 ± 1.044  ns/op
TopicBenchmark.testValidate                    longer-topic-name  avgt   15  36.696 ± 1.220  ns/op
TopicBenchmark.testValidate  very-long-topic-name_with_more_text  avgt   15  65.956 ± 0.669  ns/op
```

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

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #3234 from ijuma/optimise-topic-is-valid
2017-06-06 03:08:40 +01:00
Ismael Juma c7bc8f7d8c MINOR: Remove redundant volatile write in RecordHeaders
The JMH benchmark included shows that the redundant
volatile write causes the constructor of `ProducerRecord`
to take more than 50% longer:

ProducerRecordBenchmark.constructorBenchmark  avgt   15  24.136 ± 1.458  ns/op (before)
ProducerRecordBenchmark.constructorBenchmark  avgt   15  14.904 ± 0.231  ns/op (after)

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

Reviewers: Jason Gustafson <jason@confluent.io>

Closes #3233 from ijuma/remove-volatile-write-in-records-header-constructor
2017-06-04 10:48:34 -07:00
Colin P. Mccabe f389b71570 KAFKA-5374; Set allow auto topic creation to false when requesting node information only
It avoids the need to handle protocol downgrades and it's safe (i.e. it will never cause
the auto creation of topics).

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

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

Closes #3220 from ijuma/kafka-5374-admin-client-metadata
2017-06-03 06:26:16 +01:00
Mario Molina dc5bf4bd45 KAFKA-5218; New Short serializer, deserializer, serde
Author: Mario Molina <mmolimar@gmail.com>

Reviewers: Matthias J. Sax <matthias@confluent.io>, Damian Guy <damian.guy@gmail.com>, Michael G. Noll <michael@confluent.io>, Guozhang Wang <wangguoz@gmail.com>

Closes #3017 from mmolimar/KAFKA-5218
2017-05-31 15:09:58 -07:00
Colin P. Mccabe da9a171c99 KAFKA-5265; Move ACLs, Config, Topic classes into org.apache.kafka.common
Also introduce TopicConfig.

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

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

Closes #3120 from cmccabe/KAFKA-5265
2017-05-31 17:35:31 +01:00
Xavier Léauté c060c48285 KAFKA-5150; Reduce LZ4 decompression overhead
- reuse decompression buffers in consumer Fetcher
- switch lz4 input stream to operate directly on ByteBuffers
- avoids performance impact of catching exceptions when reaching the end of legacy record batches
- more tests with both compressible / incompressible data, multiple
  blocks, and various other combinations to increase code coverage
- fixes bug that would cause exception instead of invalid block size
  for invalid incompressible blocks
- fixes bug if incompressible flag is set on end frame block size

Overall this improves LZ4 decompression performance by up to 40x for small batches.
Most improvements are seen for batches of size 1 with messages on the order of ~100B.
We see at least 2x improvements for for batch sizes of < 10 messages, containing messages < 10kB

This patch also yields 2-4x improvements on v1 small single message batches for other compression types.

Full benchmark results can be found here
https://gist.github.com/xvrl/05132e0643513df4adf842288be86efd

Author: Xavier Léauté <xavier@confluent.io>
Author: Ismael Juma <ismael@juma.me.uk>

Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>

Closes #2967 from xvrl/kafka-5150
2017-05-31 02:22:07 +01:00
Jason Gustafson dfa3c8a92d KAFKA-5316; LogCleaner should account for larger record sets after cleaning
Author: Jason Gustafson <jason@confluent.io>

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

Closes #3142 from hachikuji/KAFKA-5316
2017-05-28 09:57:59 -07:00
Rajini Sivaram 73ca0d215e KAFKA-5320: Include all request throttling in client throttle metrics
Author: Rajini Sivaram <rajinisivaram@googlemail.com>

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

Closes #3137 from rajinisivaram/KAFKA-5320
2017-05-25 20:28:18 +01:00
Matthias J. Sax 495836a4a2 KAFKA-4923: Modify compatibility test for Exaclty-Once Semantics in Streams
- add broker compatibility system tests

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

Reviewers: Damian Guy, Eno Thereska, Guozhang Wang

Closes #2974 from mjsax/kafka-4923-add-eos-to-streams-add-broker-check-and-system-test
2017-05-21 22:16:18 -07:00
Jiangjie Qin 7fad45557e KAFKA-3995; KIP-126 Allow KafkaProducer to split and resend oversized batches
Author: Jiangjie Qin <becket.qin@gmail.com>

Reviewers: Joel Koshy <jjkoshy.w@gmail.com>, Ismael Juma <ismael@juma.me.uk>

Closes #2638 from becketqin/KAFKA-3995
2017-05-21 17:31:31 -07:00
Xavier Léauté e287523577 KAFKA-5192: add WindowStore range scan (KIP-155)
Implements range scan for keys in windowed and session stores

Modifies caching session and windowed stores to use segmented cache keys.
Cache keys are internally prefixed with their segment id to ensure key ordering in the cache matches the ordering in the underlying store for keys spread across multiple segments.
This should also result in fewer cache keys getting scanned for queries spanning only some segments.

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

Reviewers: Damian Guy, Guozhang Wang

Closes #3027 from xvrl/windowstore-range-scan
2017-05-18 17:02:51 -07:00
Konstantine Karantasis 45f2261763 KAFKA-3487: Support classloading isolation in Connect (KIP-146)
Author: Konstantine Karantasis <konstantine@confluent.io>

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

Closes #3028 from kkonstantine/KAFKA-3487-Support-classloading-isolation-in-Connect
2017-05-18 10:39:15 -07:00
Ismael Juma 972b754536 KAFKA-3267; Describe and Alter Configs Admin APIs (KIP-133)
Author: Ismael Juma <ismael@juma.me.uk>

Reviewers: Jun Rao <junrao@gmail.com>

Closes #3076 from ijuma/kafka-3267-describe-alter-configs-protocol
2017-05-18 06:51:02 +01:00
Colin P. Mccabe 9815e18fef KAFKA-3266; Describe, Create and Delete ACLs Admin APIs (KIP-140)
Includes server-side code, protocol and AdminClient.

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

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

Closes #2941 from cmccabe/KAFKA-3266
2017-05-18 03:20:30 +01:00
Guozhang Wang c64cfd2e2b KAFKA-5231: Bump up producer epoch when sending abort txn markers on InitPid
Author: Guozhang Wang <wangguoz@gmail.com>

Reviewers: Jason Gustafson, Jun Rao

Closes #3066 from guozhangwang/K5231-bump-up-epoch-when-abort-txn
2017-05-17 18:05:12 -07:00
Rajini Sivaram 4c75f31a5f KAFKA-5179; Log connection termination during authentication
Author: Rajini Sivaram <rajinisivaram@googlemail.com>

Reviewers: Ismael Juma, Jun Rao

Closes #2980 from rajinisivaram/KAFKA-5179
2017-05-15 18:13:20 -04:00
Colin P. Mccabe 4aed28d189 KAFKA-3265; Add a public AdminClient API in Java (KIP-117)
Author: Colin P. Mccabe <cmccabe@confluent.io>

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

Closes #2472 from cmccabe/KAFKA-3265
2017-05-02 00:20:22 +01:00
Michael Andre Pearce 6185bc0276 KAFKA-4208; Add Record Headers
As per KIP-82

Adding record headers api to ProducerRecord, ConsumerRecord
Support to convert from protocol to api added Kafka Producer, Kafka Fetcher (Consumer)
Updated MirrorMaker, ConsoleConsumer and scala BaseConsumer
Add RecordHeaders and RecordHeader implementation of the interfaces Headers and Header

Some bits using are reverted to being Java 7 compatible, for the moment until KIP-118 is implemented.

Author: Michael Andre Pearce <Michael.Andre.Pearce@me.com>

Reviewers: Radai Rosenblatt <radai.rosenblatt@gmail.com>, Jiangjie Qin <becket.qin@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>

Closes #2772 from michaelandrepearce/KIP-82
2017-04-28 19:18:27 -07:00
Apurva Mehta a82f194b21 KAFKA-4818; Exactly once transactional clients
Author: Apurva Mehta <apurva@confluent.io>

Reviewers: Guozhang Wang <wangguoz@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>

Closes #2840 from apurvam/exactly-once-transactional-clients
2017-04-27 14:11:36 -07:00
Damian Guy f69d94158c KAFKA-5059: Implement Transactional Coordinator
Author: Damian Guy <damian.guy@gmail.com>
Author: Guozhang Wang <wangguoz@gmail.com>
Author: Apurva Mehta <apurva@confluent.io>

Reviewers: Guozhang Wang, Jason Gustafson, Apurva Mehta, Jun Rao

Closes #2849 from dguy/exactly-once-tc
2017-04-26 14:10:38 -07:00
Matthias J. Sax 148f8c2545 KAFKA-4986; Producer per StreamTask support (KIP-129)
Enable producer per task if exactly-once config is enabled.

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

Reviewers: Eno Thereska <eno@confluent.io>, Damian Guy <damian.guy@gmail.com>, Ismael Juma <ismael@juma.me.uk>

Closes #2773 from mjsax/exactly-once-streams-producer-per-task
2017-04-14 15:19:52 +01:00
Matthias J. Sax 865d82af2c KAFKA-4990; Request/response classes for transactions (KIP-98)
Author: Matthias J. Sax <matthias@confluent.io>
Author: Guozhang Wang <wangguoz@gmail.com>
Author: Jason Gustafson <jason@confluent.io>

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

Closes #2799 from mjsax/kafka-4990-add-api-stub-config-parameters-request-types
2017-04-07 12:07:25 +01:00
Ben Stopford 0baea2ac13 KIP-101: Alter Replication Protocol to use Leader Epoch rather than High Watermark for Truncation
This PR replaces https://github.com/apache/kafka/pull/2743 (just raising from Confluent repo)

This PR describes the addition of Partition Level Leader Epochs to messages in Kafka as a mechanism for fixing some known issues in the replication protocol. Full details can be found here:

[KIP-101 Reference](https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Epoch+rather+than+High+Watermark+for+Truncation)

*The key elements are*:
- Epochs are stamped on messages as they enter the leader.
- Epochs are tracked in both leader and follower in a new checkpoint file.
- A new API allows followers to retrieve the leader's latest offset for a particular epoch.
- The logic for truncating the log, when a replica becomes a follower, has been moved from Partition into the ReplicaFetcherThread
- When partitions are added to the ReplicaFetcherThread they are added in an initialising state. Initialising partitions request leader epochs and then truncate their logs appropriately.

This test provides a good overview of the workflow `EpochDrivenReplicationProtocolAcceptanceTest.shouldFollowLeaderEpochBasicWorkflow()`

The corrupted log use case is covered by the test
`EpochDrivenReplicationProtocolAcceptanceTest.offsetsShouldNotGoBackwards()`

Remaining work: There is a do list here: https://docs.google.com/document/d/1edmMo70MfHEZH9x38OQfTWsHr7UGTvg-NOxeFhOeRew/edit?usp=sharing

Author: Ben Stopford <benstopford@gmail.com>
Author: Jun Rao <junrao@gmail.com>

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

Closes #2808 from benstopford/kip-101-v2
2017-04-06 14:51:09 -07:00
Apurva Mehta bdf4cba047 KAFKA-4817; Add idempotent producer semantics
This is from the KIP-98 proposal.

The main points of discussion surround the correctness logic, particularly the Log class where incoming entries are validated and duplicates are dropped, and also the producer error handling to ensure that the semantics are sound from the users point of view.

There is some subtlety in the idempotent producer semantics. This patch only guarantees idempotent production upto the point where an error has to be returned to the user. Once we hit a such a non-recoverable error, we can no longer guarantee message ordering nor idempotence without additional logic at the application level.

In particular, if an application wants guaranteed message order without duplicates, then it needs to do the following in the error callback:

1. Close the producer so that no queued batches are sent. This is important for guaranteeing ordering.
2. Read the tail of the log to inspect the last message committed. This is important for avoiding duplicates.

Author: Apurva Mehta <apurva@confluent.io>
Author: hachikuji <jason@confluent.io>
Author: Apurva Mehta <apurva.1618@gmail.com>
Author: Guozhang Wang <wangguoz@gmail.com>
Author: fpj <fpj@apache.org>
Author: Jason Gustafson <jason@confluent.io>

Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>

Closes #2735 from apurvam/exactly-once-idempotent-producer
2017-04-02 19:41:44 -07:00
Colin P. Mccabe d345d53e4e KAFKA-4902; Utils#delete should correctly handle I/O errors and symlinks
Author: Colin P. Mccabe <cmccabe@confluent.io>

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

Closes #2691 from cmccabe/KAFKA-4902
2017-03-30 13:38:09 +01:00
Jason Gustafson 5bd06f1d54 KAFKA-4816; Message format changes for idempotent/transactional producer (KIP-98)
Author: Jason Gustafson <jason@confluent.io>

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

Closes #2614 from hachikuji/exactly-once-message-format
2017-03-24 19:38:43 +00:00