* Allow the Trogdor agent to be started in "exec mode", where it simply
runs a single task and exits after it is complete.
* For AgentClient and CoordinatorClient, allow the user to pass the path
to a file containing JSON, instead of specifying the JSON object in the
command-line text itself. This means that we can get rid of the bash
scripts whose only function was to load task specs into a bash string
and run a Trogdor command.
* Print dates and times in a human-readable way, rather than as numbers
of milliseconds.
* When listing tasks or workers, output human-readable tables of
information.
* Allow the user to filter on task ID name, task ID pattern, or task
state.
* Support a --json flag to provide raw JSON output if desired.
Reviewed-by: David Arthur <mumrah@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
Reviewers: Bill Bejeck <bill@confluent.io>, John Roesler <john@confluent.io>, Guozhang Wang <guozhang@confluent.io>, Ryanne Dolan <ryannedolan@gmail.com>, Ismael Juma <ismael@confuent.io>
This patch adds a framework to automatically generate the request/response classes for Kafka's protocol. The code will be updated to use the generated classes in follow-up patches. Below is a brief summary of the included components:
**buildSrc/src**
The message generator code is here. This code is automatically re-run by gradle when one of the schema files changes. The entire directory is processed at once to minimize the number of times we have to start a new JVM. We use Jackson to translate the JSON files into Java objects.
**clients/src/main/java/org/apache/kafka/common/protocol/Message.java**
This is the interface implemented by all automatically generated messages.
**clients/src/main/java/org/apache/kafka/common/protocol/MessageUtil.java**
Some utility functions used by the generated message code.
**clients/src/main/java/org/apache/kafka/common/protocol/Readable.java, Writable.java, ByteBufferAccessor.java**
The generated message code uses these classes for writing to a buffer.
**clients/src/main/message/README.md**
This README file explains how the JSON schemas work.
**clients/src/main/message/\*.json**
The JSON files in this directory implement every supported version of every Kafka API. The unit tests automatically validate that the generated schemas match the hand-written schemas in our code. Additionally, there are some things like request and response headers that have schemas here.
**clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashSet.java**
I added an optimization here for empty sets. This is useful here because I want all messages to start with empty sets by default prior to being loaded with data. This is similar to the "empty list" optimizations in the `java.util.ArrayList` class.
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Ismael Juma <ismael@juma.me.uk>, Bob Barrett <bob.barrett@outlook.com>, Jason Gustafson <jason@confluent.io>
KIP-368 implementation to enable periodic re-authentication of SASL clients. Also adds a broker configuration option to terminate client connections that do not re-authenticate within the configured interval.
Modified several classes' `equals` methods and simplified a complex method to
reduce the NPath complexity so they could be removed from the checkstyle
suppressions that were required with the recent move to Java 8 and upgrade
of Checkstyle: https://github.com/apache/kafka/pull/5046.
Reviewers: Robert Yokota <rayokota@gmail.com>, Arjun Satish <arjun@confluent.io>, Ismael Juma <ismael@juma.me.uk>
A call to `kafka-consumer-groups --describe --group ...` can result in NullPointerException for two reasons:
1) `Fetcher.fetchOffsetsByTimes()` may return too early, without sending list offsets request for topic partitions that are not in cached metadata.
2) `ConsumerGroupCommand.getLogEndOffsets()` and `getLogStartOffsets()` assumed that endOffsets()/beginningOffsets() which eventually call Fetcher.fetchOffsetsByTimes(), would return a map with all the topic partitions passed to endOffsets()/beginningOffsets() and that values are not null. Because of (1), null values were possible if some of the topic partitions were already known (in metadata cache) and some not (metadata cache did not have entries for some of the topic partitions). However, even with fixing (1), endOffsets()/beginningOffsets() may return a map with some topic partitions missing, when list offset request returns a non-retriable error. This happens in corner cases such as message format on broker is before 0.10, or maybe in cases of some other errors.
Testing:
-- added unit test to verify fix in Fetcher.fetchOffsetsByTimes()
-- did some manual testing with `kafka-consumer-groups --describe`, causing NPE. Was not able to reproduce any NPE cases with DescribeConsumerGroupTest.scala,
Reviewers: Jason Gustafson <jason@confluent.io>
We need to use the same lock for metric update and read to avoid NPE and concurrent modification exceptions. Sensor add/remove/update are synchronized on Sensor since they access lists and maps that are not thread-safe. Reporters are notified of metrics add/remove while holding (Sensor, Metrics) locks and reporters may synchronize on the reporter lock. Metric read may be invoked by metrics reporters while holding a reporter lock. So read/update cannot be synchronized using Sensor since that could lead to deadlock. This PR introduces a new lock in Sensor for update/read.
Locking order:
- Sensor#add: Sensor -> Metrics -> MetricsReporter
- Metrics#removeSensor: Sensor -> Metrics -> MetricsReporter
- KafkaMetric#metricValue: MetricsReporter -> Sensor#metricLock
- Sensor#record: Sensor -> Sensor#metricLock
Reviewers: Jun Rao <junrao@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
This commit allows secrets in Connect configs to be externalized and replaced with variable references of the form `${provider:[path:]key}`, where the "path" is optional.
There are 2 main additions to `org.apache.kafka.common.config`: a `ConfigProvider` and a `ConfigTransformer`. The `ConfigProvider` is an interface that allows key-value pairs to be provided by an external source for a given "path". An a TTL can be associated with the key-value pairs returned from the path. The `ConfigTransformer` will use instances of `ConfigProvider` to replace variable references in a set of configuration values.
In the Connect framework, `ConfigProvider` classes can be specified in the worker config, and then variable references can be used in the connector config. In addition, the herder can be configured to restart connectors (or not) based on the TTL returned from a `ConfigProvider`. The main class that performs restarts and transformations is `WorkerConfigTransformer`.
Finally, a `configs()` method has been added to both `SourceTaskContext` and `SinkTaskContext`. This allows connectors to get configs with variables replaced by the latest values from instances of `ConfigProvider`.
Most of the other changes in the Connect framework are threading various objects through classes to enable the above functionality.
Author: Robert Yokota <rayokota@gmail.com>
Author: Ewen Cheslack-Postava <me@ewencp.org>
Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5068 from rayokota/KAFKA-6886-connect-secrets
* Set --source, --target and --release to 1.8.
* Build Scala 2.12 by default.
* Remove some conditionals in the build file now that Java 8
is the minimum version.
* Bump the version of Jetty, Jersey and Checkstyle (the newer
versions require Java 8).
* Fixed issues uncovered by the new version if Checkstyle.
* A couple of minor updates to handle an incompatible source
change in the new version of Jetty.
* Add dependency to jersey-hk2 to fix failing tests caused
by Jersey upgrade.
* Update release script to use Java 8 and to take into account
that Scala 2.12 is now built by default.
* While we're at it, bump the version of Gradle, Gradle plugins,
ScalaLogging, JMH and apache directory api.
* Minor documentation updates including the readme and upgrade
notes. A number of Streams Java 7 examples can be removed
subsequently.
Implement destroying tasks and workers. This means erasing all record of them on the Coordinator and the Agent.
Workers should be identified by unique 64-bit worker IDs, rather than by the names of the tasks they are implementing. This ensures that when a task is destroyed and re-created with the same task ID, the old workers will be not be treated as part of the new task instance.
Fix some return results from RPCs. In some cases RPCs were returning values that were never used. Attempting to re-create the same task ID with different arguments should fail. Add RequestConflictException to represent HTTP error code 409 (CONFLICT) for this scenario.
If only one worker in a task stops, don't stop all the other workers for that task, unless the worker that stopped had an error.
Reviewers: Anna Povzner <anna@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
Added configs to ProducerBenchSpec:
topicPrefix: name of topics will be of format topicPrefix + topic index. If not provided, default is "produceBenchTopic".
partitionsPerTopic: number of partitions per topic. If not provided, default is 1.
replicationFactor: replication factor per topic. If not provided, default is 3.
The behavior of producer bench is changed such that if some or all topics already exist (with topic names = topicPrefix + topic index), and they have the same number of partitions as requested, the worker uses those topics and does not fail. The producer bench fails if one or more existing topics has number of partitions that is different from expected number of partitions.
Added unit test for WorkerUtils -- for existing methods and new methods.
Fixed bug in MockAdminClient, where createTopics() would over-write existing topic's replication factor and number of partitions while correctly completing the appropriate futures exceptionally with TopicExistsException.
Reviewers: Colin P. Mccabe <cmccabe@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
This is a straight-forward change that make the name of the partition assignor to be aligned with Streams.
Reviewers: Matthias J. Sax <mjsax@apache.org>
Submitting a fail safe fix for rare IOExceptions on symbolic links.
The fix is submitted without a test case since it does seem easy to reproduce such type of failures (just having a broken symbolic link does not reproduce the issue) and it's considered pretty low risk.
If accepted, needs to be ported at least to 1.0, if not 0.11
Author: Konstantine Karantasis <konstantine@confluent.io>
Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4481 from kkonstantine/KAFKA-6288-Broken-symlink-interrupts-scanning-the-plugin-path
**[KIP-145](https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect) has been accepted, and this PR implements KIP-145 except without the SMTs.**
Changed the Connect API and runtime to support message headers as described in [KIP-145](https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect).
The new `Header` interface defines an immutable representation of a Kafka header (key-value pair) with support for the Connect value types and schemas. This interface provides methods for easily converting between many of the built-in primitive, structured, and logical data types.
The new `Headers` interface defines an ordered collection of headers and is used to track all headers associated with a `ConnectRecord` (and thus `SourceRecord` and `SinkRecord`). This does allow multiple headers with the same key. The `Headers` contains methods for adding, removing, finding, and modifying headers. Convenience methods allow connectors and transforms to easily use and modify the headers for a record.
A new `HeaderConverter` interface is also defined to enable the Connect runtime framework to be able to serialize and deserialize headers between the in-memory representation and Kafka’s byte[] representation. A new `SimpleHeaderConverter` implementation has been added, and this serializes to strings and deserializes by inferring the schemas (`Struct` header values are serialized without the schemas, so they can only be deserialized as `Map` instances without a schema.) The `StringConverter`, `JsonConverter`, and `ByteArrayConverter` have all been extended to also be `HeaderConverter` implementations. Each connector can be configured with a different header converter, although by default the `SimpleHeaderConverter` is used to serialize header values as strings without schemas.
Unit and integration tests are added for `ConnectHeader` and `ConnectHeaders`, the two implementation classes for headers. Additional test methods are added for the methods added to the `Converter` implementations. Finally, the `ConnectRecord` object is already used heavily, so only limited tests need to be added while quite a few of the existing tests already cover the changes.
Author: Randall Hauch <rhauch@gmail.com>
Reviewers: Arjun Satish <arjun@confluent.io>, Ted Yu <yuzhihong@gmail.com>, Magesh Nandakumar <magesh.n.kumar@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4319 from rhauch/kafka-5142-b
* KAFKA-3625: Add public test utils for Kafka Streams
- add new artifact test-utils
- add TopologyTestDriver
- add MockTime, TestRecord, add TestRecordFactory
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Damian Guy <damian.guy@gmail.com>, Bill Bejeck <bill@confluent.io>
Enable dynamic broker configuration (see KIP-226 for details). Includes
- Base implementation to allow specific broker configs and custom configs to be dynamically updated
- Extend DescribeConfigsRequest/Response to return all synonym configs and their sources in the order of precedence
- Extend AdminClient to alter dynamic broker configs
- Dynamic update of SSL keystores
Reviewers: Ted Yu <yuzhihong@gmail.com>, Jason Gustafson <jason@confluent.io>
- Add capability to create delegation token
- Add authentication based on delegation token.
- Add capability to renew/expire delegation tokens.
- Add units tests and integration tests
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Jun Rao <junrao@gmail.com>
Closes#3616 from omkreddy/KAFKA-4541
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
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
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
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
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
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
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
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
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
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
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
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
Author: Rajini Sivaram <rajinisivaram@googlemail.com>
Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#3137 from rajinisivaram/KAFKA-5320
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
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
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
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
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
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
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
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