This patch changes the name of the `Resources` field of AlterConfigsResponse to `Responses`. This makes it consistent with AlterConfigsResponse, which has a differently-named but structurally-identical field. Tested with unit tests.
Reviewers: Jason Gustafson <jason@confluent.io>
We have seen some failures recently in `RestServerTest`. It's the usual problem with reliance on static ports.
```
Caused by: java.io.IOException: Failed to bind to 0.0.0.0/0.0.0.0:8083
at org.eclipse.jetty.server.ServerConnector.openAcceptChannel(ServerConnector.java:346)
at org.eclipse.jetty.server.ServerConnector.open(ServerConnector.java:308)
at org.eclipse.jetty.server.AbstractNetworkConnector.doStart(AbstractNetworkConnector.java:80)
at org.eclipse.jetty.server.ServerConnector.doStart(ServerConnector.java:236)
at org.eclipse.jetty.util.component.AbstractLifeCycle.start(AbstractLifeCycle.java:68)
at org.eclipse.jetty.server.Server.doStart(Server.java:396)
at org.eclipse.jetty.util.component.AbstractLifeCycle.start(AbstractLifeCycle.java:68)
at org.apache.kafka.connect.runtime.rest.RestServer.initializeServer(RestServer.java:178)
... 56 more
Caused by: java.net.BindException: Address already in use
```
This patch makes the chosen port dynamic.
Reviewers: Ismael Juma <ismael@juma.me.uk>
The RegexSourceIntegrationTest has some flakiness as it deletes and re-creates the same output topic before each test. This PR reduces the chance for errors by creating a unique output topic for each test.
Reviewers: Matthias J. Sax <mjsax@apache.org>, Boyang Chen <boyang@confluent.io>
Correct the Flatten SMT to properly handle null key or value `Struct` instances.
Author: Michal Borowiecki <michal.borowiecki@openbet.com>
Reviewers: Arjun Satish <arjun@confluent.io>, Robert Yokota <rayokota@gmail.com>, Randall Hauch <rhauch@gmail.com>
fix checkpoint file warning by filtering checkpointable offsets per task
clean up state manager hierarchy to prevent similar bugs
Reviewers: Bruno Cadonna <bruno@confluent.io>, Bill Bejeck <bbejeck@gmail.com>
This patch fixes a compatibility breaking `MemberDescription` constructor change in #6957. It also updates `equals` and `hashCode` for the new `groupInstanceId` field that was added in the same patch.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
The purpose of this patch is to restrict the paths for updating and accessing the high watermark and the last stable offset. By doing so, we can validate that these offsets always remain within the range of the log. We also ensure that we never expose `LogOffsetMetadata` unless it is fully materialized. Finally, this patch makes a few naming changes. In particular, we remove the `highWatermark_=` and `highWatermarkMetadata_=` which are both misleading and cumbersome to test.
Reviewers: David Arthur <mumrah@gmail.com>
The rebalance timeout was added to the JoinGroup protocol in version 1. Prior to 2.3,
we handled version 0 JoinGroup requests by setting the rebalance timeout to be equal
to the session timeout. We lost this logic when we converted the API to use the
generated schema definition (#6419) which uses the default value of -1. The impact
of this is that the group rebalance timeout becomes 0, so rebalances finish immediately
after we enter the PrepareRebalance state and kick out all old members. This causes
consumer groups to enter an endless rebalance loop. This patch restores the old
behavior.
Reviewers: Ismael Juma <ismael@juma.me.uk>
ZooKeeper 3.5.5 is the first stable release in the 3.5.x series. The key new feature
in is TLS support, but there are a few more noteworthy features:
* Dynamic reconfiguration
* Local sessions
* New node types: Container, TTL
* Ability to remove watchers
* Multi-threaded commit processor
* Upgraded to Netty 4.1
See the release notes for more detail:
https://zookeeper.apache.org/doc/r3.5.5/releasenotes.html
In addition to the version bump, we:
* Add `commons-cli` dependency as it's required by `ZooKeeperMain`, but specified as
`provided` in their pom.
* Remove unnecessary `ZooKeeperMainWrapper`, the bug it worked around was fixed
upstream a long time ago.
* Ignore non zero exit in one system test invocation of `ZooKeeperMain`.
`ZooKeeperMainWrapper` always returned `0` and `ZooKeeperService.query` relies
on that for correct behavior.
Reviewers: Jason Gustafson <jason@confluent.io>
We noticed a lot of messages like the following in the broker logs recently:
```
[2019-07-10 02:01:23,946] WARN [ReplicaManager broker=0] While updating the HW for follower -1 for partition connect-storage-topic-connect-cluster-0, the replica could not be found. (kafka.server.ReplicaManager:70)
```
In the KIP-392 PR, we added logic to track the high watermark of followers, but it is invoked even for consumer fetches. This doesn't cause any harm other than all the log noise.
This patch just adds the missing follower check.
Reviewers: David Arthur <mumrah@gmail.com>
Clarify the behavior of `max.poll.interval.ms` for static consumers since it is slightly different from dynamic members.
Reviewers: Bill Bejeck <bbejeck@gmail.com>, Matthias J. Sax <mjsax@apache.org>, Jason Gustafson <jason@confluent.io>
We don't allow changing number of partitions for internal topics. To do
so we check if the topic name belongs to the set of internal topics
directly instead of using the "isInternalTopic" method. This breaks the
encapsulation by making client aware of the fact that internal topics
have special names.
This is a simple change to use the method `Topic::isInternalTopic`
method instead of checking it directly in "alterTopic" command. We
also reduce visibility to `Topic::INTERNAL_TOPICS` to avoid
unnecessary reliance on it in the future.
Reviewers: Jason Gustafson <jason@confluent.io>
A bug in `WorkerConfigTransformer` prevents the connector configuration reload when the ConfigData TTL expires.
The issue boils down to the fact that `worker.herder().restartConnector` is receiving a null callback.
```
[2019-06-17 14:34:12,320] INFO Scheduling a restart of connector workshop-incremental in 60000 ms (org.apache.kafka.connect.runtime.WorkerConfigTransformer:88)
[2019-06-17 14:34:12,321] ERROR Uncaught exception in herder work thread, exiting: (org.apache.kafka.connect.runtime.distributed.DistributedHerder:227)
java.lang.NullPointerException
at org.apache.kafka.connect.runtime.distributed.DistributedHerder$19.onCompletion(DistributedHerder.java:1187)
at org.apache.kafka.connect.runtime.distributed.DistributedHerder$19.onCompletion(DistributedHerder.java:1183)
at org.apache.kafka.connect.runtime.distributed.DistributedHerder.tick(DistributedHerder.java:273)
at org.apache.kafka.connect.runtime.distributed.DistributedHerder.run(DistributedHerder.java:219)
```
This patch adds a callback which just logs the error.
Reviewers: Robert Yokota <rayokota@gmail.com>, Jason Gustafson <jason@confluent.io>
Users often use the RocksDBConfigSetter to modify parameters such as cache or block size, which must be set through the BlockBasedTableConfig object. Rather than creating a new object in the config setter, however, users should most likely retrieve a reference to the existing one so as to not lose the other defaults (eg the BloomFilter)
There have been notes from the community that it is not obvious this should be done, nor is it immediately clear how to do so. This PR updates the RocksDBConfigSetter docs to hopefully improve things.
I also piggybacked a few minor cleanups in the docs
Reviewers: Kamal Chandraprakash, Jim Galasyn <jim.galasyn@confluent.io>, Bruno Cadonna <bruno@confluent.io>, Bill Bejeck <bbejeck@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>
We recently saw a few failing tests recently due to the static reliance on port 5000. For example:
```
org.apache.kafka.common.KafkaException: Socket server failed to bind to localhost:5000: Address already in use.
at kafka.network.Acceptor.openServerSocket(SocketServer.scala:605)
at kafka.network.Acceptor.<init>(SocketServer.scala:481)
at kafka.network.SocketServer.createAcceptor(SocketServer.scala:253)
at kafka.network.SocketServer.$anonfun$createControlPlaneAcceptorAndProcessor$1(SocketServer.scala:234)
at kafka.network.SocketServer.$anonfun$createControlPlaneAcceptorAndProcessor$1$adapted(SocketServer.scala:232)
at scala.Option.foreach(Option.scala:438)
at kafka.network.SocketServer.createControlPlaneAcceptorAndProcessor(SocketServer.scala:232)
at kafka.network.SocketServer.startup(SocketServer.scala:119)
at kafka.network.SocketServerTest.withTestableServer(SocketServerTest.scala:1139)
at kafka.network.SocketServerTest.testControlPlaneRequest(SocketServerTest.scala:198)
```
This patch fixes the failing tests to dynamically select the port.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Follow on to #6731, this PR adds broker-side support for [KIP-392](https://cwiki.apache.org/confluence/display/KAFKA/KIP-392%3A+Allow+consumers+to+fetch+from+closest+replica) (fetch from followers).
Changes:
* All brokers will handle FetchRequest regardless of leadership
* Leaders can compute a preferred replica to return to the client
* New ReplicaSelector interface for determining the preferred replica
* Incremental fetches will include partitions with no records if the preferred replica has been computed
* Adds new JMX to expose the current preferred read replica of a partition in the consumer
Two new conditions were added for completing a delayed fetch. They both relate to communicating the high watermark to followers without waiting for a timeout:
* For regular fetches, if the high watermark changes within a single fetch request
* For incremental fetch sessions, if the follower's high watermark is lower than the leader
A new JMX attribute `preferred-read-replica` was added to the `kafka.consumer:type=consumer-fetch-manager-metrics,client-id=some-consumer,topic=my-topic,partition=0` object. This was added to support the new system test which verifies that the fetch from follower behavior works end-to-end. This attribute could also be useful in the future when debugging problems with the consumer.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Jun Rao <junrao@gmail.com>, Jason Gustafson <jason@confluent.io>
This PR fixes a wrong input stream name in PipeDemo's javadoc.
Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Jason Gustafson <jason@confluent.io>
Add a "useConfiguredPartitioner" boolean to specify testing with the configured partitioner, rather than overriding the partitioner in the test.
Add a "skipFlush" boolean to specify skipping the flush operation when producing. This is helpful when testing some scenarios where linger.ms is greater than 0.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
The Pipe.java file should exist within the myapps package directory.
Reviewers: Boyang Chen <boyang@confluent.io>, Jason Gustafson <jason@confluent.io>
The patch clarifies the exception message for unknown key versions when loading from the group metadata topic. The patch also makes a trivial change in `KafkaAdminClient` to use `Map.computeIfAbsent`.
Reviewers: Viktor Somogyi <viktorsomogyi@gmail.com>, Jason Gustafson <jason@confluent.io>
`EmbeddedConnectCluster` has the ability to mask system exits to avoid killing the jvm. It appears that the default was intended to be `true`, but is actually `false`. The `maskExitProcedures` method on `EmbeddedConnectCluster.Builder` documents the parameter as:
```
* @param mask if false, exit and halt procedures remain unchanged; true is the default.
```
Because this is not enabled by default as intended, we are seeing some build failures which exit abruptly:
```
17:29:11 Execution failed for task ':connect:runtime:integrationTest'.
17:29:11 > Process 'Gradle Test Executor 25' finished with non-zero exit value 1
```
The culprit often appears to be `ExampleConnectIntegrationTest`, which indeed does not override the default value of `maskExitProcedures`.
Reviewers: Ewen Cheslack-Postava <me@ewencp.org>
KIP-91 was included in Kafka 2.1.0, so we should mention
`delivery.timeout.ms` in the hint as it's the config that
users would want to change in most cases.
Reviewers: Matthias J. Sax <matthias@confluent.io>, John Roesler <john@confluent.io>, Bill Bejeck <bbejeck@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
Leaders should make changes to the assignment and the ISR at the same time as part of processing the LeaderAndIsr requests. The leader should also preserve the order of assignment mainly for consistency with the Controller's code and data representation.
Reviewers: Vikas Singh, David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>
Connect tests were using String version for KafkaService instead of the expected KafkaVersion object. This broke due to recent changes to KafkaVersion. It turns out that the tests with String version were running compatibility tests against `dev` brokers rather than the older broker versions they were expecting to run against. When version was fixed, tests using 0.9.0.1 brokers started failing since new clients are not compatible with 0.9.0.1 brokers. So this PR fixes version parameter and removes the two tests against 0.9.0.1 brokers.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Rajini Sivaram <rajinisivaram@googlemail.com>
Scala 2.13 support was added to build via #5454. This PR adjusts the code so that
it compiles with 2.11, 2.12 and 2.13.
Changes:
* Add `scala-collection-compat` dependency.
* Import `scala.collection.Seq` in a number of places for consistent behavior between
Scala 2.11, 2.12 and 2.13.
* Remove wildcard imports that were causing the Java classes to have priority over the
Scala ones, related Scala issue: https://github.com/scala/scala/pull/6589.
* Replace parallel collection usage with `Future`. The former is no longer included by
default in the standard library.
* Replace val _: Unit workaround with one that is more concise and works with Scala 2.13
* Replace `filterKeys` with `filter` when we expect a `Map`. `filterKeys` returns a view
that doesn't implement the `Map` trait in Scala 2.13.
* Replace `mapValues` with `map` or add a `toMap` as an additional transformation
when we expect a `Map`. `mapValues` returns a view that doesn't implement the
`Map` trait in Scala 2.13.
* Replace `breakOut` with `iterator` and `to`, `breakOut` was removed in Scala
2.13.
* Replace to() with toMap, toIndexedSeq and toSet
* Replace `mutable.Buffer.--` with `filterNot`.
* ControlException is an abstract class in Scala 2.13.
* Variable arguments can only receive arrays or immutable.Seq in Scala 2.13.
* Use `Factory` instead of `CanBuildFrom` in DecodeJson. `CanBuildFrom` behaves
a bit differently in Scala 2.13 and it's been deprecated. `Factory` has the behavior
we need and it's available via the compat library.
* Fix failing tests due to behavior change in Scala 2.13,
"Map.values.map is not strict in Scala 2.13" (https://github.com/scala/bug/issues/11589).
* Use Java collections instead of Scala ones in StreamResetter (a Java class).
* Adjust CheckpointFile.write to take an `Iterable` instead of `Seq` to avoid
unnecessary collection copies.
* Fix DelayedElectLeader to use a Map instead of Set and avoid `to` call that
doesn't work in Scala 2.13.
* Use unordered map for mapping in SimpleAclAuthorizer, mapping of ordered
maps require an `Ordering` in Scala 2.13 for safety reasons.
* Adapt `ConsumerGroupCommand` to compile with Scala 2.13.
* CoreUtils.min takes an `Iterable` instead of `TraversableOnce`, the latter does
not exist in Scala 2.13.
* Replace `Unit` with `()` in a couple places. Scala 2.13 is stricter when it expects
a value instead of a type.
* Fix bug in CustomQuotaCallbackTest where we did not necessarily set `partitionRatio`
correctly, `forall` can terminate early.
* Add a couple of spotbugs exclusions that are needed by code generated by Scala 2.13
* Remove unused variables, simplify some code and remove procedure syntax in a few
places.
* Remove unused `CoreUtils.JSONEscapeString`.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, José Armando García Sancio <jsancio@users.noreply.github.com>
Due to the accidental duplication of `case e: ExecutionException`,
the verification code was not actually running.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, David Arthur <mumrah@gmail.com>
Followers should cache the log offset metadata for the start offset of each transaction in order to be able to compute the last stable offset without an offset index lookup. This is needed for follower fetching in KIP-392.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Include group.instance.id in the describe group result for better visibility.
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
The purpose here is to leverage static membership information during round robin consumer assignment, because persistent member id could help make the assignment remain the same during rebalance.
The comparison logic is changed to:
1. If member A and member B both have group.instance.id, then compare their group.instance.id
2. If member A has group.instance.id, while member B doesn't, then A < B
3. If both member A and B don't have group.instance.id, compare their member.id
In round robin assignor, we use ephemeral member.id to sort the members in order for assignment. This semantic is not stable and could trigger unnecessary shuffle of tasks. By leveraging group.instance.id the static member assignment shall be persist when satisfying following conditions:
1. number of members remain the same across generation
2. static members' identities persist across generation
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Guozhang Wang <wangguoz@gmail.com>