Commit Graph

7658 Commits

Author SHA1 Message Date
Navina Ramesh 7777dc8f8f
KAFKA-10012; Reduce overhead of strings in SelectorMetrics (#8684)
`SelectorMetrics` has a per-connection metrics, which means the number of `MetricName` objects and the strings associated with it (such as group name and description) grows with the number of connections in the client. This overhead of duplicate string objects is amplified when there are multiple instances of kafka clients within the same JVM.  This patch addresses some of the memory overhead by making `metricGrpName` a constant field and introducing a new field `perConnectionMetricGrpName`. 

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
2020-06-07 18:44:44 -07:00
Evelyn Bayes 9a0b694a66
KAFKA-9216: Enforce internal config topic settings for Connect workers during startup (#8270)
Currently, Kafka Connect creates its config backing topic with a fire and forget approach.
This is fine unless someone has manually created that topic already with the wrong partition count.

In such a case Kafka Connect may run for some time. Especially if it's in standalone mode and once switched to distributed mode it will almost certainly fail.

This commits adds a check when the KafkaConfigBackingStore is starting.
This check will throw a ConfigException if there is more than one partition in the backing store.

This exception is then caught upstream and logged by either:
- DistributedHerder#run
- ConnectStandalone#main

A unit tests was added in KafkaConfigBackingStoreTest to verify the behaviour.

Author: Evelyn Bayes <evelyn@confluent.io>
Co-authored-by: Randall Hauch <rhauch@gmail.com>

Reviewer: Konstantine Karantasis <konstantine@confluent.io>
2020-06-07 12:42:00 -07:00
Boyang Chen d4ef46c69d
KAFKA-10097: Internalize checkpoint data (#8820)
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2020-06-06 17:34:03 -07:00
Brian Byrne dd7c036956 KAFKA-10033: Throw UnknownTopicOrPartitionException when modifying a non-existent topic's config
Author: Brian Byrne <bbyrne@confluent.io>

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Boyang Chan <boyang@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>

Closes #8717 from bdbyrne/KAFKA-10033
2020-06-06 21:04:04 +05:30
Matthias J. Sax 4eb3f75556
MINOR: improve code encapsulation between StreamThread and TaskManager (#8819)
Reviewers: Boyang Chen <boyang@confluent.io>, John Roesler <john@confluent.io>
2020-06-05 20:29:50 -07:00
Mandar Tillu bdae26d047
Fixing KAFKA-10094 (#8797) 2020-06-05 15:56:32 -07:00
Konstantine Karantasis 464aca3627
KAFKA-9851: Revoking Connect tasks due to connectivity issues should also clear the running assignment (#8804)
Until recently revocation of connectors and tasks was the result of a rebalance that contained a new assignment. Therefore the view of the running assignment was kept consistent outside the call to `RebalanceListener#onRevoke`. However, after KAFKA-9184 the need appeared for the worker to revoke tasks voluntarily and proactively without having received a new assignment. 

This commit will allow the worker to restart tasks that have been stopped as a result of voluntary revocation after a rebalance reassigns these tasks to the work. 

The fix is tested by extending an existing integration test.

Reviewers: Randall Hauch <rhauch@gmail.com>
2020-06-05 15:56:02 -07:00
Boyang Chen 910f317996
KAFKA-9840; Skip End Offset validation when the leader epoch is not reliable (#8486)
This PR provides two fixes:
1. Skip offset validation if the current leader epoch cannot be reliably determined.
2. Raise an out of range error if the leader returns an undefined offset in response to the OffsetsForLeaderEpoch request.

Reviewers: Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
2020-06-05 15:53:13 -07:00
Bill Bejeck e385e4639e
HOT_FIX: Update javadoc since imports added (#8817)
Reviewers: John Roesler <john@confluent.io>, Matthias J. Sax <mjsax@apache.org>
2020-06-05 18:29:17 -04:00
Matthias J. Sax 5a0e65ed39
KAFKA-8011: Fix flaky RegexSourceIntegrationTest (#8799)
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2020-06-05 14:38:08 -07:00
Chris Egerton 2d9376c8bb
KAFKA-9570: Define SSL configs in all worker config classes, not just distributed (#8135)
Define SSL configs in all worker config classes, not just distributed

Author: Chris Egerton <chrise@confluent.io>
Reviewers: Nigel Liang <nigel@nigelliang.com>, Randall Hauch <rhauch@gmail.com>
2020-06-05 16:02:17 -05:00
Randall Hauch da32720c19
KAFKA-10111: Make SinkTaskContext.errantRecordReporter() a default method (#8814)
Connector projects may have their own mock or testing implementations of the `SinkTaskContext`, and this newly-added method should be a default method to prevent breaking those projects. Changing this to a default method that returns null also makes sense w/r/t the method semantics, since the method is already defined to return null if the reporter has not been configured.

Author: Randall Hauch <rhauch@gmail.com>
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
2020-06-05 15:20:24 -05:00
Randall Hauch b44ce35fe9
KAFKA-10110: Corrected potential NPE when null label value added to KafkaMetricsContext (#8811)
Also added a new unit test to verify the functionality and expectations.

Author: Randall Hauch <rhauch@gmail.com>
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
2020-06-05 15:19:23 -05:00
Randall Hauch 8f552fd6ac
MINOR: Change the order that Connect calls `config()` and `validate()` to avoid validating if the required ConfigDef is null (#8810)
Author: Randall Hauch <rhauch@gmail.com>
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
2020-06-05 15:02:11 -05:00
Xavier Léauté 7a876ec9de MINOR: fix backwards incompatibility in JmxReporter introduced by KIP-606
cc omkreddy this should also get backported to 2.6.x

Author: Xavier Léauté <xvrl@apache.org>

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>

Closes #8813 from xvrl/fix-jmx-reset
2020-06-06 01:30:48 +05:30
Bruno Cadonna b020671f31
MINOR: Fix javadoc warnings (#8809)
This commit fixes javadoc warnings caused by missing imports.

Reviewers: Bill Bejeck <bbejeck@apache.org>
2020-06-05 14:12:40 -04:00
Matthias J. Sax 9774c73e43
KAFKA-9441: Improve Kafka Streams task management (#8776)
- make task manager agnostic to task state
 - make tasks state transitions idempotent

Reviewers: Boyang Chen <boyang@confluent.io>, A. Sophie Blee-Goldman <sophie@confluent.io>, John Roesler <john@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2020-06-05 10:40:59 -07:00
Vito Jeng 31c81a6e9e
fix the broken links of streams javadoc (#8789)
There are some broken links of streams javadoc, for example:

ReadOnlyKeyValueStore
ReadOnlyWindowStore
Only javadoc is update.

Reviewers: Bill Bejeck <bbejeck@apache.org>
2020-06-05 12:21:47 -04:00
David Jacot 7867c24a40
KAFKA-10040; Make computing the PreferredReplicaImbalanceCount metric more efficient (#8724)
This PR changes the way `PreferredReplicaImbalanceCount` is computed. It moves from re-computing after the processing of each event in the controller, which requires a full pass over all partitions, to incrementally maintaining the count as assignments and leaders are changing.

Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Jason Gustafson <jason@confluent.io>
2020-06-04 17:31:18 -07:00
Matthias J. Sax 6383a22faf
KAFKA-10066: TestOutputTopic should pass record headers into deserializers (#8759)
Reviewers: John Roesler <john@confluent.io>, Boyang Chen <boyang@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2020-06-04 16:00:59 -07:00
Antony Stubbs 542b94be54
MINOR: Add explanation for disabling forwarding from value transformers (#8771)
Reviewers: Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>
2020-06-04 15:42:13 -07:00
gnkoshelev f1b0931447 KAFKA-10033: Throw UnknownTopicOrPartitionException if altering configs of non-existing topic
Fixes KAFKA-10033.

Replace AdminOperationException with UnknownTopicOrPartitionException if topic does not exist when validating topic configs in AdminZkClient.

Author: gnkoshelev <gnkoshelev@gmail.com>
Author: Gregory <gnkoshelev@gmail.com>

Reviewers: Brian Byrne <bbyrne@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>

Closes #8715 from gnkoshelev/KAFKA-10033
2020-06-05 00:02:43 +05:30
Tom Bentley 78e8a49cda
KAFKA-9434: automated protocol for alterReplicaLogDirs (#8311)
Reviewers: David Jacot <djacot@confluent.io>, Mickael Maison <mickael.maison@gmail.com>
2020-06-04 15:36:37 +01:00
Badai Aqrandista 50c3012890
KAFKA-9313: Set `use_all_dns_ips` as the new default for `client.dns.lookup` (KIP-602) (#8644)
This applies to the producer, consumer, admin client, connect worker
and inter broker communication.

`ClientDnsLookup.DEFAULT` has been deprecated and a warning
will be logged if it's explicitly set in a client config.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2020-06-04 06:21:52 -07:00
Chia-Ping Tsai 6074f864b9
KAFKA-10069: Correctly remove user-defined "predicate" and "negate" configs from transformation properties (#8755)
With the recent introduction of predicated SMTs, properties named "predicate" and "negate" should be ignored and removed in case they are present in transformation configs. 
This commit fixes the equality check to be with the key of the config to apply proper removal. 

Reviewers: Tom Bentley <tbentley@redhat.com>, Konstantine Karantasis <konstantine@confluent.io>
2020-06-03 22:13:50 -07:00
Bob Barrett 87bef1d1b0
KAFKA-9788; Use distinct names for transaction and group load time sensors (#8784)
Sensor objects are stored in the Kafka metrics registry and keyed by name. If a new sensor is created with the same name as an existing one, the existing one is returned rather than a new object being created. The partition load time sensors for the transaction and group coordinators used the same name, so data recorded to either was stored in the same object. This meant that the metrics values for both metrics were identical and consisted of the combined data. This patch changes the names to be distinct so that the data will be stored in separate Sensor objects.

Reviewers: Jason Gustafson <jason@confluent.io>
2020-06-03 17:53:30 -07:00
David Jacot 21362ad7f7
KAFKA-9514; The protocol generator generated useless condition when a field is made nullable and flexible version is used (#8793)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2020-06-03 17:06:22 -07:00
Ismael Juma 3af3ab9bf9
MINOR: Update to Gradle 6.5 and tweak build jvm config (#8751)
Gradle 6.5 includes a fix for https://github.com/gradle/gradle/pull/12866, which
affects the performance of Scala compilation.

I profiled the scalac build with async profiler and 54% of the time was on GC
even after the Gradle upgrade (it was more than 60% before), so I switched to
the throughput GC (GC latency is less important for batch builds) and it
was reduced to 38%.

I also centralized the jvm configuration in `build.gradle` and simplified it a bit
by removing the minHeapSize configuration from the test tasks.

On my desktop, the time to execute clean builds with no cached Gradle daemon
was reduced from 127 seconds to 97 seconds. With a cached daemon, it was
reduced from 120 seconds to 88 seconds. The performance regression when
we upgraded to Gradle 6.x was 27 seconds with a cached daemon 
(https://github.com/apache/kafka/pull/7677#issuecomment-616271179), so it
should be fixed now.

Gradle 6.4 with no cached daemon:

```
BUILD SUCCESSFUL in 2m 7s
115 actionable tasks: 112 executed, 3 up-to-date
./gradlew clean compileScala compileJava compileTestScala compileTestJava  1.15s user 0.12s system 0% cpu 2:08.06 total
```

Gradle 6.4 with cached daemon:

```
BUILD SUCCESSFUL in 2m 0s
115 actionable tasks: 111 executed, 4 up-to-date
./gradlew clean compileScala compileJava compileTestScala compileTestJava  0.95s user 0.10s system 0% cpu 2:01.42 total
```

Gradle 6.5 with no cached daemon:

```
BUILD SUCCESSFUL in 1m 46s
115 actionable tasks: 111 executed, 4 up-to-date
./gradlew clean compileScala compileJava compileTestScala compileTestJava  1.27s user 0.12s system 1% cpu 1:47.71 total
```

Gradle 6.5 with cached daemon:

```
BUILD SUCCESSFUL in 1m 37s
115 actionable tasks: 111 executed, 4 up-to-date
./gradlew clean compileScala compileJava compileTestScala compileTestJava  1.02s user 0.10s system 1% cpu 1:38.31 total
```

This PR with no cached Gradle daemon:

```
BUILD SUCCESSFUL in 1m 37s
115 actionable tasks: 81 executed, 34 up-to-date
./gradlew clean compileScala compileJava compileTestScala compileTestJava  1.27s user 0.10s system 1% cpu 1:38.70 total
```

This PR with cached Gradle daemon:

```
BUILD SUCCESSFUL in 1m 28s
115 actionable tasks: 111 executed, 4 up-to-date
./gradlew clean compileScala compileJava compileTestScala compileTestJava  1.02s user 0.10s system 1% cpu 1:29.35 total
```

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2020-06-03 13:18:50 -07:00
showuon 162884409d
MINOR: Upgrade spotbugs and spotbugsPlugin (#8790)
Upgrade spotbugsPlugin to have clear output indicating where the error is. 

When investigating KAFKA-10081, I found the error output of spotbugs is very poor. It doesn't even tell you where the error is and how many errors found, it will take a lot of time for the developers to find out where the error is, and then fix it.
![image](https://user-images.githubusercontent.com/43372967/83590263-efc42a80-a587-11ea-95cf-e9097d9a662e.png)
https://builds.apache.org/blue/organizations/jenkins/kafka-trunk-jdk8/detail/kafka-trunk-jdk8/4596/pipeline/

Then, I found out there's a bug in spotbugsPlugin in V4.0.x, and got fixed in V4.2.x
https://github.com/spotbugs/spotbugs-gradle-plugin/issues/210

So, after upgrading to V4.2.x (I followed to the latest version V4.2.4), the output is like this:
![image](https://user-images.githubusercontent.com/43372967/83590913-60b81200-a589-11ea-9a04-1449d693c2f2.png)
So you know there's 1 error and you can also open the report file to find out the error.

Upgraded to the latest bug fix release of spotbugs (4.0.3) while at it.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2020-06-03 13:17:31 -07:00
Chia-Ping Tsai 98cd4b8cab
KAFKA-10089 The stale ssl engine factory is not closed after reconfigure (#8792)
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
2020-06-03 19:04:21 +01:00
Jason Gustafson 0ffbc6e75f
KAFKA-10080; Fix race condition on txn completion which can cause duplicate appends (#8782)
The method `maybeWriteTxnCompletion` is unsafe for concurrent calls. This can cause duplicate attempts to write the completion record to the log, which can ultimately lead to illegal state errors and possible to correctness violations if another transaction had been started before the duplicate was written. This patch fixes the problem by ensuring only one thread can successfully remove the pending completion from the map.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
2020-06-03 10:37:53 -07:00
John Roesler cb88be45eb
KAFKA-10084: Fix EosTestDriver end offset (#8785)
Check the uncommitted end offset after the committed end offset,
so we can be sure never to miss a pending end-transaction marker.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2020-06-03 11:12:47 -05:00
A. Sophie Blee-Goldman 35a0692ce1
KAFKA-10083: fix failed testReassignmentWithRandomSubscriptionsAndChanges tests (#8786)
Minimum fix needed to stop this test failing and unblock others

Co-authored-by: Luke Chen <showuon@gmail.com>

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2020-06-02 20:39:02 -07:00
vinoth chandar ea6d373a93
KAFKA-9945: TopicCommand should support --if-exists and --if-not-exists when --bootstrap-server is used (#8737)
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
2020-06-02 16:17:05 -07:00
Nikolay 8b22b81596
KAFKA-9320: Enable TLSv1.3 by default (KIP-573) (#8695)
1. Enables `TLSv1.3` by default with Java 11 or newer.
2. Add unit tests that cover the various TLSv1.2 and TLSv1.3 combinations.
3. Extend `benchmark_test.py` and `replication_test.py` to run with 'TLSv1.2'
or 'TLSv1.3'.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2020-06-02 15:34:43 -07:00
showuon 9e5e4cd558
KAFKA-10082: Fix the failed testMultiConsumerStickyAssignment (#8777)
Fix the failed testMultiConsumerStickyAssignment by modifying the logic error in allSubscriptionsEqual method.

We will create the consumerToOwnedPartitions to keep the set of previously owned partitions encoded in the Subscription. It's our basis to do the reassignment. In the allSubscriptionsEqual, we'll get the member generation of the subscription, and remove all previously owned partitions as invalid if the current generation is higher. However, the logic before my fix, will remove the current highest member out of the consumerToOwnedPartitions, which should be kept because it's the current higher generation member. Fix this logic error.

Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2020-06-02 09:50:20 -07:00
showuon 1d7168ce5f
KAFKA-10081: remove an unused local variable to pass spotbugsMain check (#8774)
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Matthias J. Sax <matthias@confluent.io>
2020-06-02 09:13:01 -07:00
Nikolay a3d79ff236
MINOR: Remove unused variable to fix spotBugs failure (#8779)
Fixed spotBugs error introduced by c6633a1:

>Dead store to isFreshAssignment in org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.generalAssign(Map, Map)

Reviewers: Ismael Juma <ismael@juma.me.uk>
2020-06-02 08:59:56 -07:00
Rohan 07f6676539
MINOR: ChangelogReader should poll for duration 0 for standby restore (#8773)
Co-authored-by: Guozhang Wang <wangguoz@gmail.com>

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2020-06-01 22:33:22 -07:00
Randall Hauch 19e40788e7
Bump trunk to 2.7.0-SNAPSHOT (#8746) 2020-06-01 21:23:09 -05:00
Dima Reznik ec67788d9d
KAFKA-10030: Allow fetching a key from a single partition (#8706)
Reviewers: Navinder Pal Singh Brar <navinder_brar@yahoo.com>, Boyang Chen <boyang@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2020-06-01 18:06:28 -07:00
feyman2016 77f123391c
Kafka-10064 Add documentation for KIP-571 (#8760)
* Update documentation for KIP-571

* update doc for KIP-571

* fix comments
2020-06-01 17:19:26 -07:00
Chia-Ping Tsai f0d6e87421
MINOR: Remove no longer used ReflectionsUtil class (#8762)
This class has been unused since KIP-146 was merged, which introduced another way to load Connect plugins. Removing from the code base. 

Reviewers: Boyang Chan <boyang@confluent.io>, Konstantine Karantasis <konstantine@confluent.io>
2020-06-01 17:17:46 -07:00
showuon 88d996f9fa
MINOR: Code cleanup and assertion message fixes in Connect integration tests (#8750)
1. Remove redundant connect#stop from test in InternalTopicsIntegrationTest since we'll do it after each test case in the @After method
2. Refine the error message in topic assertions to make it better explain the errors

Reviewers: Konstantine Karantasis <konstantine@confluent.io>
2020-06-01 17:04:34 -07:00
A. Sophie Blee-Goldman c6633a157e
KAFKA-9987: optimize sticky assignment algorithm for same-subscription case (#8668)
Motivation and pseudo code algorithm in the ticket.

Added a scale test with large number of topic partitions and consumers and 30s timeout.
With these changes, assignment with 2,000 consumers and 200 topics with 2,000 each completes within a few seconds.

Porting the same test to trunk, it took 2 minutes even with a 100x reduction in the number of topics (ie, 2 minutes for 2,000 consumers and 2 topics with 2,000 partitions)

Should be cherry-picked to 2.6, 2.5, and 2.4

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2020-06-01 15:57:15 -07:00
Rajini Sivaram 66fdb59ed0
KAFKA-9392; Clarify deleteAcls javadoc and add test for create/delete timing (#7956)
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2020-06-01 16:38:21 +01:00
Ismael Juma e2e2c628b9
KAFKA-10074: Improve performance of `matchingAcls` (#8769)
This PR reduces allocations by using a plain old `foreach` in
`matchingAcls` and improves `AclSeqs.find` to only search the inner
collections that are required to find a match (instead of searching all
of them).

A recent change (90bbeedf52) in `matchingAcls` to remove `filterKeys` in
favor of filtering inside `flatMap` caused a performance regression in
cases where there are large number of topics, prefix ACLs and
TreeMap.from/to filtering is ineffective. In such cases, we rely on
string comparisons to exclude entries from the ACL cache that are not
relevant.

This issue is not present in any release yet, so we should include the
simple fix in the 2.6 branch.

The original benchmark did not show a performance difference, so I
adjusted the benchmark to stress the relevant code more. More
specifically, `aclCacheSnapshot.from(...).to(...)` returns nearly 20000
entries where each map value contains 1000 AclEntries. Out of the 200k
AclEntries, only 1050 are retained due to the `startsWith` filtering.

This is the case where the implementation in master is least
efficient when compared to the previous version and the version in this
PR.

The adjusted benchmark results for testAuthorizer are 4.532ms for
master, 2.903ms for the previous version and 2.877ms for this PR.
Normalized allocation rate was 593 KB/op for master, 597 KB/op for the
previous version and 101 KB/s for this PR. Full results follow:

master with adjusted benchmark:
```
Benchmark                                                                 (aclCount)  (resourceCount)  Mode  Cnt          Score          Error   Units
AclAuthorizerBenchmark.testAclsIterator                                           50           200000  avgt    5        680.805 ±       44.318   ms/op
AclAuthorizerBenchmark.testAclsIterator:·gc.alloc.rate                            50           200000  avgt    5        549.879 ±       36.259  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.alloc.rate.norm                       50           200000  avgt    5  411457042.000 ±     4805.461    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Eden_Space                   50           200000  avgt    5        331.110 ±       95.821  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Eden_Space.norm              50           200000  avgt    5  247799480.320 ± 72877192.319    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Survivor_Space               50           200000  avgt    5          0.891 ±        3.183  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Survivor_Space.norm          50           200000  avgt    5     667593.387 ±  2369888.357    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.count                                 50           200000  avgt    5         28.000                 counts
AclAuthorizerBenchmark.testAclsIterator:·gc.time                                  50           200000  avgt    5       3458.000                     ms
AclAuthorizerBenchmark.testAuthorizer                                             50           200000  avgt    5          4.532 ±        0.546   ms/op
AclAuthorizerBenchmark.testAuthorizer:·gc.alloc.rate                              50           200000  avgt    5        119.036 ±       14.261  MB/sec
AclAuthorizerBenchmark.testAuthorizer:·gc.alloc.rate.norm                         50           200000  avgt    5     593524.310 ±       22.452    B/op
AclAuthorizerBenchmark.testAuthorizer:·gc.churn.G1_Eden_Space                     50           200000  avgt    5        117.091 ±     1008.188  MB/sec
AclAuthorizerBenchmark.testAuthorizer:·gc.churn.G1_Eden_Space.norm                50           200000  avgt    5     598574.303 ±  5153905.271    B/op
AclAuthorizerBenchmark.testAuthorizer:·gc.churn.G1_Survivor_Space                 50           200000  avgt    5          0.034 ±        0.291  MB/sec
AclAuthorizerBenchmark.testAuthorizer:·gc.churn.G1_Survivor_Space.norm            50           200000  avgt    5        173.001 ±     1489.593    B/op
AclAuthorizerBenchmark.testAuthorizer:·gc.count                                   50           200000  avgt    5          1.000                 counts
AclAuthorizerBenchmark.testAuthorizer:·gc.time                                    50           200000  avgt    5         13.000                     ms
```

master with filterKeys like 90bbeedf52 and adjusted benchmark:
```
Benchmark                                                                 (aclCount)  (resourceCount)  Mode  Cnt          Score          Error   Units
AclAuthorizerBenchmark.testAclsIterator                                           50           200000  avgt    5        729.163 ±       20.842   ms/op
AclAuthorizerBenchmark.testAclsIterator:·gc.alloc.rate                            50           200000  avgt    5        513.005 ±       13.966  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.alloc.rate.norm                       50           200000  avgt    5  411459778.400 ±     3178.045    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Eden_Space                   50           200000  avgt    5        307.041 ±       94.544  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Eden_Space.norm              50           200000  avgt    5  246385400.686 ± 82294899.881    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Survivor_Space               50           200000  avgt    5          1.571 ±        2.590  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Survivor_Space.norm          50           200000  avgt    5    1258291.200 ±  2063669.849    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.count                                 50           200000  avgt    5         33.000                 counts
AclAuthorizerBenchmark.testAclsIterator:·gc.time                                  50           200000  avgt    5       3266.000                     ms
AclAuthorizerBenchmark.testAuthorizer                                             50           200000  avgt    5          2.903 ±        0.175   ms/op
AclAuthorizerBenchmark.testAuthorizer:·gc.alloc.rate                              50           200000  avgt    5        187.088 ±       11.301  MB/sec
AclAuthorizerBenchmark.testAuthorizer:·gc.alloc.rate.norm                         50           200000  avgt    5     597962.743 ±       14.237    B/op
AclAuthorizerBenchmark.testAuthorizer:·gc.churn.G1_Eden_Space                     50           200000  avgt    5        118.602 ±     1021.202  MB/sec
AclAuthorizerBenchmark.testAuthorizer:·gc.churn.G1_Eden_Space.norm                50           200000  avgt    5     383359.632 ±  3300842.044    B/op
AclAuthorizerBenchmark.testAuthorizer:·gc.count                                   50           200000  avgt    5          1.000                 counts
AclAuthorizerBenchmark.testAuthorizer:·gc.time                                    50           200000  avgt    5         14.000                     ms
```

This PR with adjusted benchmark:
```
Benchmark                                                                 (aclCount)  (resourceCount)  Mode  Cnt          Score          Error   Units
AclAuthorizerBenchmark.testAclsIterator                                           50           200000  avgt    5        706.774 ±       32.353   ms/op
AclAuthorizerBenchmark.testAclsIterator:·gc.alloc.rate                            50           200000  avgt    5        529.879 ±       25.416  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.alloc.rate.norm                       50           200000  avgt    5  411458751.497 ±     4424.187    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Eden_Space                   50           200000  avgt    5        310.559 ±      112.310  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Eden_Space.norm              50           200000  avgt    5  241364219.611 ± 97317733.967    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Old_Gen                      50           200000  avgt    5          0.690 ±        5.937  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Old_Gen.norm                 50           200000  avgt    5     531278.507 ±  4574468.166    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Survivor_Space               50           200000  avgt    5          2.550 ±       17.243  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Survivor_Space.norm          50           200000  avgt    5    1969325.592 ± 13278191.648    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.count                                 50           200000  avgt    5         32.000                 counts
AclAuthorizerBenchmark.testAclsIterator:·gc.time                                  50           200000  avgt    5       3489.000                     ms
AclAuthorizerBenchmark.testAuthorizer                                             50           200000  avgt    5          2.877 ±        0.530   ms/op
AclAuthorizerBenchmark.testAuthorizer:·gc.alloc.rate                              50           200000  avgt    5         31.963 ±        5.912  MB/sec
AclAuthorizerBenchmark.testAuthorizer:·gc.alloc.rate.norm                         50           200000  avgt    5     101057.225 ±        9.468    B/op
AclAuthorizerBenchmark.testAuthorizer:·gc.count                                   50           200000  avgt    5            ≈ 0                 counts
```

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2020-06-01 07:01:18 -07:00
Chia-Ping Tsai 36aa366434
KAFKA-10060 GroupMetadataManager should not log if there are no offsets to expire (#8767)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2020-05-31 14:59:40 -07:00
Ismael Juma d7ce35d61a
MINOR: Update zstd to 1.4.5 (#8766)
It improves decompression speed:

>For x64 cpus, expect a speed bump of at least +5%, and up to +10% in favorable cases.
>ARM cpus receive more benefit, with speed improvements ranging from +15% vicinity,
>and up to +50% for certain SoCs and scenarios (ARM‘s situation is more complex due
>to larger differences in SoC designs).

See https://github.com/facebook/zstd/releases/tag/v1.4.5 for more details.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2020-05-31 14:58:30 -07:00
Chia-Ping Tsai d4c1ef4a10
MINOR: Align the constructor of KafkaConsumer to KafkaProducer (#8605)
1. Move KafkaProducer#propsToMap to Utils#propsToMap
2. Apply Utils#propsToMap to constructor of KafkaConsumer

Reviewers: Noa Resare <noa@resare.com>, Ismael Juma <ismael@juma.me.uk>
2020-05-31 14:36:27 -07:00