This change is the client-side part of KIP-360. It identifies cases where it is safe to abort a transaction, bump the producer epoch, and allow the application to continue without closing the producer. In these cases, when KafkaProducer.abortTransaction() is called, the producer sends an InitProducerId following the transaction abort, which causes the producer epoch to be bumped. The application can then start a new transaction and continue processing.
For recoverable errors in the idempotent producer, the epoch is bumped locally. In-flight requests for partitions with an error are rewritten to reflect the new epoch, and in-flights of all other partitions are allowed to complete using the old epoch.
Reviewers: Boyang Chen <boyang@confluent.io>, Jason Gustafson <jason@confluent.io>
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>
This change mainly have 2 components:
1. extend the existing transactions_test.py to also try out new sendTxnOffsets(groupMetadata) API to make sure we are not introducing any regression or compatibility issue
a. We shrink the time window to 10 seconds for the txn timeout scheduler on broker so that we could trigger expiration earlier than later
2. create a completely new system test class called group_mode_transactions_test which is more complicated than the existing system test, as we are taking rebalance into consideration and using multiple partitions instead of one. For further breakdown:
a. The message count was done on partition level, instead of global as we need to visualize
the per partition order throughout the test. For this sake, we extend ConsoleConsumer to print out the data partition as well to help message copier interpret the per partition data.
b. The progress count includes the time for completing the pending txn offset expiration
c. More visibility and feature improvements on TransactionMessageCopier to better work under either standalone or group mode.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Corrects a flaw leading to an exception while building topologies that include both:
* A foreign-key join with the result not explicitly materialized
* An operation after the join that requires source materialization
Also corrects a flaw in TopologyTestDriver leading to output records being enqueued in the wrong order under some (presumably rare) circumstances.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
This PR is collaborated by Guozhang Wang and John Roesler. It is a significant tech debt cleanup on task management and state management, and is broken down by several sub-tasks listed below:
Extract embedded clients (producer and consumer) into RecordCollector from StreamTask.
guozhangwang#2
guozhangwang#5
Consolidate the standby updating and active restoring logic into ChangelogReader and extract out of StreamThread.
guozhangwang#3
guozhangwang#4
Introduce Task state life cycle (created, restoring, running, suspended, closing), and refactor the task operations based on the current state.
guozhangwang#6
guozhangwang#7
Consolidate AssignedTasks into TaskManager and simplify the logic of changelog management and task management (since they are already moved in step 2) and 3)).
guozhangwang#8
guozhangwang#9
Also simplified the StreamThread logic a bit as the embedded clients / changelog restoration logic has been moved into step 1) and 2).
guozhangwang#10
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Bruno Cadonna <bruno@confluent.io>, Boyang Chen <boyang@confluent.io>
This patch adds a new API to the producer to implement transactional offset commit fencing through the group coordinator as proposed in KIP-447. This PR mainly changes on the Producer end for compatible paths to old `sendOffsetsToTxn(offsets, groupId)` vs new `sendOffsetsToTxn(offsets, groupMetadata)`.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
Allow null as a valid default for tagged fields. Fix a bunch of cases where this would previously result in null pointer dereferences.
Also allow inferring FieldSpec#versions based on FieldSpec#taggedVersions. Prefix 'key' with an underscore when it is used in the generated code, to avoid potential name collisions if someone names an RPC field "key".
Allow setting setting hexadecimal constants and 64-bit contstants.
Add a lot more test cases to SimpleExampleMessage.json.
Reviewers: Jason Gustafson <jason@confluent.io>
Rather than maintain hand coded protocol serialization code, Streams could use the same code-generation framework as Clients/Core.
There isn't a perfect match, since the code generation framework includes an assumption that you're generating "protocol messages", rather than just arbitrary blobs, but I think it's close enough to justify using it, and improving it over time.
Using the code generation allows us to drop a lot of detail-oriented, brittle, and hard-to-maintain serialization logic in favor of a schema spec.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Boyang Chen <boyang@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
The RPC code generator should support using the ByteBuffer class in addition to byte arrays. By using the ByteBuffer class, we can avoid performing a copy in many situations. Also modify TestByteBufferDataTest to test the new feature.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Guozhang Wang <wangguoz@gmail.com>
Migrate this integration test to use TopologyTestDriver instead of running 3 Streams instances.
Dropped one test that was attempting to produce specific interleavings. If anything, these should be verified deterministically by unit testing.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@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>
Key improvements with this PR:
* tasks will remain available for IQ during a rebalance (but not during restore)
* continue restoring and processing standby tasks during a rebalance
* continue processing active tasks during rebalance until the RecordQueue is empty*
* only revoked tasks must suspended/closed
* StreamsPartitionAssignor tries to return tasks to their previous consumers within a client
* but do not try to commit, for now (pending KAFKA-7312)
Reviewers: John Roesler <john@confluent.io>, Boyang Chen <boyang@confluent.io>, Guozhang Wang <wangguoz@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
https://issues.apache.org/jira/browse/KAFKA-3705
Allows for a KTable to map its value to a given foreign key and join on another KTable keyed on that foreign key. Applies the joiner, then returns the tuples keyed on the original key. This supports updates from both sides of the join.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Matthias J. Sax <mjsax@apache.org>, John Roesler <john@confluent.io>, Boyang Chen <boyang@confluent.io>, Christopher Pettitt <cpettitt@confluent.io>, Bill Bejeck <bbejeck@gmail.com>, Jan Filipiak <Jan.Filipiak@trivago.com>, pgwhalen, Alexei Daniline
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>
Implemented KIP-440 to allow Connect converters to use record headers when serializing or deserializing keys and values. This change is backward compatible in that the new methods default to calling the older existing methods, so existing Converter implementations need not be changed. This changes the WorkerSinkTask and WorkerSourceTask to use the new converter methods, but Connect's existing Converter implementations and the use of converters for internal topics are intentionally not modified. Added unit tests.
Author: Yaroslav Tkachenko <sapiensy@gmail.com>
Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Ewen Cheslack-Postava <me@ewencp.org>, 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.
Part of supporting KIP-213 ( https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable ). Murmur3 hash is used as a hashing mechanism in KIP-213 for the large range of uniqueness. The Murmur3 class and tests are ported directly from Apache Hive, with no alterations to the code or dependencies.
Author: Adam Bellemare <adam.bellemare@wishabi.com>
Reviewers: John Roesler <vvcephei@users.noreply.github.com>, Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>
Closes#7271 from bellemare/murmur3hash
This PR makes two changes to code in the ReplicaManager.updateFollowerFetchState path, which is in the hot path for follower fetches. Although calling ReplicaManager.updateFollowerFetch state is inexpensive on its own, it is called once for each partition every time a follower fetch occurs.
1. updateFollowerFetchState no longer calls maybeExpandIsr when the follower is already in the ISR. This avoid repeated expansion checks.
2. Partition.maybeIncrementLeaderHW is also in the hot path for ReplicaManager.updateFollowerFetchState. Partition.maybeIncrementLeaderHW calls Partition.remoteReplicas four times each iteration, and it performs a toSet conversion. maybeIncrementLeaderHW now avoids generating any intermediate collections when updating the HWM.
**Benchmark results for Partition.updateFollowerFetchState on a r5.xlarge:**
Old:
```
1288.633 ±(99.9%) 1.170 ns/op [Average]
(min, avg, max) = (1287.343, 1288.633, 1290.398), stdev = 1.037
CI (99.9%): [1287.463, 1289.802] (assumes normal distribution)
```
New (when follower fetch offset is updated):
```
261.727 ±(99.9%) 0.122 ns/op [Average]
(min, avg, max) = (261.565, 261.727, 261.937), stdev = 0.114
CI (99.9%): [261.605, 261.848] (assumes normal distribution)
```
New (when follower fetch offset is the same):
```
68.484 ±(99.9%) 0.025 ns/op [Average]
(min, avg, max) = (68.446, 68.484, 68.520), stdev = 0.023
CI (99.9%): [68.460, 68.509] (assumes normal distribution)
```
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
This creates a test that generates sustained connections against Kafka. There
are three different components we can stress with this, KafkaConsumer,
KafkaProducer, and AdminClient. This test tries use minimal bandwidth per
connection to reduce overhead impacts.
This test works by creating a threadpool that creates connections and then
maintains a central pool of connections at a specified keepalive rate. The
keepalive action varies by which component is being stressed:
* KafkaProducer: Sends one single produce record. The configuration for
the produce request uses the same key/value generator as the ProduceBench
test.
* KafkaConsumer: Subscribes to a single partition, seeks to the end, and
then polls a minimal number of records. Each consumer connection is its
own consumer group, and defaults to 1024 bytes as FETCH_MAX_BYTES to keep
traffic to a minimum.
* AdminClient: Makes an API call to get the nodes in the cluster.
NOTE: This test is designed to be run alongside a ProduceBench test for a
specific topic, due to the way the Consumer test polls a single partition.
There may be no data returned by the consumer test if this is run on its own.
The connection should still be kept alive, but with no data returned.
Author: Scott Hendricks <scott.hendricks@confluent.io>
Reviewers: Stanislav Kozlovski, Gwen Shapira
Closes#7289 from scott-hendricks/trunk
This PR changes the TxnOffsetCommit protocol to auto-generated types, and add more unit test coverage to the plain OffsetCommit protocol.
Reviewers: Jason Gustafson <jason@confluent.io>
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>
Add the AlterPartitionReassignments and ListPartitionReassignments APIs. Also remove an unused methodlength suppression for KafkaAdminClient.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Viktor Somogyi <viktorsomogyi@gmail.com>
Implement KIP-480, which specifies that the default partitioner should use a "sticky" partitioning strategy for records that have a null key.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Lucas Bradstreet <lucasbradstreet@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Jun Rao <junrao@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
This patch is part of KIP-345. We are aiming to support batch leave group request issued from admin client. This diff is the first effort to bump leave group request version.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
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>
This upgrade exposes a number of new options, including the WriteBufferManager which -- along with existing TableConfig options -- allows users to limit the total memory used by RocksDB across instances. This can alleviate some cascading OOM potential when, for example, a large number of stateful tasks are suddenly migrated to the same host.
The RocksDB docs guarantee backwards format compatibility across versions
Reviewers: Matthias J. Sax <mjsax@apache.org>, Bill Bejeck <bbejeck@gmail.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>
This is the first diff for the implementation of JoinGroup logic for static membership. The goal of this diff contains:
* Add group.instance.id to be unique identifier for consumer instances, provided by end user;
Modify group coordinator to accept JoinGroupRequest with/without static membership, refactor the logic for readability and code reusability.
* Add client side support for incorporating static membership changes, including new config for group.instance.id, apply stream thread client id by default, and new join group exception handling.
* Increase max session timeout to 30 min for more user flexibility if they are inclined to tolerate partial unavailability than burdening rebalance.
* Unit tests for each module changes, especially on the group coordinator logic. Crossing the possibilities like:
6.1 Dynamic/Static member
6.2 Known/Unknown member id
6.3 Group stable/unstable
6.4 Leader/Follower
The rest of the 345 change will be broken down to 4 separate diffs:
* Avoid kicking out members through rebalance.timeout, only do the kick out through session timeout.
* Changes around LeaveGroup logic, including version bumping, broker logic, client logic, etc.
* Admin client changes to add ability to batch remove static members
* Deprecate group.initial.rebalance.delay
Reviewers: Liquan Pei <liquanpei@gmail.com>, Stanislav Kozlovski <familyguyuser192@windowslive.com>, Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Viktor Somogyi <viktorsomogyi@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Rajini Sivaram <rajinisivaram@googlemail.com>, Ismael Juma <ismael@juma.me.uk>
Currently close() only awaits completion of pending produce requests. If there is a transaction ongoing, it may be dropped. For example, if one thread is calling commitTransaction() and another calls close(), then the commit may never happen even if the caller is willing to wait for it (by using a long timeout). What's more, the thread blocking in commitTransaction() will be stuck since the result will not be completed once the producer has shutdown.
This patch ensures that 1) completing transactions are awaited, 2) ongoing transactions are aborted, and 3) pending callbacks are completed before close() returns.
Reviewers: Jason Gustafson <jason@confluent.io>
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
Reviewers: Guozhang Wang <guozhang@confluent.io>, Ismael Juma <ismael@confluent.io>, Jorge Quilcate Otoya <quilcate.jorge@gmail.com>, John Roesler <john@confluent.io>, Bill Bejeck <bill@confluent.io>
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>
* 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>