Added support for customizing the HTTP response headers for Kafka Connect as described in KIP-577.
Author: Jeff Huang <jeff.huang@confluent.io>
Reviewer: Randall Hauch <rhauch@gmail.com>
This PR works to improve high watermark checkpointing performance.
`ReplicaManager.checkpointHighWatermarks()` was found to be a major contributor to GC pressure, especially on Kafka clusters with high partition counts and low throughput.
Added a JMH benchmark for `checkpointHighWatermarks` which establishes a
performance baseline. The parameterized benchmark was run with 100, 1000 and
2000 topics.
Modified `ReplicaManager.checkpointHighWatermarks()` to avoid extra copies and cached
the Log parent directory Sting to avoid frequent allocations when calculating
`File.getParent()`.
A few clean-ups:
* Changed all usages of Log.dir.getParent to Log.parentDir and Log.dir.getParentFile to
Log.parentDirFile.
* Only expose public accessor for `Log.dir` (consistent with `Log.parentDir`)
* Removed unused parameters in `Partition.makeLeader`, `Partition.makeFollower` and `Partition.createLogIfNotExists`.
Benchmark results:
| Topic Count | Ops/ms | MB/sec allocated |
|-------------|---------|------------------|
| 100 | + 51% | - 91% |
| 1000 | + 143% | - 49% |
| 2000 | + 149% | - 50% |
Reviewers: Lucas Bradstreet <lucas@confluent.io>. Ismael Juma <ismael@juma.me.uk>
Co-authored-by: Gardner Vickers <gardner@vickers.me>
Co-authored-by: Ismael Juma <ismael@juma.me.uk>
Correct the Connect worker logic to properly disable the new topic status (KIP-558) feature when `topic.tracking.enable=false`, and fix automatic topic status reset after a connector is deleted.
Also adds new `ConnectorTopicsIntegrationTest` and expanded unit tests.
Reviewers: Randall Hauch <rhauch@gmail.com>
Added plugin isolation unit tests for various scenarios, with a `TestPlugins` class that compiles and builds multiple test plugins without them being on the classpath and verifies that the Plugins and DelegatingClassLoader behave properly. These initially failed for several cases, but now pass since the issues have been fixed.
KAFKA-8340 and KAFKA-8819 are closely related, and this fix corrects the problems reported in both issues.
Author: Greg Harris <gregh@confluent.io>
Reviewers: Chris Egerton <chrise@confluent.io>, Magesh Nandakumar <mageshn@confluent.io>, Konstantine Karantasis <konstantine@confluent.io>, Randall Hauch <rhauch@gmail.com>
Implementation of [KIP-382 "MirrorMaker 2.0"](https://cwiki.apache.org/confluence/display/KAFKA/KIP-382%3A+MirrorMaker+2.0)
Author: Ryanne Dolan <ryannedolan@gmail.com>
Author: Arun Mathew <arunmathew88@gmail.com>
Author: In Park <inpark@cloudera.com>
Author: Andre Price <obsoleted@users.noreply.github.com>
Author: christian.hagel@rio.cloud <christian.hagel@rio.cloud>
Reviewers: Eno Thereska <eno.thereska@gmail.com>, William Hammond <william.t.hammond@gmail.com>, Viktor Somogyi <viktorsomogyi@gmail.com>, Jakub Korzeniowski, Tim Carey-Smith, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Arun Mathew, Jeremy-l-ford, vpernin, Oleg Kasian <oleg.kasian@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Qihong Chen, Sriharsha Chintalapani <sriharsha@apache.org>, Jun Rao <junrao@gmail.com>, Randall Hauch <rhauch@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#6295 from ryannedolan/KIP-382
Implemented KIP-507 to secure the internal Connect REST endpoints that are only for intra-cluster communication. A new V2 of the Connect subprotocol enables this feature, where the leader generates a new session key, shares it with the other workers via the configuration topic, and workers send and validate requests to these internal endpoints using the shared key.
Currently the internal `POST /connectors/<connector>/tasks` endpoint is the only one that is secured.
This change adds unit tests and makes some small alterations to system tests to target the new `sessioned` Connect subprotocol. A new integration test ensures that the endpoint is actually secured (i.e., requests with missing/invalid signatures are rejected with a 400 BAD RESPONSE status).
Author: Chris Egerton <chrise@confluent.io>
Reviewed: Konstantine Karantasis <konstantine@confluent.io>, Randall Hauch <rhauch@gmail.com>
Implemented KIP-495 to expose a new `admin/loggers` endpoint for the Connect REST API that lists the current log levels and allows the caller to change log levels.
Author: Arjun Satish <arjun@confluent.io>
Reviewer: Randall Hauch <rhauch@gmail.com>
Move the generator checkstyle suppressions to a special section, rather
than mixing them in with the other sections. For generated code, do not
complain about variable names or cyclic complexity.
FieldType.java: remove isInteger since it isn't used anywhere. This way, we
don't have to decide whether a UUID is an integer or not (there are arguments
for both choices). Add FieldType#serializationIsDifferentInFlexibleVersions
and FieldType#isVariableLength.
HeaderGenerator: add the ability to generate static imports. Add
IsNullConditional, VersionConditional, and ClauseGenerator as easier ways of
generating "if" statements.
New Java Authorizer API and a new out-of-the-box authorizer (AclAuthorizer) that implements the new interface.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Prior to this change an NPE is raised when calling AssignedTasks.close
under the following conditions:
1. EOS is enabled
2. The task was in a suspended state
The cause for the NPE is that when a clean close is requested for a
StreamTask the StreamTask tries to commit. However, in the suspended
state there is no producer so ultimately an NPE is thrown for the
contained RecordCollector in flush.
The fix put forth in this commit is to have AssignedTasks call
closeSuspended when it knows the underlying StreamTask is suspended.
Note also that this test is quite involved. I could have just tested
that AssignedTasks calls closeSuspended when appropriate, but that is
testing, IMO, a detail of the implementation and doesn't actually verify
we reproduced the original problem as it was described. I feel much more
confident that we are reproducing the behavior - and we can test exactly
the conditions that lead to it - when testing across AssignedTasks and
StreamTask. I believe this is an additional support for the argument of
eventually consolidating the state split across classes.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
ZkUtils was removed so we don't need this anymore.
Also:
* Fix ZkSecurityMigrator and ReplicaManagerTest not to
reference ZkClient classes.
* Remove references to zkclient in various `log4j.properties`
and `import-control.xml`.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
Implementation to enable policy for Connector Client config overrides. This is
implemented per the KIP-458.
Reviewers: Randall Hauch <rhauch@gmail.com>
Fix registration of Connect REST extensions to prevent deadlocks when extensions get the list of connectors before the herder is available. Added integration test to check the behavior.
Author: Chris Egerton <cegerton@oberlin.edu>
Reviewers: Arjun Satish <arjun@confluent.io>, Randall Hauch <rhauch@gmail.com>
Extend Connect's integration test framework to add or remove workers to EmbeddedConnectCluster, and choosing whether to fail the test on ungraceful service shutdown. Also added more JavaDoc and other minor improvements.
Author: Konstantine Karantasis <konstantine@confluent.io>
Reviewers: Arjun Satish <arjun@confluent.io>, Randall Hauch <rhauch@gmail.com>
Closes#6342 from kkonstantine/KAFKA-8014
The test `org.apache.kafka.connect.runtime.rest.RestServerTest#testCORSEnabled` assumes Jersey client can send restricted HTTP headers(`Origin`).
Jersey client uses `sun.net.www.protocol.http.HttpURLConnection`.
`sun.net.www.protocol.http.HttpURLConnection` drops restricted headers(`Host`, `Keep-Alive`, `Origin`, etc) based on static property `allowRestrictedHeaders`.
This property is initialized in a static block by reading Java system property `sun.net.http.allowRestrictedHeaders`.
So, if classloader loads `HttpURLConnection` before we set `sun.net.http.allowRestrictedHeaders=true`, then all subsequent changes of this system property won't take any effect(which happens if `org.apache.kafka.connect.integration.ExampleConnectIntegrationTest` is executed before `RestServerTest`).
To prevent this, we have to either make sure we set `sun.net.http.allowRestrictedHeaders=true` as early as possible or do not rely on this system property at all.
This PR adds test dependency on `httpcomponents-client` which doesn't depend on `sun.net.http.allowRestrictedHeaders` system property. Thus none of existing tests should interfere with `RestServerTest`.
Author: Alex Diachenko <sansanichfb@gmail.com>
Reviewers: Randall Hauch, Konstantine Karantasis, Gwen Shapira
Closes#6236 from avocader/KAFKA-7799
See also KIP-183.
This implements the following algorithm:
AdminClient sends ElectPreferredLeadersRequest.
KafakApis receives ElectPreferredLeadersRequest and delegates to
ReplicaManager.electPreferredLeaders()
ReplicaManager delegates to KafkaController.electPreferredLeaders()
KafkaController adds a PreferredReplicaLeaderElection to the EventManager,
ReplicaManager.electPreferredLeaders()'s callback uses the
delayedElectPreferredReplicasPurgatory to wait for the results of the
election to appear in the metadata cache. If there are no results
because of errors, or because the preferred leaders are already leading
the partitions then a response is returned immediately.
In the EventManager work thread the preferred leader is elected as follows:
The EventManager runs PreferredReplicaLeaderElection.process()
process() calls KafkaController.onPreferredReplicaElectionWithResults()
KafkaController.onPreferredReplicaElectionWithResults()
calls the PartitionStateMachine.handleStateChangesWithResults() to
perform the election (asynchronously the PSM will send LeaderAndIsrRequest
to the new and old leaders and UpdateMetadataRequest to all brokers)
then invokes the callback.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Jun Rao <junrao@gmail.com>
Expose a programmatic way to bring up a Kafka and Zk cluster through Java API to facilitate integration tests for framework level changes in Kafka Connect. The Kafka classes would be similar to KafkaEmbedded in streams. The new classes would reuse the kafka.server.KafkaServer classes from :core, and provide a simple interface to bring up brokers in integration tests.
Signed-off-by: Arjun Satish <arjunconfluent.io>
Author: Arjun Satish <arjun@confluent.io>
Author: Arjun Satish <wicknicks@users.noreply.github.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5516 from wicknicks/connect-integration-test
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>
Add CreatePartitionsRequest.PartitionDetails similar to CreateTopicsRequest.TopicDetails to avoid references from `common.requests` package to `clients`.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Development of EasyMock and PowerMock has stagnated while Mockito
continues to be actively developed. With the new Java release cadence,
it's a problem to depend on libraries that do bytecode manipulation
and are not actively maintained. In addition, Mockito is also
easier to use.
While updating the tests, I attempted to go from failing test to
passing test. In cases where the updated test passed on the first
attempt, I artificially broke it to ensure the test was still doing its
job.
I included a few improvements that were helpful while making these
changes:
1. Better exception if there are no nodes in `leastLoadedNodes`
2. Always close the producer in `KafkaProducerTest`
3. requestsInFlight producer metric should not hold a reference to
`Sender`
Finally, `Metadata` is no longer final so that we don't need
`PowerMock` to mock it. It's an internal class, so it's OK.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Viktor Somogyi <viktorsomogyi@gmail.com>, Dong Lin <lindong28@gmail.com>
Closes#5691 from ijuma/kafka-7438-mockito
Previously, we depicted creating a Jackson serde for every pojo class, which becomes a burden in practice. There are many ways to avoid this and just have a single serde, so we've decided to model this design choice instead.
Reviewers: Viktor Somogyi <viktorsomogyi@gmail.com>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Reviewers: Colin Patrick McCabe <colin@cmccabe.xyz>, Jun Rao <junrao@gmail.com>
Co-authored-by: Piyush Vijay <pvijay@apple.com>
Co-authored-by: Andy Coates <big-andy-coates@users.noreply.github.com>
This PR provides the implementation for KIP-285 and also a reference implementation for authenticating BasicAuth credentials using JAAS LoginModule
Author: Magesh Nandakumar <magesh.n.kumar@gmail.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Arjun Satish <wicknicks@users.noreply.github.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4931 from mageshn/KIP-285
This KIP adds the following functionality related to SASL/OAUTHBEARER:
1) Allow clients (both brokers when SASL/OAUTHBEARER is the inter-broker protocol as well as non-broker clients) to flexibly retrieve an access token from an OAuth 2 authorization server based on the declaration of a custom login CallbackHandler implementation and have that access token transparently and automatically transmitted to a broker for authentication.
2) Allow brokers to flexibly validate provided access tokens when a client establishes a connection based on the declaration of a custom SASL Server CallbackHandler implementation.
3) Provide implementations of the above retrieval and validation features based on an unsecured JSON Web Token that function out-of-the-box with minimal configuration required (i.e. implementations of the two types of callback handlers mentioned above will be used by default with no need to explicitly declare them).
4) Allow clients (both brokers when SASL/OAUTHBEARER is the inter-broker protocol as well as non-broker clients) to transparently retrieve a new access token in the background before the existing access token expires in case the client has to open new connections.
* 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.
* unify skipped records metering
* log warnings when things get skipped
* tighten up metrics usage a bit
### Testing strategy:
Unit testing of the metrics and the logs should be sufficient.
Author: John Roesler <john@confluent.io>
Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Closes#4812 from vvcephei/kip-274-streams-skip-metrics
Dynamic update of listeners as described in KIP-226. This includes:
- Addition of new listeners with listener-prefixed security configs
- Removal of existing listeners
- Password encryption
- sasl.jaas.config property for broker's JAAS config prefixed with listener and mechanism name
**[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
* 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
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
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
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
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
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
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
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
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
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