This is required for ACLs where SSL principals contain
special characters (e.g. comma) that are escaped using
backslash. The strings need to be quoted for JSON to
ensure that the JSON stored in ZK is valid.
Also converted `SslEndToEndAuthorizationTest` to use a
principal with special characters to ensure that this
path is tested.
Author: Rajini Sivaram <rajinisivaram@googlemail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4303 from rajinisivaram/KAFKA-6319
- Rename `delete()` to `deleteIfExists()` in `LogSegment`, `AbstractIndex`
and `TxnIndex`. Throw exception in case of IO errors for more informative
errors and to make it less likely that errors are ignored, `boolean` is used
for the case where the file does not exist (like `Files.deleteIfExists()`).
- Fix an instance of delete while open (should fix KAFKA-6322 and
KAFKA-6075).
- `LogSegment.deleteIfExists` no longer throws an exception if any of
the files it tries to delete does not exist (fixes KAFKA-6194).
- Remove unnecessary `FileChannel.force(true)` when deleting file.
- Introduce `LogSegment.open()` and use it to improve encapsulation
and reduce duplication.
- Expand functionality of `LogSegment.onBecomeInactiveSegment()`
to reduce duplication and improve encapsulation.
- Use `AbstractIndex.deleteIfExists()` instead of deleting files manually.
- Improve logging when deleting swap files.
- Use CorruptIndexException instead of IllegalArgumentException.
- Simplify `LogCleaner.cleanSegments()` to reduce duplication and
improve encapsulation.
- A few other clean-ups in Log, LogSegment, etc.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jun Rao <junrao@gmail.com>, Ted Yu <yuzhihong@gmail.com>
Closes#4040 from ijuma/kafka-5829-follow-up
- set auto.offset.reste to "none" for restore and global consumer
- handle InvalidOffsetException for restore and global consumer
- add corresponding tests
- some minor cleanup
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Damian Guy <damian.guy@gmail.com, Bill Bejeck <bill@confluent.io>, GuozhangWang <wangguoz@gmail.com>
Closes#4215 from mjsax/kafka-6121-restore-global-consumer-handle-reset
The NetworkClient internally ApiVersion requests to each broker following connection establishment. If this request happens to fail (perhaps due to an incompatible broker), the NetworkClient includes the response in the result of poll(). Applications will generally not be expecting this response which may lead to failed assertions (or in the case of AdminClient, an obscure log message).
I've added test cases which await the ApiVersion request sent by NetworkClient to be in-flight, and then disconnect the connection and verify that the response is not included from poll().
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4280 from hachikuji/KAFKA-6289
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Bill Bejeck <bbejeck@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
Closes#4242 from mjsax/kafka-4857-admit-client
- Ensure that `partitionsBeingReassigned` is fully populated before
`removePartitionFromReassignedPartitions` is invoked. This is
necessary to avoid premature deletion of the `reassign_partitions`
znode.
- Modify and add tests to verify the fixes.
- Add documentation.
- Use `info` log message if assignedReplicas == newReplicas and
remove control flow based on exceptions.
- General logging improvements.
- Simplify `initializePartitionAssignment` by relying on logic already
present in `maybeTriggerPartitionReassignment`.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jun Rao <junrao@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
Closes#4283 from ijuma/kafka-6193-flaky-shouldPerformMultipleReassignmentOperationsOverVariousTopics
From 0.11 to 1.0, we moved `DescribeClusterOptions timeoutMs(Integer timeoutMs)` from
DescribeClusterOptions to AbstractOptions (similarly for other Options classes). This can
cause code compiled against 0.11.0.x to fail when it is executed with 1.0 kafka-clients jar.
This patch adds back these methods to restore binary compatibility with 0.11.
Author: Dong Lin <lindong28@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4257 from lindong28/KAFKA-6174
1. Add the repartition topics information into ProcessorTopology: personally I do not like leaking this information into the topology but it seems not other simple way around.
2. StreamTask: added one more function to expose the consumed offsets from repartition topics only.
3. TaskManager: use the AdminClient to send the gathered offsets to delete only if a) previous call has completed and client intentionally ignore-and-log any errors, or b) no requests have ever called before.
NOTE that this code depends on the assumption that purge is only called right after the commit has succeeded, hence we presume all consumed offsets are committed.
4. MINOR: Added a few more constructor for ProcessorTopology for cleaner unit tests.
5. MINOR: Extracted MockStateStore out of the deprecated class.
6. MINOR: Made a pass over some unit test classes for clean ups.
Author: Guozhang Wang <wangguoz@gmail.com>
Reviewers: Matthias J. Sax <matthias@confluent.io>, Damian Guy <damian.guy@gmail.com>
Closes#4270 from guozhangwang/K6150-purge-repartition-topics
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Closes#4284 from mjsax/minor-improve-eos-docs
When a socket is closed, we must remove corresponding selection keys from internal collections. This fixes an NPE which is caused by attempting to access the selection key's attached channel after it had been cleared after disconnecting.
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4276 from hachikuji/KAFKA-6260
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Apurva Mehta <apurva@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
Closes#3912 from mjsax/kafka-5936-producer-close
Only expect responseAsString to be set if request logging is
enabled _and_ responseSend is defined.
Also fixed a couple of issues that would manifest themselves
if trace logging is enabled:
- `MemoryRecords.toString` should not throw exception if data is corrupted
- Generate `responseString` correctly if unsupported api versions request is
received.
Unit tests were added for every issue fixed. Also changed
SocketServerTest to run with trace logging enabled as
request logging breakage has been a common issue.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4250 from ijuma/fix-issues-when-trace-logging-is-enabled
1. Add The AdminClient into Kafka Streams, which is shared among all the threads.
2. Add ADMIN_PREFIX to StreamsConfig.
3. Also made a few tweaks on the metrics of the AdminClient, which is slightly different from the StreamsKafkaClient (note these changes will not be reflected in this PR but only take place when we eventually replace StreamsKafkaClient):
3.1. "clientId" tag will be set as "clientId-admin": in StreamsKafkaClient it is whatever user sets, and hence could even be null.
3.2. "groupPrefix" will be set as "admin-client": in StreamsKafkaClient it will be "kafka-client".
So the metrics from `StreamsKafkaClient` to `AdminClient` would be changed from
`kafka.admin.client:type=kafka-client-metrics,client-id=`
to
`kafka.admin.client:type=admin-client-metrics,client-id=myApp-UUID-admin`
Author: Guozhang Wang <wangguoz@gmail.com>
Reviewers: Damian Guy <damian.guy@gmail.com>, Matthias J. Sax <matthias@confluent.io>, Ted Yu <yuzhihong@gmail.com>
Closes#4211 from guozhangwang/K6170-admin-client
Implements KIP-224:
- adding new StreamsConfig `retires`
- uses `retires` and `retry.backoff.ms` to handle TimeoutException in GlobalStateManager
- adds two new tests to trigger TimeoutException in global consumer
- some minor code cleanup to reduce number of parameters we need to pass around
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Damian Guy <damian.guy@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
Closes#4206 from mjsax/kafka-6122-global-consumer-timeout-exception
This is the PR related to the [KIP-204](https://cwiki.apache.org/confluence/display/KAFKA/KIP-204+%3A+Adding+records+deletion+operation+to+the+new+Admin+Client+API) in order to add the `deleteRecords` operation to the new Admin Client (it's already available in the "legacy" one).
Other than that, unit test and integration tests are added as well (such integration tests come from the "legacy" integration tests in order to test the new addition in the same way as the "legacy" one).
Author: Paolo Patierno <ppatierno@live.com>
Reviewers: Colin P. Mccabe <cmccabe@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Closes#4132 from ppatierno/kafka-5925
Added unit test for ApiVersion and testApiVersions from
Scala to Java.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4220 from ijuma/kafka-6210-iae-if-1.0.0-inter-broker-protocol-version
When reviewing https://github.com/apache/kafka/pull/4132, I felt that NOT_ENOUGH_REPLICAS should never happen actually. Hence proposing to remove it from the listed error code as well in the broker-side capture clause.
Testing added in 4132 should have been sufficient.
Author: Guozhang Wang <wangguoz@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Dong Lin <lindong28@gmail.com>
Closes#4208 from guozhangwang/KMinor-delete-records-error-code
This memory leak could eventually lead to an OutOfMemoryError. This
was particularly likely in case of down conversions as the leaked
channels would hold on to the record batch (which is only loaded
into the heap in case of down conversions).
Author: Rajini Sivaram <rajinisivaram@googlemail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4193 from rajinisivaram/KAFKA-6185-oom
Windows directory paths often contain colons which are not allowed in
yammer metrics. Metric tag values with special characters must be
quoted.
Author: huxihx <huxi_2b@hotmail.com>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4173 from huxihx/KAFKA-6156
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
It was committed inadvertently.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4172 from ijuma/remove-out-folder
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Bill Bejeck <bill@confluent.io>, Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Closes#4128 from mjsax/minor-cleanup
minor fix
Author: Richard Yu <richardyu@Richards-Air.attlocal.net>
Reviewers: Jason Gustafson <jason@confluent.io>
Closes#4110 from ConcurrencyPractitioner/trunk
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jason Gustafson <jason@confluent.io>
Closes#4158 from ijuma/kafka-2903-file-records-read-slice-size-greater
This has been disabled since the start and since
it's removed in TLS 1.3, there are no plans to
ever support it.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4034 from ijuma/remove-tls-renegotiation-support
A couple of root causes of this flaky test is fixed:
1. The MockTime was incorrectly used across multiple test methods within the class, as a class rule. Instead we set it on each test case; also remove the scala MockTime dependency.
2. List topics may not contain the deleted topics while their ZK paths are yet to be deleted; so the delete-check-recreate pattern may fail to successfully recreate the topic at all. Change the checking to read from zk path directly instead.
Another minor fix is to remove the misleading wait condition error message as the accumData is always empty.
Author: Guozhang Wang <wangguoz@gmail.com>
Reviewers: Bill Bejeck <bill@confluent.io>, Damian Guy <damian.guy@gmail.com>, Matthias J. Sax <matthias@confluent.io>
Closes#4095 from guozhangwang/KMinor-reset-integration-test
The methods resetReconnectBackoff and updateReconnectBackoff in ClusterConnectionStates both take an instance of a private inner class as parameter and thus cannot be called from outside the class anyway.
Author: Soenke Liebau <soenke.liebau@opencore.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4114 from soenkeliebau/MINOR_private
The idempotent producer doesn't change that setting any more and the
accepted range has changed.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Apurva Mehta <apurva@confluent.io>, Jason Gustafson <jason@confluent.io>
Closes#4097 from ijuma/fix-javadoc-wrt-max-in-flight-for-idempotent
Currently, in branches _trunk_, _0.11.0_, and _1.0_ the property **max.in.flight.requests.per.connection** is incorrectly misspelled as _max.inflight.requests.per.connection_
harshach ijuma guozhangwang can you please review. Thank you.
Author: Hugo Louro <hmclouro@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#4094 from hmcl/trunk_MINOR_Doc_InflightProp
Author: Dong Lin <lindong28@gmail.com>
Reviewers: Tom Bentley <tbentley@redhat.com>, Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
Closes#3874 from lindong28/KAFKA-5163
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#3814 from omkreddy/KAFKA-4504
Author: Jacek Laskowski <jacek@japila.pl>
Reviewers: Apurva Mehta <apurva@confluent.io>, Jason Gustafson <jason@confluent.io>
Closes#4038 from jaceklaskowski/KAFKA-4818-isolationLevel
Author: Matthias J. Sax <matthias@confluent.io>
Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Closes#4064 from mjsax/minor-add-state-serdes-test
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#4035 from hachikuji/KAFKA-5547-followup and squashes the following commits:
f6b04ce1a [Jason Gustafson] Add a couple missed common fields
d3473b14d [Jason Gustafson] Fix compilation errors and a few warnings
58a0ae695 [Jason Gustafson] MINOR: Avoid some unnecessary collection copies in KafkaApis
With these changes, we are ensuring that the partitions being reassigned are from non-zero offsets. We also ensure that every message in the log has producerId and sequence number.
This means that it successfully reproduces https://issues.apache.org/jira/browse/KAFKA-6003.
Author: Apurva Mehta <apurva@confluent.io>
Reviewers: Jason Gustafson <jason@confluent.io>
Closes#4029 from apurvam/KAFKA-6016-add-idempotent-producer-to-reassign-partitions