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 scalac optimizer is able to inline methods to avoid lambda allocations, eliminating
the runtime cost of higher order functions in many cases. The compilation parameters
we are using here were introduced in 2.12.x, so we don't enable them for Scala 2.11.
Also, we enable a more aggressive inlining policy for the `core` project since it's
not meant to be used as a library.
See https://www.lightbend.com/blog/scala-inliner-optimizer for more information about
the optimizer.
I verified that the lambda allocation in the code below (from LogCleaner.scala) went away
after this change with Scala 2.12 and 2.13.
```scala
private def consumeAbortedTxnsUpTo(offset: Long): Unit = {
while (abortedTransactions.headOption.exists(_.firstOffset <= offset)) {
val abortedTxn = abortedTransactions.dequeue()
ongoingAbortedTxns.getOrElseUpdate(abortedTxn.producerId, new AbortedTransactionMetadata(abortedTxn))
}
}
```
The relevant part of the bytecode when compiled with Scala 2.13 looks like:
```text
private void consumeAbortedTxnsUpTo(long);
Code:
0: aload_0
1: invokespecial #54 // Method abortedTransactions:()Lscala/collection/mutable/PriorityQueue;
4: invokevirtual #175 // Method scala/collection/mutable/PriorityQueue.headOption:()Lscala/Option;
7: dup
8: ifnonnull 13
11: aconst_null
12: athrow
13: astore 4
15: aload 4
17: invokevirtual #145 // Method scala/Option.isEmpty:()Z
20: ifne 48
23: aload 4
25: invokevirtual #148 // Method scala/Option.get:()Ljava/lang/Object;
28: checkcast #177 // class kafka/log/AbortedTxn
```
The increased inlining causes some spurious spotBugs warnings, I added a few suppressions
and fixed one warning by avoiding unnecessary boxing.
Reviewers: 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
The Kafka Clients library includes a version file that contains its version and Git commit ID.
Since Kafka Streams wants to expose version and commit ID in the metrics it needs to read the version file. To enable the users to check during runtime for version mismatches between the Streams library and the Clients library, the version file is copied from Clients during build
time and during runtime only the Streams version file is read.
If Streams would read Clients' version file during runtime, it would read a wrong version and commit ID if the libraries where not build from repositories in different states.
Reviewers: John Roesler <vvcephei@users.noreply.github.com>, Guozhang Wang <wangguoz@gmail.com>
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>
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>
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>
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>
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>
- include Scala 2.13 in gradle build
- handle future milestone and RC versions of Scala in a better way
- if no Scala version is specified, default to scala 2.12 (bump from 2.11)
- include certain Xlint options (removed by Scala 2.13) for Scala 2.11/2.12 build only
- upgrade versions for dependencies:
- scalaLogging: 3.9.0 -->> 3.9.2
- scalatest: 3.0.7 -->> 3.0.8
- scoverage: 1.3.1 -->> 1.4.0
Reviewers: Ewen Cheslack-Postava <me@ewencp.org>, Ismael Juma <ismael@juma.me.uk>
This commit makes three changes:
- Adds a constructor for NewTopic(String, Optional<Integer>, Optional<Short>)
which allows users to specify Optional.empty() for numPartitions or
replicationFactor in order to use the broker default.
- Changes AdminManager to accept -1 as valid options for replication
factor and numPartitions (resolving to broker defaults).
- Makes --partitions and --replication-factor optional arguments when creating
topics using kafka-topics.sh.
- Adds a dependency on scalaJava8Compat library to make it simpler to
convert Scala Option to Java Optional
Reviewers: Ismael Juma <ismael@juma.me.uk>, Ryanne Dolan <ryannedolan@gmail.com>, Jason Gustafson <jason@confluent.io>
Reviewers: Bill Bejeck <bill@confluent.io>, Boyang Chen <boyang@confluent.io>, Bruno Cadonna <bruno@confluent.io>, Guozhang Wang <guozhang@confuent.io>
* StreamsMetricsImpl wraps the Kafka Streams' metrics registry and provides logic to create
and register sensors and their corresponding metrics. An example for such logic can be found in
threadLevelSensor(). Furthermore, StreamsMetricsmpl keeps track of the sensors on the
different levels of an application, i.e., thread, task, etc., and provides logic to remove sensors per
level, e.g., removeAllThreadLevelSensors(). There is one StreamsMetricsImpl object per
application instance.
* ThreadMetrics contains only static methods that specify all built-in thread-level sensors and
metrics and provide logic to register and retrieve those thread-level sensors, e.g., commitSensor().
* From anywhere inside the code base with access to StreamsMetricsImpl, thread-level sensors can be accessed by using ThreadMetrics.
* ThreadsMetrics does not inherit from StreamsMetricsImpl anymore.
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, John Roesler <john@confluent.io>, Guozhang Wang <guozhang@confluent.io>
Some versions of OpenJDK 11 do not properly handle external javadocs links referencing previous Java versions. See: https://bugs.openjdk.java.net/browse/JDK-8212233.
Failure symptom:
`> Task :connect:api:javadoc
javadoc: error - The code being documented uses modules but the packages defined in https://docs.oracle.com/javase/8/docs/api/ are in the unnamed module.
1 error`
This PR conditionally sets the Java api docs link for the affected Gradle tasks. I verified that the links render correctly in the generated documentation when building with `1.8.0_181` and `11.0.3`. For example, in `build/docs/javadoc/org/apache/kafka/connect/source/SourceTask.html` the hyperlink to `java.nio.channels.Selector` points to a valid page on Oracle's site in both cases.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Jason Gustafson <jason@confluent.io>
Added the incremental cooperative rebalancing in Connect to avoid global rebalances on all connectors and tasks with each new/changed/removed connector. This new protocol is backward compatible and will work with heterogeneous clusters that exist during a rolling upgrade, but once the clusters consist of new workers only some affected connectors and tasks will be rebalanced: connectors and tasks on existing nodes still in the cluster and not added/changed/removed will continue running while the affected connectors and tasks are rebalanced.
This commit attempted to minimize the changes to the existing V0 protocol logic, though that was not entirely possible.
This commit adds extensive unit and integration tests for both the old V0 protocol and the new v1 protocol. Soak testing has been performed multiple times to verify behavior while connectors and added, changed, and removed and while workers are added and removed from the cluster.
Author: Konstantine Karantasis <konstantine@confluent.io>
Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <me@ewencp.org>, Robert Yokota <rayokota@gmail.com>, David Arthur <mumrah@gmail.com>, Ryanne Dolan <ryannedolan@gmail.com>
Verified that the https links work.
I didn't update the license header in this PR since that touches
so many files. Will file a separate one for that.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
* Describe/Delete/Reset offsets on multiple consumer groups at a time (including each group by repeating `--group` parameter)
* Describe/Delete/Reset offsets on ALL consumer groups at a time (add new `--all-groups` option similar to `--all-topics`)
* Reset plan CSV file generation reworked: structure updated to support multiple consumer groups and make sure that CSV file generation is done properly since there are no restrictions on consumer group names and symbols like commas and quotes are allowed.
* Extending data output table format by adding `GROUP` column for all `--describe` queries
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
The presence of the buildSrc subproject is causing problems when we try
to run installAll, jarAll, and the other "all" targets. It's easier
just to make the generator code a regular subproject and use the
JavaExec gradle task to run the code. This also makes it more
straightforward to run the generator unit tests.
Reviewers: David Arthur <mumrah@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Co-authored-by: Colin P. Mccabe <cmccabe@confluent.io>
Co-authored-by: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
Track the last seen partition epoch in the Metadata class. When handling metadata updates, check that the partition info being received is for the last seen epoch or a newer one. This prevents stale metadata from being loaded into the client.
Reviewers: Jason Gustafson <jason@confluent.io>
Expose a programmatic way to bring up a Kafka and Zk cluster through Java API to facilitate integration tests for framework level changes in Kafka Connect. The Kafka classes would be similar to KafkaEmbedded in streams. The new classes would reuse the kafka.server.KafkaServer classes from :core, and provide a simple interface to bring up brokers in integration tests.
Signed-off-by: Arjun Satish <arjunconfluent.io>
Author: Arjun Satish <arjun@confluent.io>
Author: Arjun Satish <wicknicks@users.noreply.github.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5516 from wicknicks/connect-integration-test
This patch adds a framework to automatically generate the request/response classes for Kafka's protocol. The code will be updated to use the generated classes in follow-up patches. Below is a brief summary of the included components:
**buildSrc/src**
The message generator code is here. This code is automatically re-run by gradle when one of the schema files changes. The entire directory is processed at once to minimize the number of times we have to start a new JVM. We use Jackson to translate the JSON files into Java objects.
**clients/src/main/java/org/apache/kafka/common/protocol/Message.java**
This is the interface implemented by all automatically generated messages.
**clients/src/main/java/org/apache/kafka/common/protocol/MessageUtil.java**
Some utility functions used by the generated message code.
**clients/src/main/java/org/apache/kafka/common/protocol/Readable.java, Writable.java, ByteBufferAccessor.java**
The generated message code uses these classes for writing to a buffer.
**clients/src/main/message/README.md**
This README file explains how the JSON schemas work.
**clients/src/main/message/\*.json**
The JSON files in this directory implement every supported version of every Kafka API. The unit tests automatically validate that the generated schemas match the hand-written schemas in our code. Additionally, there are some things like request and response headers that have schemas here.
**clients/src/main/java/org/apache/kafka/common/utils/ImplicitLinkedHashSet.java**
I added an optimization here for empty sets. This is useful here because I want all messages to start with empty sets by default prior to being loaded with data. This is similar to the "empty list" optimizations in the `java.util.ArrayList` class.
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Ismael Juma <ismael@juma.me.uk>, Bob Barrett <bob.barrett@outlook.com>, Jason Gustafson <jason@confluent.io>
As documented in https://issues.apache.org/jira/browse/KAFKA-7741,
the javax dependency we receive transitively from connect is incompatible
with SBT builds.
Streams doesn't use the portion of Connect that needs the dependency,
so we can fix the builds by simply excluding it.
Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>
The StreamsUpgradeTest::test_upgrade_downgrade_brokers used sleep calls in the test which led to flaky test performance and as a result, we placed an @ignore annotation on the test. This PR uses log events instead of the sleep calls hence we can now remove the @ignore setting.
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Gradle 5.0 was released on 26 November. See the release notes for enhancements and fixes: https://docs.gradle.org/5.0/release-notes.html.
A notable bugfix ensures that Javadoc artifacts are cleaned between builds.
The upgraded wrapper was spot-checked against the commands in the README and the
README was updated not to use removed system property `-Dtest.single`.
Reviewers: Ismael Juma <ismael@juma.me.uk>
See https://github.com/spotbugs/spotbugs/issues/756 for details on
the false positives affecting try with resources. An example is:
> RCN | Nullcheck of fc at line 629 of value previously dereferenced in
> org.apache.kafka.common.utils.Utils.readFileAsString(String, Charset)
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
KAFKA-7597: Add configurable transaction support to ProduceBenchWorker. In order to get support for serializing Optional<> types to JSON, add a new library: jackson-datatype-jdk8. Once Jackson 3 comes out, this library will not be needed.
Reviewers: Colin McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
- Use Xlint:all with 3 exclusions (filed KAFKA-7613 to remove the exclusions)
- Use the same javac options when compiling tests (seems accidental that
we didn't do this before)
- Replaced several deprecated method calls with non-deprecated ones:
- `KafkaConsumer.poll(long)` and `KafkaConsumer.close(long)`
- `Class.newInstance` and `new Integer/Long` (deprecated since Java 9)
- `scala.Console` (deprecated in Scala 2.11)
- `PartitionData` taking a timestamp (one of them seemingly a bug)
- `JsonMappingException` single parameter constructor
- Fix unnecessary usage of raw types in several places.
- Add @SuppressWarnings for deprecations, unchecked and switch fallthrough in
several places.
- Scala clean-ups (var -> val, ETA expansion warnings, avoid reflective calls)
- Use lambdas to simplify code in a few places
- Add @SafeVarargs, fix varargs usage and remove unnecessary `Utils.mkList` method
Reviewers: Matthias J. Sax <mjsax@apache.org>, Manikumar Reddy <manikumar.reddy@gmail.com>, Randall Hauch <rhauch@gmail.com>, Bill Bejeck <bill@confluent.io>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
EasyMock 4.0.x includes a change that relies on the caller for inferring
the return type of mock creator methods. Updated a number of Scala
tests for compilation and execution to succeed.
The versions of EasyMock and PowerMock in this PR include full support
for Java 11.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
- SslFactoryTest should use SslFactory to create SSLEngine
- Use Mockito instead of EasyMock in `ConsoleConsumerTest` as one of
the tests mocks a standard library class and the latest released EasyMock
version can't do that when Java 11 is used.
- Avoid mocking `ConcurrentMap` in `SourceTaskOffsetCommitterTest`
for similar reasons. As it happens, mocking is not actually needed here.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Development of EasyMock and PowerMock has stagnated while Mockito
continues to be actively developed. With the new Java release cadence,
it's a problem to depend on libraries that do bytecode manipulation
and are not actively maintained. In addition, Mockito is also
easier to use.
While updating the tests, I attempted to go from failing test to
passing test. In cases where the updated test passed on the first
attempt, I artificially broke it to ensure the test was still doing its
job.
I included a few improvements that were helpful while making these
changes:
1. Better exception if there are no nodes in `leastLoadedNodes`
2. Always close the producer in `KafkaProducerTest`
3. requestsInFlight producer metric should not hold a reference to
`Sender`
Finally, `Metadata` is no longer final so that we don't need
`PowerMock` to mock it. It's an internal class, so it's OK.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Viktor Somogyi <viktorsomogyi@gmail.com>, Dong Lin <lindong28@gmail.com>
Closes#5691 from ijuma/kafka-7438-mockito
Removed ignore annotations from the upgrade tests. This PR includes the following changes for updating the upgrade tests:
* Uploaded new versions 0.10.2.2, 0.11.0.3, 1.0.2, 1.1.1, and 2.0.0 (in the associated scala versions) to kafka-packages
* Update versions in version.py, Dockerfile, base.sh
* Added new versions to StreamsUpgradeTest.test_upgrade_downgrade_brokers including version 2.0.0
* Added new versions StreamsUpgradeTest.test_simple_upgrade_downgrade test excluding version 2.0.0
* Version 2.0.0 is excluded from the streams upgrade/downgrade test as StreamsConfig needs an update for the new version, requiring a KIP. Once the community votes the KIP in, a minor follow-up PR can be pushed to add the 2.0.0 version to the upgrade test.
* Fixed minor bug in kafka-run-class.sh for classpath in upgrade/downgrade tests across versions.
* Follow on PRs for 0.10.2x, 0.11.0x, 1.0.x, 1.1.x, and 2.0.x will be pushed soon with the same updates required for the specific version.
Reviewers: Eno Thereska <eno.thereska@gmail.com>, John Roesler <vvcephei@users.noreply.github.com>, Guozhang Wang <wangguoz@gmail.com>, Matthias J. Sax <matthias@confluent.io>
findBugs is abandoned, it doesn't work with Java 9 and the Gradle plugin will be deprecated in
Gradle 5.0: https://github.com/gradle/gradle/pull/6664
spotBugs is actively maintained and it supports Java 8, 9 and 10. Java 11 is not supported yet,
but it's likely to happen soon.
Also fixed a file leak in Connect identified by spotbugs.
Manually tested spotBugsMain, jarAll and importing kafka in IntelliJ and running
a build in the IDE.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Colin Patrick McCabe <colin@cmccabe.xyz>, Dong Lin <lindong28@gmail.com>
Closes#5625 from ijuma/kafka-5887-spotbugs