Commit Graph

224 Commits

Author SHA1 Message Date
Matthias J. Sax b351493543
KAFKA-9274: Remove `retries` for global task (#9047)
- part of KIP-572
 - removed the usage of `retries` in `GlobalStateManger`
 - instead of retries the new `task.timeout.ms` config is used

Reviewers: John Roesler <john@confluent.io>, Boyang Chen <boyang@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2020-08-05 14:14:18 -07:00
John Roesler 26a217c8e7
MINOR: Streams integration tests should not call exit (#9067)
- replace System.exit with Exit.exit in all relevant classes
- forbid use of System.exit in all relevant classes and add exceptions for others

Co-authored-by: John Roesler <vvcephei@apache.org>
Co-authored-by: Matthias J. Sax <matthias@confluent.io>

Reviewers: Lucas Bradstreet <lucas@confluent.io>, Ismael Juma <ismael@confluent.io>
2020-08-05 13:52:50 -07:00
David Arthur 4cd2396db3
KAFKA-9629 Use generated protocol for Fetch API (#9008)
Refactored FetchRequest and FetchResponse to use the generated message classes for serialization and deserialization. This allows us to bypass unnecessary Struct conversion in a few places. A new "records" type was added to the message protocol which uses BaseRecords as the field type. When sending, we can set a FileRecords instance on the message, and when receiving the message class will use MemoryRecords. 

Also included a few JMH benchmarks which indicate a small performance improvement for requests with high partition counts or small record sizes.

Reviewers: Jason Gustafson <jason@confluent.io>, Boyang Chen <boyang@confluent.io>, David Jacot <djacot@confluent.io>, Lucas Bradstreet <lucas@confluent.io>, Ismael Juma <ismael@juma.me.uk>, Colin P. McCabe <cmccabe@apache.org>
2020-07-30 13:29:39 -04:00
Mickael Maison caa806cd82
KAFKA-10232: MirrorMaker2 internal topics Formatters KIP-597 (#8604)
This PR includes 3 MessageFormatters for MirrorMaker2 internal topics:
- HeartbeatFormatter
- CheckpointFormatter
- OffsetSyncFormatter

This also introduces a new public interface org.apache.kafka.common.MessageFormatter that users can implement to build custom formatters.

Reviewers: Konstantine Karantasis <k.karantasis@gmail.com>, Ryanne Dolan <ryannedolan@gmail.com>, David Jacot <djacot@confluent.io>

Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
Co-authored-by: Edoardo Comar <ecomar@uk.ibm.com>
2020-07-03 10:41:45 +01:00
A. Sophie Blee-Goldman 42aa0f38b9
MINOR: clean up unused checkstyle suppressions for Streams (#8861)
Reviewers: John Roesler <john@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2020-06-17 17:04:43 -07:00
A. Sophie Blee-Goldman 03ed08d0d1
KAFKA-10144: clean up corrupted standby tasks before attempting a commit (#8849)
We need to make sure that corrupted standby tasks are actually cleaned up upon a TaskCorruptedException. However due to the commit prior to invoking handleCorruption, it's possible to throw a TaskMigratedException before actually cleaning up any of the corrupted tasks.

This is fine for active tasks since handleLostAll will finish up the job, but it does nothing with standby tasks. We should make sure that standby tasks are handled before attempting to commit (which we can do, since we don't need to commit anything for the corrupted standbys)

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2020-06-12 16:21:57 -07:00
Adam Bellemare bcf45b09d3
KAFKA-10049: Fixed FKJ bug where wrapped serdes are set incorrectly when using default StreamsConfig serdes (#8764)
Bug Details:
Mistakenly setting the value serde to the key serde for an internal wrapped serde in the FKJ workflow.

Testing:
Modified the existing test to reproduce the issue, then verified that the test passes.

Reviewers: Guozhang Wang <wangguoz@gmail.com>, John Roesler <vvcephei@apache.org>
2020-06-12 10:00:38 -05:00
Kowshik Prakasam 4f96c5b424
KAFKA-10027: Implement read path for feature versioning system (KIP-584) (#8680)
In this PR, I have implemented various classes and integration for the read path of the feature versioning system (KIP-584). The ultimate plan is that the cluster-wide finalized features information is going to be stored in ZK under the node /feature. The read path implemented in this PR is centered around reading this finalized features information from ZK, and, processing it inside the Broker.

Here is a summary of what's in this PR (a lot of it is new classes):

A facility is provided in the broker to declare its supported features, and advertise its supported features via its own BrokerIdZNode under a features key.
A facility is provided in the broker to listen to and propagate cluster-wide finalized feature changes from ZK.
When new finalized features are read from ZK, feature incompatibilities are detected by comparing against the broker's own supported features.
ApiVersionsResponse is now served containing supported and finalized feature information (using the newly added tagged fields).

Reviewers: Boyang Chen <boyang@confluent.io>, Jun Rao <junrao@gmail.com>
2020-06-11 11:28:57 -07:00
Konstantine Karantasis 09b22e7e67
KAFKA-9848: Avoid triggering scheduled rebalance delay when task assignment fails but Connect workers remain in the group (#8805)
In the first version of the incremental cooperative protocol, in the presence of a failed sync request by the leader, the assignor was designed to treat the unapplied assignments as lost and trigger a rebalance delay. 

This commit applies optimizations in these cases to avoid the unnecessary activation of the rebalancing delay. First, if the worker that loses the sync group request or response is the leader, then it detects this failure by checking the what is the expected generation when it performs task assignments. If it's not the expected one, it resets its view of the previous assignment because it wasn't successfully applied and it doesn't represent a correct state. Furthermore, if the worker that has missed the assignment sync is an ordinary worker, then the leader is able to detect that there are lost assignments and instead of triggering a rebalance delay among the same members of the group, it treats the lost tasks as new tasks and reassigns them immediately. If the lost assignment included revocations that were not applied, the leader reapplies these revocations again. 

Existing unit tests and integration tests are adapted to test the proposed optimizations. 

Reviewers: Randall Hauch <rhauch@gmail.com>
2020-06-09 09:41:11 -07:00
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
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
Mickael Maison fe948d39e5
KAFKA-9130; KIP-518 Allow listing consumer groups per state (#8238)
Implementation of KIP-518: https://cwiki.apache.org/confluence/display/KAFKA/KIP-518%3A+Allow+listing+consumer+groups+per+state. 

Reviewers: David Jacot <djacot@confluent.io>, Jason Gustafson <jason@confluent.io>

Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
Co-authored-by: Edoardo Comar <ecomar@uk.ibm.com>
2020-05-29 11:25:20 -07:00
A. Sophie Blee-Goldman 9d52deca24
KAFKA-9501: convert between active and standby without closing stores (#8248)
This PR has gone through several significant transitions of its own, but here's the latest:

* TaskManager just collects the tasks to transition and refers to the active/standby task creator to handle closing & recycling the old task and creating the new one. If we ever hit an exception during the close, we bail and close all the remaining tasks as dirty.

* The task creators tell the task to "close but recycle state". If this is successful, it tells the recycled processor context and state manager that they should transition to the new type.

* During "close and recycle" the task just does a normal clean close, but instead of closing the state manager it informs it to recycle itself: maintain all of its store information (most importantly the current store offsets) but unregister the changelogs from the changelog reader

* The new task will (re-)register its changelogs during initialization, but skip re-registering any stores. It will still read the checkpoint file, but only use the written offsets if the store offsets are not already initialized from pre-transition

* To ensure we don't end up with manual compaction disabled for standbys, we have to call the state restore listener's onRestoreEnd for any active restoring stores that are switching to standbys

Reviewers: John Roesler <vvcephei@apache.org>, Guozhang Wang <wangguoz@gmail.com>
2020-05-29 10:48:03 -07:00
Aakash Shah 38c1e96d2c
KAFKA-9971: Error Reporting in Sink Connectors (KIP-610) (#8720)
Implementation for KIP-610: https://cwiki.apache.org/confluence/display/KAFKA/KIP-610%3A+Error+Reporting+in+Sink+Connectors based on which sink connectors can now report errors at the final stages of the stream that exports records to the sink system.
 
This PR adds the `ErrantRecordReporter` interface as well as its implementation - `WorkerErrantRecordReporter`. The `WorkerErrantRecordReporter` is created in `Worker` and brought up through `WorkerSinkTask` to `WorkerSinkTaskContext`. 

An integration test and unit tests have been added.

Reviewers: Lev Zemlyanov <lev@confluent.io>, Greg Harris <gregh@confluent.io>, Chris Egerton <chrise@confluent.io>, Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>
2020-05-27 23:49:57 -07:00
xiaodongdu 9c833f665f
KAFKA-9960: implement KIP-606 to add metadata context to MetricsReporter (#8691)
Implemented KIP-606 to add metadata context to MetricsReporter.

Author: Xiaodong Du <xdu@confluent.io>
Reviewers: David Arthur <mumrah@gmail.com>, Randall Hauch <rhauch@gmail.com>, Xavier Léauté <xavier@confluent.io>, Ryan Pridgeon <ryan.n.pridgeon@gmail.com>
2020-05-27 20:18:36 -05:00
A. Sophie Blee-Goldman 83c616f706
KAFKA-9983: KIP-613: add INFO level e2e latency metrics (#8697)
Add e2e latency metrics at the beginning and end of task topologies
as INFO-level processor-node-level metrics.

Implements: KIP-613
Reviewers: John Roesler <vvcephei@apache.org>, Andrew Choi <a24choi@edu.uwaterloo.ca>, Bruno Cadonna <cadonna@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2020-05-27 15:55:29 -05:00
Konstantine Karantasis 371f14c3c1
KAFKA-5295: Allow source connectors to specify topic-specific settings for new topics (KIP-158) (#8722)
Kafka Connect workers have been able to create Connect's internal topics using the new admin client for some time now (see KAFKA-4667 for details). However, tasks of source connectors are still relying upon the broker to auto-create topics with default config settings if they don't exist, or expect these topics to exist before the connector is deployed, if their configuration needs to be specialized. 

With the implementation of KIP-158 here, if `topic.creation.enable=true`, Kafka Connect will supply the source tasks of connectors that are configured to create topics with an admin client that will allow them to create new topics on-the-fly before writing the first source records to a new topic. Additionally, each source connector has the opportunity to customize the topic-specific settings of these new topics by defining groups of topic configurations. 

This feature is tested here via unit tests (old tests that have been adjusted and new ones) as well as integration tests.

Reviewers: Randall Hauch <rhauch@gmail.com>
2020-05-26 22:07:34 -07:00
Jeff Huang 2988eac082
KAFKA-9944: Added supporting customized HTTP response headers for Kafka Connect. (#8620)
Added support for customizing the HTTP response headers for Kafka Connect as described in KIP-577.

Author: Jeff Huang <jeff.huang@confluent.io>
Reviewer: Randall Hauch <rhauch@gmail.com>
2020-05-24 08:56:27 -05:00
Randall Hauch 981ef5166d
KAFKA-9931: Implement KIP-605 to expand support for Connect worker internal topic configurations (#8654)
Added support for -1 replication factor and partitions for distributed worker internal topics by expanding the allowed values for the internal topics’ replication factor and partitions from positive values to also include -1 to signify that the broker defaults should be used.

The Kafka storage classes were already constructing a `NewTopic` object (always with a replication factor and partitions) and sending it to Kafka when required. This change will avoid setting the replication factor and/or number of partitions on this `NewTopic` if the worker configuration uses -1 for the corresponding configuration value.

Also added support for extra settings for internal topics on distributed config, status, and offset internal topics.

Quite a few new tests were added to verify that the `TopicAdmin` utility class is correctly using the AdminClient, and that the `DistributedConfig` validators for these configurations are correct. Also added integration tests for internal topic creation, covering preexisting functionality plus the new functionality.

Author: Randall Hauch <rhauch@gmail.com>
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
2020-05-23 09:00:32 -05:00
Matthias J. Sax 1daa8f638b
KAFKA-9748: Add Streams eos-beta integration test (#8496)
Reviewers: Boyang Chen <boyang@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2020-05-04 22:45:54 -07:00
A. Sophie Blee-Goldman b5de449377
KAFKA-9127: don't create StreamThreads for global-only topology (#8540)
Reviewers: Matthias J. Sax <matthias@confluent.io>, John Roesler <vvcephei@apache.org>
2020-04-28 22:34:17 -05:00
Colin Patrick McCabe bf6dffe93b
KAFKA-9309: Add the ability to translate Message classes to and from JSON (#7844)
Reviewers: David Arthur <mumrah@gmail.com>, Ron Dagostino <rdagostino@confluent.io>
2020-04-09 13:11:36 -07:00
Chia-Ping Tsai 833dc7725c
HOTFIX: exclude ConsumerCoordinator from NPathComplexity check (#8447)
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
2020-04-08 13:00:03 +01:00
Lucas Bradstreet 46540eb5e0
KAFKA-9820: validateMessagesAndAssignOffsetsCompressed allocates unused iterator (#8422)
3e9d1c1411 introduced skipKeyValueIterator(s) which were intended to be used, but in this case were created but were not used in offset validation.

A subset of the benchmark results follow. Looks like a 20% improvement in validation performance and a 40% reduction in garbage allocation for 1-2 batch sizes.

**# Parameters: (bufferSupplierStr = NO_CACHING, bytes = RANDOM, compressionType = LZ4, maxBatchSize = 1, messageSize = 1000, messageVersion = 2)**

Before:
Result "org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation":
  64851.837 ±(99.9%) 944.248 ops/s [Average]              
  (min, avg, max) = (64505.317, 64851.837, 65114.359), stdev = 245.218
  CI (99.9%): [63907.589, 65796.084] (assumes normal distribution)                                       
                                                             
"org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm":
  164088.003 ±(99.9%) 0.004 B/op [Average]                                                                                 
  (min, avg, max) = (164088.001, 164088.003, 164088.004), stdev = 0.001
  CI (99.9%): [164087.998, 164088.007] (assumes normal distribution)

After:

Result "org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation":                                      
  78910.273 ±(99.9%) 707.024 ops/s [Average]                                                                               
  (min, avg, max) = (78785.486, 78910.273, 79234.007), stdev = 183.612                                                     
  CI (99.9%): [78203.249, 79617.297] (assumes normal distribution)                                       

"org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm":                                                                                                                                   
  96440.002 ±(99.9%) 0.001 B/op [Average]                                                                                  
  (min, avg, max) = (96440.002, 96440.002, 96440.002), stdev = 0.001                                                       
  CI (99.9%): [96440.002, 96440.003] (assumes normal distribution)   

 **# Parameters: (bufferSupplierStr = NO_CACHING, bytes = RANDOM, compressionType = LZ4, maxBatchSize = 2, messageSize = 1000, messageVersion = 2)**

Before:
Result "org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation":                                      
  64815.364 ±(99.9%) 639.309 ops/s [Average]                                                                               
  (min, avg, max) = (64594.545, 64815.364, 64983.305), stdev = 166.026                                                                                                                                                                                
  CI (99.9%): [64176.056, 65454.673] (assumes normal distribution)                                                         
                                                                                                                                                                                        "org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm":        
  163944.003 ±(99.9%) 0.001 B/op [Average]                                                                                 
  (min, avg, max) = (163944.002, 163944.003, 163944.003), stdev = 0.001                                                    
  CI (99.9%): [163944.002, 163944.004] (assumes normal distribution)                                     

After:
Result "org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation":
  77075.096 ±(99.9%) 201.092 ops/s [Average]              
  (min, avg, max) = (77021.537, 77075.096, 77129.693), stdev = 52.223
  CI (99.9%): [76874.003, 77276.188] (assumes normal distribution)                                       
                                                             
"org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm":
  96504.002 ±(99.9%) 0.003 B/op [Average]                                                                                  
  (min, avg, max) = (96504.001, 96504.002, 96504.003), stdev = 0.001
  CI (99.9%): [96503.999, 96504.005] (assumes normal distribution)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2020-04-04 10:05:51 -07:00
A. Sophie Blee-Goldman 6e0d553350
MINOR: clean up Streams assignment classes and tests (#8406)
First set of cleanup pushed to followup PR after KIP-441 Pt. 5. Main changes are:

1. Moved `RankedClient` and the static `buildClientRankingsByTask` to a new file
2. Moved `Movement` and the static `getMovements` to a new file (also renamed to `TaskMovement`)
3. Consolidated the many common variables throughout the assignment tests to the new `AssignmentTestUtils` 
4. New utility to generate comparable/predictable UUIDs for tests, and removed the generic from `TaskAssignor` and all related classes

Reviewers: John Roesler <vvcephei@apache.org>, Andrew Choi <a24choi@edu.uwaterloo.ca>
2020-04-03 13:53:51 -05:00
A. Sophie Blee-Goldman 2322bc0a6f
KAFKA-6145: Pt. 5 Implement high availability assignment (#8337)
Adds a new TaskAssignor implementation, currently hidden behind an internal feature flag, that implements the high availability algorithm of KIP-441.

Reviewers: Bruno Cadonna <bruno@confluent.io>, John Roesler <vvcephei@apache.org>
2020-04-02 13:36:03 -05:00
Gardner Vickers 8cf781ef01
MINOR: Improve performance of checkpointHighWatermarks, patch 1/2 (#6741)
This PR works to improve high watermark checkpointing performance.

`ReplicaManager.checkpointHighWatermarks()` was found to be a major contributor to GC pressure, especially on Kafka clusters with high partition counts and low throughput.

Added a JMH benchmark for `checkpointHighWatermarks` which establishes a
performance baseline. The parameterized benchmark was run with 100, 1000 and
2000 topics. 

Modified `ReplicaManager.checkpointHighWatermarks()` to avoid extra copies and cached
the Log parent directory Sting to avoid frequent allocations when calculating
`File.getParent()`.

A few clean-ups:
* Changed all usages of Log.dir.getParent to Log.parentDir and Log.dir.getParentFile to
Log.parentDirFile.
* Only expose public accessor for `Log.dir` (consistent with `Log.parentDir`)
* Removed unused parameters in `Partition.makeLeader`, `Partition.makeFollower` and `Partition.createLogIfNotExists`.

Benchmark results:

| Topic Count | Ops/ms | MB/sec allocated |
|-------------|---------|------------------|
| 100               | + 51%    |  - 91% |
| 1000             | + 143% |  - 49% |
| 2000            | + 149% |   - 50% |

Reviewers: Lucas Bradstreet <lucas@confluent.io>. Ismael Juma <ismael@juma.me.uk>

Co-authored-by: Gardner Vickers <gardner@vickers.me>
Co-authored-by: Ismael Juma <ismael@juma.me.uk>
2020-03-25 20:53:42 -07:00
A. Sophie Blee-Goldman 6cf27c9c77
KAFKA-6145: Pt 2.5 Compute overall task lag per client (#8252)
Once we have encoded the offset sums per task for each client, we can compute the overall lag during assign by fetching the end offsets for all changelog and subtracting.

If the listOffsets request fails, we simply return a "completely sticky" assignment, ie all active tasks are given to previous owners regardless of balance.

Builds (but does not yet use) the statefulTasksToRankedCandidates map with the ranking:
Rank -1: active running task
Rank 0: standby or restoring task whose overall lag is within acceptableRecoveryLag
Rank 1: tasks whose lag is unknown (eg during version probing)
Rank 1+: all other tasks are ranked according to their actual total lag

Implements: KIP-441
Reviewers: Bruno Cadonna <bruno@confluent.io>, John Roesler <vvcephei@apache.org>
2020-03-21 13:40:34 -05:00
Colin Patrick McCabe 56051e7639
KAFKA-8820: kafka-reassign-partitions.sh should support the KIP-455 API (#8244)
Rewrite ReassignPartitionsCommand to use the KIP-455 API when possible, rather
than direct communication with ZooKeeper.  Direct ZK access is still supported,
but deprecated, as described in KIP-455.

As specified in KIP-455, the tool has several new flags.  --cancel stops
an assignment which is in progress.  --preserve-throttle causes the
--verify and --cancel commands to leave the throttles alone.
--additional allows users to execute another partition assignment even
if there is already one in progress.  Finally, --show displays all of
the current partition reassignments.

Reorganize the reassignment code and tests somewhat to rely more on unit
testing using the MockAdminClient and less on integration testing.  Each
integration test where we bring up a cluster seems to take about 5 seconds, so
it's good when we can get similar coverage from unit tests.  To enable this,
MockAdminClient now supports incrementalAlterConfigs, alterReplicaLogDirs,
describeReplicaLogDirs, and some other APIs.  MockAdminClient is also now
thread-safe, to match the real AdminClient implementation.

In DeleteTopicTest, use the KIP-455 API rather than invoking the reassignment
command.
2020-03-19 20:44:34 -07:00
Matthias J. Sax 89cd2f2a0b
KAFKA-9441: Unify committing within TaskManager (#8218)
- part of KIP-447
 - commit all tasks at once using non-eos (and eos-beta in follow up work)
 - unified commit logic into TaskManager
 - split existing methods of Task interface in pre/post parts

Reviewers: Boyang Chen <boyang@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2020-03-19 11:31:51 -07:00
Manikumar Reddy a0e1407820
KAFKA-9670; Reduce allocations in Metadata Response preparation (#8236)
This PR removes  intermediate  conversions between `MetadataResponse.TopicMetadata` => `MetadataResponseTopic` and `MetadataResponse.PartitionMetadata` => `MetadataResponsePartition` objects.

There is 15-20% reduction in object allocations and 5-10% improvement in metadata request performance.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson<jason@confluent.io>
2020-03-16 09:30:48 -07:00
Brian Byrne 227a7322b7
KIP-546: Implement describeClientQuotas and alterClientQuotas. (#8083)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2020-03-14 23:03:13 -07:00
jiao e3ccf20794 KAFKA-9685: Solve Set concatenation perf issue in AclAuthorizer
To dismiss the usage of operation ++ against Set which is slow when Set has many entries. This pr introduces a new class 'AclSets' which takes multiple Sets as parameters and do 'find' against them one by one. For more details about perf and benchmark, refer to [KAFKA-9685](https://issues.apache.org/jira/browse/KAFKA-9685)

Author: jiao <jiao.zhang@linecorp.com>

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

Closes #8261 from jiao-zhangS/jira-9685
2020-03-13 20:53:29 +05:30
John Roesler 78374a1549
KAFKA-9615: Clean up task/producer create and close (#8213)
* Consolidate task/producer management. Now, exactly one component manages
  the creation and destruction of Producers, whether they are per-thread or per-task.
* Add missing test coverage on TaskManagerTest

Reviewers: Guozhang Wang <wangguoz@gmail.com>, Boyang Chen <boyang@confluent.io>
2020-03-05 14:20:46 -06:00
Manikumar Reddy 8dff0b168a Kafka 9626: Improve ACLAuthorizer.acls() performance
This PR avoids creation of unnecessary sets in AclAuthorizer.acls() method implementation.

Perf results:
**Old**
```
Benchmark                                (aclCount)  (resourceCount)  Mode  Cnt    Score   Error  Units
AclAuthorizerBenchmark.testAclsIterator           5             5000  avgt   15    5.821 ? 0.309  ms/op
AclAuthorizerBenchmark.testAclsIterator           5            10000  avgt   15   15.303 ? 0.107  ms/op
AclAuthorizerBenchmark.testAclsIterator           5            50000  avgt   15   74.976 ? 0.543  ms/op
AclAuthorizerBenchmark.testAclsIterator          10             5000  avgt   15   15.366 ? 0.184  ms/op
AclAuthorizerBenchmark.testAclsIterator          10            10000  avgt   15   29.899 ? 0.129  ms/op
AclAuthorizerBenchmark.testAclsIterator          10            50000  avgt   15  167.301 ? 1.723  ms/op
AclAuthorizerBenchmark.testAclsIterator          15             5000  avgt   15   21.980 ? 0.114  ms/op
AclAuthorizerBenchmark.testAclsIterator          15            10000  avgt   15   44.385 ? 0.255  ms/op
AclAuthorizerBenchmark.testAclsIterator          15            50000  avgt   15  241.919 ? 3.955  ms/op
```
**New**

```
Benchmark                                (aclCount)  (resourceCount)  Mode  Cnt   Score   Error  Units
AclAuthorizerBenchmark.testAclsIterator           5             5000  avgt   15   0.666 ? 0.004  ms/op
AclAuthorizerBenchmark.testAclsIterator           5            10000  avgt   15   1.427 ? 0.015  ms/op
AclAuthorizerBenchmark.testAclsIterator           5            50000  avgt   15  21.410 ? 0.225  ms/op
AclAuthorizerBenchmark.testAclsIterator          10             5000  avgt   15   1.230 ? 0.018  ms/op
AclAuthorizerBenchmark.testAclsIterator          10            10000  avgt   15   4.303 ? 0.744  ms/op
AclAuthorizerBenchmark.testAclsIterator          10            50000  avgt   15  36.724 ? 0.409  ms/op
AclAuthorizerBenchmark.testAclsIterator          15             5000  avgt   15   2.433 ? 0.379  ms/op
AclAuthorizerBenchmark.testAclsIterator          15            10000  avgt   15   9.818 ? 0.214  ms/op
AclAuthorizerBenchmark.testAclsIterator          15            50000  avgt   15  52.886 ? 0.525  ms/op
```

Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Author: Lucas Bradstreet <lucas@confluent.io>

Reviewers: Ismael Juma <ismael@juma.me.uk>, Rajini Sivaram <rajinisivaram@googlemail.com>, Lucas Bradstreet <lucas@confluent.io>

Closes #8199 from omkreddy/KAFKA-9626
2020-03-03 01:51:09 +05:30
Bob Barrett 937f1f741c
KAFKA-8805; Bump producer epoch on recoverable errors (#7389)
This change is the client-side part of KIP-360. It identifies cases where it is safe to abort a transaction, bump the producer epoch, and allow the application to continue without closing the producer. In these cases, when KafkaProducer.abortTransaction() is called, the producer sends an InitProducerId following the transaction abort, which causes the producer epoch to be bumped. The application can then start a new transaction and continue processing.

For recoverable errors in the idempotent producer, the epoch is bumped locally. In-flight requests for partitions with an error are rewritten to reflect the new epoch, and in-flights of all other partitions are allowed to complete using the old epoch. 

Reviewers: Boyang Chen <boyang@confluent.io>, Jason Gustafson <jason@confluent.io>
2020-02-15 22:47:10 -08:00
Konstantine Karantasis 16ee326755
KAFKA-9556; Fix two issues with KIP-558 and expand testing coverage (#8085)
Correct the Connect worker logic to properly disable the new topic status (KIP-558) feature when `topic.tracking.enable=false`, and fix automatic topic status reset after a connector is deleted.

Also adds new `ConnectorTopicsIntegrationTest` and expanded unit tests.

Reviewers: Randall Hauch <rhauch@gmail.com>
2020-02-14 14:34:34 -08:00
Xavier Léauté 7e1c39f75a
KAFKA-9106 make metrics exposed via jmx configurable (#7674)
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Rajini Sivaram <rajinisivaram@googlemail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
2020-02-13 10:21:14 -08:00
Boyang Chen 07db26c20f
KAFKA-9417: New Integration Test for KIP-447 (#8000)
This change mainly have 2 components:

1. extend the existing transactions_test.py to also try out new sendTxnOffsets(groupMetadata) API to make sure we are not introducing any regression or compatibility issue
  a. We shrink the time window to 10 seconds for the txn timeout scheduler on broker so that we could trigger expiration earlier than later

2. create a completely new system test class called group_mode_transactions_test which is more complicated than the existing system test, as we are taking rebalance into consideration and using multiple partitions instead of one. For further breakdown:
  a. The message count was done on partition level, instead of global as we need to visualize 
the per partition order throughout the test. For this sake, we extend ConsoleConsumer to print out the data partition as well to help message copier interpret the per partition data.
  b. The progress count includes the time for completing the pending txn offset expiration
  c. More visibility and feature improvements on TransactionMessageCopier to better work under either standalone or group mode.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2020-02-12 12:34:12 -08:00
John Roesler 1681c78f60
KAFKA-9500: Fix FK Join Topology (#8015)
Corrects a flaw leading to an exception while building topologies that include both:

* A foreign-key join with the result not explicitly materialized
* An operation after the join that requires source materialization

Also corrects a flaw in TopologyTestDriver leading to output records being enqueued in the wrong order under some (presumably rare) circumstances.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2020-02-11 22:38:05 -06:00
Guozhang Wang 4090f9a2b0
KAFKA-9113: Clean up task management and state management (#7997)
This PR is collaborated by Guozhang Wang and John Roesler. It is a significant tech debt cleanup on task management and state management, and is broken down by several sub-tasks listed below:

Extract embedded clients (producer and consumer) into RecordCollector from StreamTask.
guozhangwang#2
guozhangwang#5

Consolidate the standby updating and active restoring logic into ChangelogReader and extract out of StreamThread.
guozhangwang#3
guozhangwang#4

Introduce Task state life cycle (created, restoring, running, suspended, closing), and refactor the task operations based on the current state.
guozhangwang#6
guozhangwang#7

Consolidate AssignedTasks into TaskManager and simplify the logic of changelog management and task management (since they are already moved in step 2) and 3)).
guozhangwang#8
guozhangwang#9

Also simplified the StreamThread logic a bit as the embedded clients / changelog restoration logic has been moved into step 1) and 2).
guozhangwang#10

Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Bruno Cadonna <bruno@confluent.io>, Boyang Chen <boyang@confluent.io>
2020-02-04 21:06:39 -08:00
Rajini Sivaram a565d1a182
KAFKA-9181; Maintain clean separation between local and group subscriptions in consumer's SubscriptionState (#7941)
Reviewers: Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2020-01-24 10:38:21 +00:00
Boyang Chen de90175fc2 KAFKA-9418; Add new sendOffsetsToTransaction API to KafkaProducer (#7952)
This patch adds a new API to the producer to implement transactional offset commit fencing through the group coordinator as proposed in KIP-447. This PR mainly changes on the Producer end for compatible paths to old `sendOffsetsToTxn(offsets, groupId)` vs new `sendOffsetsToTxn(offsets, groupMetadata)`.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
2020-01-22 13:48:36 -08:00
Mickael Maison 3953204d35 MINOR: Fix connect:mirror checkstyle (#7951)
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
2020-01-13 15:25:24 -08:00
John Roesler 717ce42a6d
KAFKA-9138: Add system test for relational joins (#7664)
Add a system test to verify the new foreign-key join introduced in KIP-213

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2019-12-11 09:48:23 -08:00
Colin Patrick McCabe 02df8e1496
KAFKA-8986: Allow null as a valid default for tagged fields. (#7585)
Allow null as a valid default for tagged fields.  Fix a bunch of cases where this would previously result in null pointer dereferences.

Also allow inferring FieldSpec#versions based on FieldSpec#taggedVersions.  Prefix 'key' with an underscore when it is used in the generated code, to avoid potential name collisions if someone names an RPC field "key".

Allow setting setting hexadecimal constants and 64-bit contstants.

Add a lot more test cases to SimpleExampleMessage.json.

Reviewers: Jason Gustafson <jason@confluent.io>
2019-11-20 16:40:18 -08:00
John Roesler 4a5155c934 KAFKA-8868: Generate SubscriptionInfo protocol message (#7248)
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>
2019-11-01 10:03:55 -07:00
Nikolay adb2bdb122 KAFKA-8584: The RPC code generator should support ByteBuffer. (#7342)
The RPC code generator should support using the ByteBuffer class in addition to byte arrays. By using the ByteBuffer class, we can avoid performing a copy in many situations. Also modify TestByteBufferDataTest to test the new feature.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Guozhang Wang <wangguoz@gmail.com>
2019-10-23 12:39:12 -07:00
John Roesler f93c473be1 KAFKA-9000: fix flaky FK join test by using TTD (#7517)
Migrate this integration test to use TopologyTestDriver instead of running 3 Streams instances.

Dropped one test that was attempting to produce specific interleavings. If anything, these should be verified deterministically by unit testing.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2019-10-16 22:40:57 -07:00
Greg Harris ff68b60429 KAFKA-8340, KAFKA-8819: Use PluginClassLoader while statically initializing plugins (#7315)
Added plugin isolation unit tests for various scenarios, with a `TestPlugins` class that compiles and builds multiple test plugins without them being on the classpath and verifies that the Plugins and DelegatingClassLoader behave properly. These initially failed for several cases, but now pass since the issues have been fixed.

KAFKA-8340 and KAFKA-8819 are closely related, and this fix corrects the problems reported in both issues.

Author: Greg Harris <gregh@confluent.io>
Reviewers: Chris Egerton <chrise@confluent.io>, Magesh Nandakumar <mageshn@confluent.io>, Konstantine Karantasis <konstantine@confluent.io>, Randall Hauch <rhauch@gmail.com>
2019-10-16 20:43:00 -05:00