Brokers are supposed to force SASL clients to re-authenticate (and kill such connections in the absence of a timely and successful re-authentication) when KIP-368 SASL Re-Authentication is enabled via a positive connections.max.reauth.ms configuration value. There was a flaw in the logic that caused connections to not be killed in the absence of a timely and successful re-authentication if the client did not leverage the SaslAuthenticateRequest API (which was defined in KIP-152).
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
KIP-320 improved fetch semantics by adding leader epoch validation. This relies on
reliable propagation of leader epoch information from the controller. Unfortunately, we
have encountered a bug during partition reassignment in which the leader epoch in the
controller context does not get properly updated. This causes UpdateMetadata requests
to be sent with stale epoch information which results in the metadata caches on the
brokers falling out of sync.
This bug has existed for a long time, but it is only a problem due to the new epoch
validation done by the client. Because the client includes the stale leader epoch in its
requests, the leader rejects them, yet the stale metadata cache on the brokers prevents
the consumer from getting the latest epoch. Hence the consumer cannot make progress
while a reassignment is ongoing.
Although it is straightforward to fix this problem in the controller for the new releases
(which this patch does), it is not so easy to fix older brokers which means new clients
could still encounter brokers with this bug. To address this problem, this patch also
modifies the client to treat the leader epoch returned from the Metadata response as
"unreliable" if it comes from an older version of the protocol. The client in this case will
discard the returned epoch and it won't be included in any requests.
Also, note that the correct epoch is still forwarded to replicas correctly in the
LeaderAndIsr request, so this bug does not affect replication.
Reviewers: Jun Rao <junrao@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Ismael Juma <ismael@juma.me.uk>
Remove in catch clause and move it to the callback.
Reviewers: John Roesler <john@confluent.io>, Matthias J. Sax <mjsax@apache.org>, Guozhang Wang <wangguoz@gmail.com>
If a non-consumer group is specified in `describeConsumerGroup`, the future will hang indefinitely because the future callback is never completed. This patch fixes the problem by completing the future exceptionally with an `IllegalArgumentException`.
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Jason Gustafson <jason@confluent.io>
This patch fixes an NPE in `DefaultMetadataUpdater` due to an inconsistency in event expectations. Whenever there is an authentication failure, we were treating it as a failed update even if was from a separate connection from an inflight metadata request. This patch fixes the problem by making the `MetadataUpdater` api clearer in terms of the events that are handled.
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
This patch fixes a bug in `SocketServer` in the expiration of connections which have not re-authenticated quickly enough. Previously these connections were left hanging, but now they are properly closed and cleaned up. This was one cause of the flaky test failures in `EndToEndAuthorizationTest.testNoDescribeProduceOrConsumeWithoutTopicDescribeAcl`.
Reviewers: Jason Gustafson<jason@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
This reverts commit 90043d5f as it caused a regression in some cases:
> Caused by: java.io.IOException: Stream frame descriptor corrupted
> at org.apache.kafka.common.record.KafkaLZ4BlockInputStream.readHeader(KafkaLZ4BlockInputStream.java:132)
> at org.apache.kafka.common.record.KafkaLZ4BlockInputStream.<init>(KafkaLZ4BlockInputStream.java:78)
> at org.apache.kafka.common.record.CompressionType$4.wrapForInput(CompressionType.java:110)
I will investigate why after, but I want to get the safe fix into 2.4.0.
The reporter of KAFKA-9203 has verified that reverting this change
makes the problem go away.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Changes the ProducerMetadata to longer record a sentinel TOPIC_EXPIRY_NEEDS_UPDATE upon topic map emplacement, and instead set the expiry time directly. Previously the expiry time was being updated for all touched topics after a metadata fetch was processed, which could be seconds/minutes in the future.
Additionally propagates the current time further in the Producer, which should reduce the total number of current-time calls.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Rajini Sivaram <rajinisivaram@googlemail.com>
This PR is a follow-up of #7087, fixing typos, styles, etc.
cc/ big-andy-coates ijuma
Author: Lee Dongjin <dongjin@apache.org>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#7217 from dongjinleekr/feature/trivial-admin-javadoc
If a field is not marked as ignorable, we should raise an exception if it has been set to a non-default value. This check already exists in `Message.write`, so this patch adds it to `Message.toStruct`. Additionally, we fix several fields which should have been marked ignorable and we fix some related test assertions.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Manikumar Reddy <manikumar.reddy@gmail.com>, Colin Patrick McCabe <cmccabe@apache.org>
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>
After a number of last minute bugs were found stemming from the incremental closing of lost tasks in StreamsRebalanceListener#onPartitionsLost, a safer approach to this edge case seems warranted. We initially wanted to be as "future-proof" as possible, and avoid baking further protocol assumptions into the code that may be broken as the protocol evolves. This meant that rather than simply closing all active tasks and clearing all associated state in #onPartitionsLost(lostPartitions) we would loop through the lostPartitions/lost tasks and remove them one by one from the various data structures/assignments, then verify that everything was empty in the end. This verification in particular has caused us significant trouble, as it turns out to be nontrivial to determine what should in fact be empty, and if so whether it is also being correctly updated.
Therefore, before worrying about it being "future-proof" it seems we should make sure it is "present-day-proof" and implement this callback in the safest possible way, by blindly clearing and closing all active task state. We log all the relevant state (at debug level) before clearing it, so we can at least tell from the logs whether/which emptiness checks were being violated.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Bill Bejeck <bbejeck@gmail.com>, Andrew Choi <andchoi@linkedin.com>
This patch removes the explicit version check pattern we used in `getErrorResponse`, which is a pain to maintain (as seen by KAFKA-9200). We already check that requests have a valid version range in the `AbstractRequest` constructor.
Reviewers: Andrew Choi <andrewchoi5@users.noreply.github.com>, Ismael Juma <ismael@juma.me.uk>
ListOffsetResponse getErrorResponse is missing a a case for version 5, introduced
by 152292994e4 and released in 2.3.0.
```
java.lang.IllegalArgumentException: Version 5 is not valid. Valid versions for ListOffsetRequest are 0 to 5
at org.apache.kafka.common.requests.ListOffsetRequest.getErrorResponse(ListOffsetRequest.java:282)
at kafka.server.KafkaApis.sendErrorOrCloseConnection(KafkaApis.scala:3062)
at kafka.server.KafkaApis.sendErrorResponseMaybeThrottle(KafkaApis.scala:3045)
at kafka.server.KafkaApis.handleError(KafkaApis.scala:3027)
at kafka.server.KafkaApis.handle(KafkaApis.scala:209)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:78)
at java.lang.Thread.run(Thread.java:748)
```
Reviewers: Ismael Juma <ismael@juma.me.uk>
The current Utils::readFileAsString method creates a FileChannel and
memory maps file and copies its content to a String and returns it. But
that means that we need to know the size of the file in advance. This
precludes us from reading files whose size is not known in advance, i.e.
any file opened with flag S_IFIFO.
This change updates the method to use stream to read the content of the file.
It has couple of practical advantages:
Allows bash process substitution to pass in strings as file. So we can
say `./bin/kafka-reassign-partitions.sh --reassignment-json-file <(echo
"reassignment json")
When adding systest for commands that take file, we don't have to
create real physical file. Instead we can just dump the content of the
file on the command line.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Explicitly mention that max.request.size validates uncompressed record sizes and max.message.bytes validates compressed record sizes.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Sometimes to be backwards compatible regarding metrics the simplest
solution is to create an empty sensor. Recording an empty sensor on
the hot path may negatively impact performance. With hasMetrics()
recordings of empty sensors on the hot path can be avoided without
being to invasive.
Reviewers: Bill Bejeck <bbejeck@gmail.com>
This patch implements the broker-side changes for KIP-360. It adds two new fields to InitProducerId: lastEpoch and producerId. Passing these values allows the TransactionCoordinator to safely bump a producer's epoch after some failures (such as UNKNOWN_PRODUCER_ID and INVALID_PRODUCER_ID_MAPPING). When a producer calls InitProducerId after a failure, the coordinator first checks the producer ID from the request to make sure no other producer has been started using the same transactional ID. If it is safe to continue, the coordinator checks the epoch from the request; if it matches the existing epoch, the epoch is bumped and the producer can safely continue. If it matches the previous epoch, the the current epoch is returned without bumping. Otherwise, the producer is fenced.
Reviewers: Boyang Chen <boyang@confluent.io>, Jason Gustafson <jason@confluent.io>
The `Epoch` field description was copy of the `SessionId` field. This
change updates it to describe `Epoch` instead.
No code change, only description changes. Code compiles.
Reviewers: Jason Gustafson <jason@confluent.io>
Author: Colin P. Mccabe <cmccabe@confluent.io>
Reviewers: Vikas Singh <vikas@confluent.io>, Jason Gustafson <jason@confluent.io>
Closes#7477 from cmccabe/KAFKA-8984
Within KafkaConsumer.poll, we have an optimization to try to send the next fetch request before returning the data in order to pipelining the fetch requests; however, this pollNoWakeup should NOT throw any exceptions, since at this point the fetch position has been updated. If an exception is thrown and the callers decide to capture and continue, those records would never be returned again, causing data loss.
Also fix the flaky test itself.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Jason Gustafson <jason@confluent.io>
This patch fixes the test utility `testAllMessageRoundTripsFromVersion` in `MessageTest` which was unintentionally excluding the highest version.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Otherwise the join-group would not be resend and we'd just fall into the endless loop.
Reviewers: Jason Gustafson <jason@confluent.io>, Boyang Chen <boyang@confluent.io>, A. Sophie Blee-Goldman <sophie@confluent.io>
This patch removes the NewPartitionReassignment#of() method in favor of a simple constructor. Said method was confusing due to breaking two conventions - always returning a non-empty Optional and thus not being used as a static factory method.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Colin P. McCabe <cmccabe@apache.org>
1. Avoid a buffer allocation and a buffer copy per file read.
2. Ensure we flush `netWriteBuffer` successfully before reading from
disk to avoid wasted disk reads.
3. 32k reads instead of 8k reads to reduce the number of disk reads
(improves efficiency for magnetic drives and reduces the number of
system calls).
4. Update SslTransportLayer.write(ByteBuffer) to loop until the socket
buffer is full or the src buffer has no remaining bytes.
5. Renamed `MappedByteBuffers` to `ByteBufferUnmapper` since it's also
applicable for direct byte buffers.
6. Skip empty `RecordsSend`
7. Some minor clean-ups for readability.
I ran a simple consumer perf benchmark on a 6 partition topic (large
enough not to fit into page cache) starting from the beginning of the
log with TLS enabled on my 6 core MacBook Pro as a sanity check.
This laptop has fast SSDs so it benefits less from the larger reads
than the case where magnetic disks are used. Consumer throughput
was ~260 MB/s before the changes and ~300 MB/s after
(~15% improvement).
Credit to @junrao for pointing out that this code could be more efficient.
Reviewers: Jun Rao <junrao@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
If there are pending bytes in the transport layer, we may
complete a send even if no bytes were recorded as written.
We assume bytes are written when they are in the netWriteBuffer,
but we only consider the send as completed when it's in
the socket channel buffer.
This fixes a regression introduced via 0971f66ff546. The impact is
that we would sometimes throw the following exception in
`MultiRecordsSend.writeTo`:
```java
if (completed())
throw new KafkaException("This operation cannot be invoked on a complete request.");
```
Added unit test verifying the bug fix. While in the area, I simplified one of the
`SslSelectorTest` methods.
Reviewers: Jun Rao <junrao@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
Currently when we identify version probing we return early from onAssignment and never get to updating the TaskManager and general state with the new assignment. Since we do actually give out "real" assignments even during version probing, a StreamThread should take real ownership of its tasks/partitions including cleaning them up in onPartitionsRevoked which gets invoked when we call onLeavePrepare as part of triggering the follow-up rebalance.
Every member will always get an assignment encoded with the lowest common version, so there should be no problem decoding a VP assignment. We should just allow onAssignment to proceed as usual so that the TaskManager is in a consistent state, and knows what all its tasks/partitions are when the first rebalance completes and the next one is triggered.
Reviewers: Boyang Chen <boyang@confluent.io>, Matthias J. Sax <mjsax@apache.org>, Guozhang Wang <wangguoz@gmail.com>
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>
`log` in KafkaConsumer does not get initialized if an invalid value for group.intance.id is given during consumer construction. In this case we should skip the catch block's close procedure since no internal objects have been initialized yet.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Inside onLeavePrepare we would look into the assignment and try to revoke the owned tasks and notify users via RebalanceListener#onPartitionsRevoked, and then clear the assignment.
However, the subscription's assignment is already cleared in this.subscriptions.unsubscribe(); which means user's rebalance listener would never be triggered. In other words, from consumer client's pov nothing is owned after unsubscribe, but from the user caller's pov the partitions are not revoked yet. For callers like Kafka Streams which rely on the rebalance listener to maintain their internal state, this leads to inconsistent state management and failure cases.
Before KIP-429 this issue is hidden away since every time the consumer re-joins the group later, it would still revoke everything anyways regardless of the passed-in parameters of the rebalance listener; with KIP-429 this is easier to reproduce now.
Our fixes are following:
• Inside unsubscribe, first do onLeavePrepare / maybeLeaveGroup and then subscription.unsubscribe. This we we are guaranteed that the streams' tasks are all closed as revoked by then.
• [Optimization] If the generation is reset due to fatal error from join / hb response etc, then we know that all partitions are lost, and we should not trigger onPartitionRevoked, but instead just onPartitionsLost inside onLeavePrepare. This is because we don't want to commit for lost tracks during rebalance which is doomed to fail as we don't have any generation info.
Reviewers: Matthias J. Sax <matthias@confluent.io>, A. Sophie Blee-Goldman <sophie@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>
This PR fixes the inconsistency involved in the `removeMembersFromGroup` admin API calls:
1. Fail the `all()` request when there is sub level error (either partition or member)
2. Change getMembers() to members()
3. Hide the actual Errors from user
4. Do not expose generated MemberIdentity type
5. Use more consistent naming for Options and Result types
Reviewers: Guozhang Wang <wangguoz@gmail.com>, David Jacot <djacot@confluent.io>, Jason Gustafson <jason@confluent.io>
Re-implement NewPartitionReassignment#of. It now takes a list rather than a variable-length list of arguments.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Vikas Singh <vikas@confluent.io>
This is not guaranteed to actually fix queryOnRebalance, since the
failure could never be reproduced locally. I did not bump timeouts
because it looks like that has been done in the past for this test
without success. Instead this change makes the following improvements:
It waits for the application to be in a RUNNING state before
proceeding with the test.
It waits for the remaining instance to return to RUNNING state
within a timeout after rebalance. I observed once that we were able to
do the KV queries but the instance was still in REBALANCING, so this
should reduce some opportunity for flakiness.
The meat of this change: we now iterate over all keys in one shot
(vs. one at a time with a timeout) and collect various failures, all of
which are reported at the end. This should help us to narrow down the
cause of flakiness if it shows up again.
Reviewers: 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>
Currently we only record completed sends and receives in the selector metrics. If there is a disconnect in the middle of the respective operation, then it is not counted. The metrics will be more accurate if we take into account partial sends and receives.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com
This patch adds a `toString()` implementation to `PartitionReassignment`. It also makes the `ListPartitionReassignmentsResult` constructor use default access, which is the standard for the admin client *Result classes.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Reviewers: Jason Gustafson <jason@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>, Vikas Singh <vikas@confluent.io>, Colin P. McCabe <cmccabe@apache.org>