Fix infinite loop in assignor when trying to resolve the number of partitions in a topology with a windowed FKJ. Also adds a check to this loop to break out and fail the application if we detect that we are/will be stuck in an infinite loop
Reviewers: Matthias Sax <matthias@confluent.io>
The map passed as an argument remains read-only and therefore can be initialized using Collections#emptyMap instead of being passed a new Map instance.
Reviewers: Konstantine Karantasis <k.karantasis@gmail.com>
In the JavaDoc, the implemented interface was described inaccurately.
Also, the ordered list was formatted as plain text, not as html "ol".
Reviewers: Boyang Chen <boyang@confluent.io>
This patch creates a new `SendBuilder` class which allows us to avoid copying "zero copy" types when transmitting an api message over the network. This generalizes the pattern that was previously used only for `FetchResponse`. Initially we only apply this optimization to the `Envelope` types and `FetchResponse`, but in the future, it can be the default implementation for `toSend`.
The patch also contains a few minor cleanups such as moving envelope parsing logic into `RequestContext`.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Rewrite DSL topology to allow reading a topic or pattern multiple times
Reviewers: Bruno Cadonna <cadonna@confluent.io>, Leah Thomas <lthomas@confluent.io>
Demote "committing offsets" log message to DEBUG and promote/add summarizing INFO level logs in the main StreamThread loop
Reviewers: Boyang Chen <boyang@confluent.io>, Walker Carlson <wcarlson@confluent.io>, John Roesler <john@confluent.io>
Change permissions on the folders for the state store so they're no readable or writable by "others", but still accessible by owner and group members.
Reviewers: Bruno Cadonna <bruno@confluent.io>, Walker Carlson <wcarlson@confluent.io>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
This newly added system test is to verify that with the fix in #9270 , the member.id update caused by static member rejoin would be persisted correctly.
Reviewers: Boyang Chen <boyang@confluent.io>
Increase the amount of time available to the `test_verifiable_producer` test to login and get the process name for the verifiable producer from 5 seconds to 10 seconds.
We were seeing some test failures due to the assertion failing because the verifiable producer would complete before we could login, list the processes, and parse out the producer version. Previously, we were giving this operation 5 seconds to run, this PR bumps it up to 10 seconds.
I verified locally that this does not flake, but even at 5 seconds I wasn't seeing any flakes. Ultimately we should find a better strategy than racing to query the producer process (as outlined in the existing comments).
Reviewers: Jason Gustafson <jason@confluent.io>
Zstd-jni 1.4.5-6 allocates large internal buffers inside of ZstdInputStream and ZstdOutputStream. This caused a lot of allocation and GC activity when creating and closing the streams. It also does not buffer the reads or writes. This causes inefficiency when DefaultRecord.writeTo() does a series of small single bytes reads using various ByteUtils methods. The JNI is more efficient if the writes of uncompressed data were flushed in large pieces rather than for each byte. This is due to the the expense of context switching between the Java code and the native code. This is also the case when reading as well. Per luben/zstd-jni#141 the maintainer of zstd-jni and I agreed to not buffer reads and writes in favor of having the caller do that, so here we are updating the caller.
In this patch, I upgraded to the most recent zstd-jni version with the buffer reuse built-in. This was done in luben/zstd-jni#143 and luben/zstd-jni#146 Since we decided not to add additional buffering of input/output with zstd-jni, I added the BufferedInputStream and BufferedOutputStream to CompressionType.ZSTD just like we currently do for CompressionType.GZIP which also is inefficient for single byte reads and writes. I used the same buffer sizes as that existing implementation.
NOTE: if so desired we could pass a wrapped BufferSupplier into the Zstd*Stream classes to have Kafka decide how the buffer recycling occurs. This functionality was added in the latter PR linked above. I am holding off on this since based on jmh benchmarking the performance gains were not clear and personally I don't know if it worth the complexity of trying to hack around the reflection at this point in time. The zstd-jni uses a very similar default recycler as snappy does currently which seems to provide decent efficiency. While this PR fixes the defect, I feel that using BufferSupplier in both zstd-jni and snappy is outside of the scope of this bugfix and should be considered a separate improvement. I would prefer this change get merged in on its own since the performance gains here are very significant relative to the more incremental and minor optimizations which could be achieved by doing that separate work.
There are some noticeable improvements in the JMH benchmarks (excerpt):
BEFORE:
Benchmark (bufferSupplierStr) (bytes) (compressionType) (maxBatchSize) (messageSize) (messageVersion) Mode Cnt Score Error Units
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed CREATE RANDOM ZSTD 200 1000 2 thrpt 15 27743.260 ± 673.869 ops/s
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.alloc.rate CREATE RANDOM ZSTD 200 1000 2 thrpt 15 3399.966 ± 82.608 MB/sec
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.alloc.rate.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 134968.010 ± 0.012 B/op
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.churn.G1_Eden_Space CREATE RANDOM ZSTD 200 1000 2 thrpt 15 3850.985 ± 84.476 MB/sec
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.churn.G1_Eden_Space.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 152881.128 ± 942.189 B/op
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.churn.G1_Survivor_Space CREATE RANDOM ZSTD 200 1000 2 thrpt 15 174.241 ± 3.486 MB/sec
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.churn.G1_Survivor_Space.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 6917.758 ± 82.522 B/op
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.count CREATE RANDOM ZSTD 200 1000 2 thrpt 15 1689.000 counts
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.time CREATE RANDOM ZSTD 200 1000 2 thrpt 15 82621.000 ms
JMH benchmarks done
Benchmark (bufferSupplierStr) (bytes) (compressionType) (maxBatchSize) (messageSize) (messageVersion) Mode Cnt Score Error Units
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage CREATE RANDOM ZSTD 200 1000 2 thrpt 15 24095.711 ± 895.866 ops/s
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.alloc.rate CREATE RANDOM ZSTD 200 1000 2 thrpt 15 2932.289 ± 109.465 MB/sec
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.alloc.rate.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 134032.012 ± 0.013 B/op
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.churn.G1_Eden_Space CREATE RANDOM ZSTD 200 1000 2 thrpt 15 3282.912 ± 115.042 MB/sec
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.churn.G1_Eden_Space.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 150073.914 ± 1342.235 B/op
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.churn.G1_Survivor_Space CREATE RANDOM ZSTD 200 1000 2 thrpt 15 149.697 ± 5.786 MB/sec
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.churn.G1_Survivor_Space.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 6842.462 ± 64.515 B/op
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.count CREATE RANDOM ZSTD 200 1000 2 thrpt 15 1449.000 counts
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.time CREATE RANDOM ZSTD 200 1000 2 thrpt 15 82518.000 ms
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize CREATE RANDOM ZSTD 200 1000 2 thrpt 15 1449.060 ± 230.498 ops/s
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize:·gc.alloc.rate CREATE RANDOM ZSTD 200 1000 2 thrpt 15 198.051 ± 31.532 MB/sec
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize:·gc.alloc.rate.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 150502.519 ± 0.186 B/op
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize:·gc.churn.G1_Eden_Space CREATE RANDOM ZSTD 200 1000 2 thrpt 15 200.064 ± 31.879 MB/sec
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize:·gc.churn.G1_Eden_Space.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 152569.341 ± 13826.686 B/op
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize:·gc.count CREATE RANDOM ZSTD 200 1000 2 thrpt 15 91.000 counts
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize:·gc.time CREATE RANDOM ZSTD 200 1000 2 thrpt 15 75869.000 ms
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize CREATE RANDOM ZSTD 200 1000 2 thrpt 15 2609.660 ± 1145.160 ops/s
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.alloc.rate CREATE RANDOM ZSTD 200 1000 2 thrpt 15 815.441 ± 357.818 MB/sec
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.alloc.rate.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 344309.097 ± 0.238 B/op
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.churn.G1_Eden_Space CREATE RANDOM ZSTD 200 1000 2 thrpt 15 808.952 ± 354.975 MB/sec
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.churn.G1_Eden_Space.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 345712.061 ± 51434.034 B/op
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.churn.G1_Old_Gen CREATE RANDOM ZSTD 200 1000 2 thrpt 15 0.019 ± 0.042 MB/sec
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.churn.G1_Old_Gen.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 18.615 ± 42.045 B/op
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.churn.G1_Survivor_Space CREATE RANDOM ZSTD 200 1000 2 thrpt 15 24.132 ± 12.254 MB/sec
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.churn.G1_Survivor_Space.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 13540.960 ± 14649.192 B/op
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.count CREATE RANDOM ZSTD 200 1000 2 thrpt 15 148.000 counts
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.time CREATE RANDOM ZSTD 200 1000 2 thrpt 15 23848.000 ms
JMH benchmarks done
AFTER
Benchmark (bufferSupplierStr) (bytes) (compressionType) (maxBatchSize) (messageSize) (messageVersion) Mode Cnt Score Error Units
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed CREATE RANDOM ZSTD 200 1000 2 thrpt 15 147792.454 ± 2721.318 ops/s
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.alloc.rate CREATE RANDOM ZSTD 200 1000 2 thrpt 15 2708.481 ± 50.012 MB/sec
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.alloc.rate.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 20184.002 ± 0.002 B/op
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.churn.G1_Eden_Space CREATE RANDOM ZSTD 200 1000 2 thrpt 15 2732.667 ± 59.258 MB/sec
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.churn.G1_Eden_Space.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 20363.460 ± 120.585 B/op
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.churn.G1_Old_Gen CREATE RANDOM ZSTD 200 1000 2 thrpt 15 0.042 ± 0.033 MB/sec
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.churn.G1_Old_Gen.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 0.316 ± 0.249 B/op
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.count CREATE RANDOM ZSTD 200 1000 2 thrpt 15 833.000 counts
CompressedRecordBatchValidationBenchmark.measureValidateMessagesAndAssignOffsetsCompressed:·gc.time CREATE RANDOM ZSTD 200 1000 2 thrpt 15 8390.000 ms
JMH benchmarks done
Benchmark (bufferSupplierStr) (bytes) (compressionType) (maxBatchSize) (messageSize) (messageVersion) Mode Cnt Score Error Units
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage CREATE RANDOM ZSTD 200 1000 2 thrpt 15 166786.092 ± 3285.702 ops/s
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.alloc.rate CREATE RANDOM ZSTD 200 1000 2 thrpt 15 2926.914 ± 57.464 MB/sec
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.alloc.rate.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 19328.002 ± 0.002 B/op
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.churn.G1_Eden_Space CREATE RANDOM ZSTD 200 1000 2 thrpt 15 2938.541 ± 66.850 MB/sec
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.churn.G1_Eden_Space.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 19404.357 ± 177.485 B/op
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.churn.G1_Old_Gen CREATE RANDOM ZSTD 200 1000 2 thrpt 15 0.516 ± 0.100 MB/sec
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.churn.G1_Old_Gen.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 3.409 ± 0.657 B/op
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.churn.G1_Survivor_Space CREATE RANDOM ZSTD 200 1000 2 thrpt 15 0.032 ± 0.131 MB/sec
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.churn.G1_Survivor_Space.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 0.207 ± 0.858 B/op
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.count CREATE RANDOM ZSTD 200 1000 2 thrpt 15 834.000 counts
RecordBatchIterationBenchmark.measureIteratorForBatchWithSingleMessage:·gc.time CREATE RANDOM ZSTD 200 1000 2 thrpt 15 9370.000 ms
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize CREATE RANDOM ZSTD 200 1000 2 thrpt 15 15988.116 ± 137.427 ops/s
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize:·gc.alloc.rate CREATE RANDOM ZSTD 200 1000 2 thrpt 15 448.636 ± 3.851 MB/sec
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize:·gc.alloc.rate.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 30907.698 ± 0.020 B/op
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize:·gc.churn.G1_Eden_Space CREATE RANDOM ZSTD 200 1000 2 thrpt 15 450.905 ± 5.587 MB/sec
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize:·gc.churn.G1_Eden_Space.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 31064.113 ± 291.190 B/op
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize:·gc.churn.G1_Old_Gen CREATE RANDOM ZSTD 200 1000 2 thrpt 15 0.043 ± 0.007 MB/sec
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize:·gc.churn.G1_Old_Gen.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 2.931 ± 0.493 B/op
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize:·gc.count CREATE RANDOM ZSTD 200 1000 2 thrpt 15 790.000 counts
RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize:·gc.time CREATE RANDOM ZSTD 200 1000 2 thrpt 15 999.000 ms
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize CREATE RANDOM ZSTD 200 1000 2 thrpt 15 11345.169 ± 206.528 ops/s
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.alloc.rate CREATE RANDOM ZSTD 200 1000 2 thrpt 15 2314.800 ± 42.094 MB/sec
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.alloc.rate.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 224714.266 ± 0.028 B/op
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.churn.G1_Eden_Space CREATE RANDOM ZSTD 200 1000 2 thrpt 15 2320.213 ± 45.521 MB/sec
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.churn.G1_Eden_Space.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 225235.965 ± 803.309 B/op
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.churn.G1_Old_Gen CREATE RANDOM ZSTD 200 1000 2 thrpt 15 0.026 ± 0.005 MB/sec
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.churn.G1_Old_Gen.norm CREATE RANDOM ZSTD 200 1000 2 thrpt 15 2.551 ± 0.455 B/op
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.count CREATE RANDOM ZSTD 200 1000 2 thrpt 15 994.000 counts
RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize:·gc.time CREATE RANDOM ZSTD 200 1000 2 thrpt 15 1189.000 ms
JMH benchmarks done
Reviewers: Ismael Juma <ismael@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
This PR follows up 0814e4f to migrate the remaining RPCs which need forwarding, including:
CreateAcls/DeleteAcls/CreateDelegationToken/RenewDelegationToken/ExpireDelegationToken/AlterPartitionReassignment/CreatePartition/DeleteTopics/UpdateFeatures/Scram
Reviewers: David Arthur <mumrah@gmail.com>
Previously the root logger level was used, ignoring intervening loggers with
different levels.
This was initially applied via fda6701837 and reverted via 75e5358525
due to the fact that the previous behavior was specified in KIP-412. It
has been been reapplied since KIP-676 has been approved.
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Lee Dongjin <dongjin@apache.org>, Ismael Juma <ismael@juma.me.uk>
When initializing the raft state machine after shutting down as a leader, we were previously entering the "unattached" state, which means we have no leader and no voted candidate. This was a bug because it allowed a reinitialized leader to cast a vote for a candidate in the same epoch that it was already the leader of. This patch fixes the problem by introducing a new "resigned" state which allows us to retain the leader state so that we cannot change our vote and we will not accept additional appends.
This patch also revamps the shutdown logic to make use of the new "resigned" state. Previously we had a separate path in `KafkaRaftClient.poll` for the shutdown logic which resulted in some duplication. Instead now we incorporate shutdown behavior into each state's respective logic.
Finally, this patch changes the shutdown logic so that `EndQuorumEpoch` is only sent by resigning leaders. Previously we allowed this request to be sent by candidates as well.
Reviewers: dengziming <dengziming1993@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
Jetty 9.4.32 and before are affected by CVE-2020-27216. This vulnerability is fixed in Jetty 9.4.33, please see the jetty project security advisory for details: https://github.com/eclipse/jetty.project/security/advisories/GHSA-g3wg-6mcf-8jj6#advisory-comment-63053
Unit tests and integration tests pass locally after the upgrade.
Author: Nitesh Mor <nmor@confluent.io>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#9556 from niteshmor/trunk
`config.interBrokerListenerName` and `config.listeners` were called several times per connection accept. As these calls are expensive, it is preferable to rely on cached values.
Reviewers: Anna Povzner <anna@confluent.io>, David Jacot <djacot@confluent.io>
This is a follow-up to initial KIP-584 development. In this PR, I've switched the FeatureZNodeStatus enum to be a sealed trait. In Scala, we prefer sealed traits over Enumeration since the former gives you exhaustiveness checking. With Scala enumeration, you don't get a warning if you add a new value that is not handled in a given pattern match.
Reviewers: Jun Rao <junrao@gmail.com>
Change the StreamThreads to be held in a List so that we
can dynamically change the number of threads easily.
Reviewers: Bruno Cadonna <bruno@confluent.io>, John Roesler <vvcephei@apache.org>
Fixed the param doc in FinalizedFeatureChangeListener.initOrThrow method. The parameter waitOnceForCacheUpdateMs is expected to be > 0, but the doc was incorrect.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This PR adds support for forwarding of the following RPCs:
AlterConfigs
IncrementalAlterConfigs
AlterClientQuotas
CreateTopics
Co-authored-by: Jason Gustafson <jason@confluent.io>
Reviewers: Jason Gustafson <jason@confluent.io>
Schedules client reinitialization if Zookeeper auth failure is encountered. This allows for reconnections when transient network errors are encountered during connection establishment. The Zookeeper client doesn't expose details of the auth failure so we can't determine whether an error is retriable or not, so all auth failures are retried.
Co-authored-by: Rajini Sivaram <rajinisivaram@googlemail.com>
Reviewers: Jun Rao <junrao@gmail.com>
During the AK website upgrade, changes made to kafka-site weren't migrated back to kafka-docs.
This PR is an attempt at porting the streams changes to kafka/docs
For the most part, the bulk of the changes in the PR are cosmetic.
For testing:
I reviewed the PR diffs
Rendered the changes locally
Reviewers: John Roesler <john@confluent.io>
StandaloneHerder and DistributedHerder have identical implementations of connectorConfig (apart from one line of logging). This commit moves the common implementation of connectorConfig to AbstractHerder.
Reviewers: Konstantine Karantasis <k.karantasis@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
During the AK website upgrade, changes made to kafka-site weren't migrated back to kafka-docs.
This PR is an initial attempt at porting the changes to kafka/docs, but it does not include the streams changes. Those will come in a separate PR.
For the most part, the bulk of the changes in the PR are cosmetic. Only the introduction.html has substantial changes, but it's a direct port from the live documentation.
For testing:
I reviewed the PR diffs
Rendered the changes locally
Reviewers: Matthias J. Sax <mjsax@apache.org>
In this PR, I have eliminated the facility in Admin#describeFeatures API and it's implementation to be able to optionally send a describeFeatures request to the controller. This feature was not seen to be particularly useful, and besides it also poses some hindrance to post KIP-500 world where no client would be able to access the controller directly.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
In #9418, we add a listener to the `RaftClient` interface. In that patch, we used it only to send commit notifications for writes from the leader. In this PR, we extend the `handleCommit` API to accept all committed data and we remove the pull-based `read` API. Additionally, we add two new callbacks to the listener interface in order to notify the state machine when the raft client has claimed or resigned leadership.
Finally, this patch allows the `RaftClient` to support multiple listeners. This is necessary for KIP-500 because we will have one listener for the controller role and one for the broker role.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Boyang Chen <boyang@confluent.io>
LogManager writes a clean shutdown file when the broker shuts down. The
presence of this file indicates that the broker had a clean shutdown and
log recovery is not needed upon the next boot up.
Earlier, LogManager would check for this file at the start of log loading workflow,
and delete it after the log has been loaded. If the broker were to crash
while loading logs, the file would not be deleted and mislead LogManager when it
tries to load logs upon next boot up. Hence, a crash during log loading
will not be considered a hard reset of broker.
As part of this fix, we delete the clean shutdown file as soon as we
look it up, at the start of log loading workflow. Thereafter, we maintain a boolean
flag to indicate if broker underwent clean shutdown or not. So, if the
broker were to crash while logs are being loaded, LogManager will be
able to detect this as a hard reset.
Reviewers: Jun Rao <junrao@gmail.com>
Couple of failures observed after KAFKA-9627: Replace ListOffset request/response with automated protocol (https://github.com/apache/kafka/pull/8295)
1. Latest consumer fails to consume from 0.10.0.1 brokers. Below system tests are failing
kafkatest.tests.client.client_compatibility_features_test.ClientCompatibilityFeaturesTest
kafkatest.tests.client.client_compatibility_produce_consume_test.ClientCompatibilityProduceConsumeTest
Solution: Current default value for MaxNumOffsets is 0. because to this brokers are not returning offsets for v0 request. Set default value for MaxNumOffsets field to 1. This is similar to previous [approach]
(https://github.com/apache/kafka/blob/2.6/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java#L204)
2. In some scenarios, latest consumer fails with below error when connecting to a Kafka cluster which consists of newer and older (<=2.0) Kafka brokers
`org.apache.kafka.common.errors.UnsupportedVersionException: Attempted to write a non-default currentLeaderEpoch at version 3`
Solution: After #8295, consumer can set non-default CurrentLeaderEpoch value for v3 and below requests. One solution is to make CurrentLeaderEpoch ignorable.
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: David Jacot <djacot@confluent.io>
Closes#9540 from omkreddy/fix-listoffsets
KIP-431 (#9099) changed the format of console consumer output to `Partition:$PARTITION\t$VALUE` whereas previously the output format was `$VALUE\t$PARTITION`. This PR updates the message verifier to accommodate the updated console consumer output format.
Read offsets directly from the checkpoint file if a task is uninitialized or closed
Reviewers: Bruno Cadonna <bruno@confluent.io>, John Roesler <vvcephei@apache.org>
Delete the existing checkpoint file if told to write empty offsets map to ensure that corrupted offsets are not re-initialized from
Reviewers: Bruno Cadonna <bruno@confluent.io>, Guozhang Wang <guozhang@apache.org>