Commit Graph

9299 Commits

Author SHA1 Message Date
Ryan Dielhenn 04fd555475
MINOR: Enable KRaft in transactions_test.py #11121
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-07-23 16:01:54 -07:00
Jason Gustafson b69027726e
MINOR: Remove redundant fields in dump log record output (#11101)
In 2.8, the dump log output regressed to print batch level information for each record, which makes the output much noisier. This patch changes the output to what it was in 2.7 and previous versions. We only print batch metadata at the batch level.

Reviewers: David Arthur <mumrah@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2021-07-23 15:56:41 -07:00
Jason Gustafson a75997f279
KAFKA-13127; Fix stray topic partition deletion for kraft (#11118)
This patch fixes BrokerMetadataPublisher.findGhostReplicas (renamed to findStrayPartitions)
so that it returns the stray partitions. Previously it was returning the non-stray partitions. This
caused all of these partitions to get deleted on startup by mistake.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, José Armando García Sancio <jsancio@gmail.com>
2021-07-23 15:01:39 -07:00
A. Sophie Blee-Goldman 03edcdd972
KAFKA-13128: wait for all keys to be fully processed in #shouldQueryStoresAfterAddingAndRemovingStreamThread (#11113)
This test is flaky due to waiting on all records to be processed for only a single key before issuing IQ lookups and asserting whether data was found.

Reviewers:  Phil Hardwick, Walker Carlson <wcarlson@confluent.io>
2021-07-23 14:56:46 -07:00
Xavier Léauté 3ba688e75f
KAFKA-9203: Check for buggy LZ4 libraries and remove corresponding workarounds (#10196)
* Remove the workarounds that were added back in https://github.com/apache/kafka/pull/7769
* Add a check to detect buggy LZ4 library versions

This check allows us to safely remove the workarounds for buggy
LZ4 versions without users encountering cryptic errors if they
accidentally have an older LZ4 library on the classpath, as
described in KAFKA-9203.

With this change the use will get a clear error message indicating
what the problem might be if they encounter this situation.

Note: This now instantiates a compressor in the decompression code.
This should be safe with respect to JNI libraries, since we always use
`LZ4Factory.fastestInstance()` which takes care of falling back to a pure
Java implementation if JNI libraries are not present.

This was tested with lz4 1.3.0 to make sure it triggers the exception when running
`KafkaLZ4Test`.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>

Co-authored-by: Ismael Juma <ismael@juma.me.uk>
2021-07-23 14:32:41 -07:00
Ismael Juma f34bb28ab6
KAFKA-13116: Fix message_format_change_test and compatibility_test_new_broker_test failures (#11108)
These failures were caused by a46b82bea9. Details for each test:

* message_format_change_test: use IBP 2.8 so that we can write in older message
formats.
* compatibility_test_new_broker_test_failures: fix down-conversion path to handle
empty record batches correctly. The record scan in the old code ensured that
empty record batches were never down-converted, which hid this bug.
* upgrade_test: set the IBP 2.8 when message format is < 0.11 to ensure we are
actually writing with the old message format even though the test was passing
without the change.

Verified with ducker that some variants of these tests failed without these changes
and passed with them. Also added a unit test for the down-conversion bug fix.

Reviewers: Jason Gustafson <jason@confluent.io>
2021-07-23 13:43:31 -07:00
Ryan Dielhenn 273d66479d
MINOR: Add kraft controller listener to AlterConfigsRequest #11119
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-07-23 13:26:54 -07:00
Justine Olshan d998cbcc88
KAFKA-8529; Flakey test ConsumerBounceTest#testCloseDuringRebalance (#11097)
When the replica fetcher receives a top-level error in the fetch response, it marks all partitions are failed and adds a backoff delay before resuming fetching from them. In addition to this, there is an additional backoff enforced after the top-level error is handled, so we end up waiting twice the configured backoff time before resuming. This patch removes this extra backoff.

Reviewers: Jason Gustafson <jason@confluent.io>
2021-07-23 09:30:55 -07:00
Luke Chen f959e6c583
KAFKA-13129: replace describe topic via zk with describe users (#11115)
Replace the unsupported describe topic via zk with describe users to fix the system tests.
For the upgrade_test case where TLS support is not required, use list_acls instead.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-07-23 05:33:43 -07:00
Guozhang Wang c38adaf74b
MINOR: only request rejoin and log if necessary for metadata snapshot and subscription checks (#11112)
Since now we call do not necessarily complete the rebalance within a poll call, we may keep checking the rejoinNeededOrPending which hits either of the conditions and returns true, but then returns early, resulting in flooding log entries. This PR would only log/set the flag when it was not set yet, effectively only logging for the first time.

Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>
2021-07-22 22:46:46 -07:00
A. Sophie Blee-Goldman d99562a145
HOTFIX: Set session interval back to 10s for StreamsCooperativeRebalanceUpgradeTest (#11103)
This test is hitting pretty frequent timeouts after bouncing a node and waiting for it to come back and fully rejoin the group. It seems to now take 45s for the initial JoinGroup to succeed, which I suspect is due to the new default session.interval.ms (which was recently changed to 45s). Let's try fixing this config to the old value of 10s and see if that helps it rejoin in time.

Reviewer: Bruno Cadonna <cadonna@apache.org>
2021-07-22 19:27:07 +02:00
David Jacot 2dbd65ecaf
MINOR: Small refactorings in admin group handlers (#11079)
Small refactoring to make the code uniform across the newly introduced admin group handlers.

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
2021-07-22 10:28:53 +02:00
Niket 57866bd588
MINOR: Rename the @metadata topic to __cluster_metadata #11102
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-07-21 17:30:35 -07:00
Phil Hardwick 02dc615c1e
KAFKA-13096: Ensure queryable store providers is up to date after adding stream thread (#10921)
When a new thread is added the queryable store providers continues to use the store providers it was given when KafkaStreams was instantiated. This means IQ will start performing lookups against an out-of-date list of threads, and may eventually become completely broken. We must make sure the QueryableStoreProvider is updated when threads are added and removed.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2021-07-21 13:23:15 -07:00
leah 89286668eb
MINOR: add serde configs to properly set serdes in failing StreamsStaticMembershipTest (#11093)
After changing the default serde to be null, some system tests started failing. This test didn't explicitly pass in a serde and didn't set the default config so when the test was trying to setup the source node it wasn't able to find any config to use and threw a config exception.

 Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Bruno Cadonna <cadonna@confluent.io>
2021-07-21 12:54:59 -07:00
Niket 6dd425e276
MINOR: Validate the KRaft controllerListener config on startup (#11070)
Reviewers: Colin P. McCabe <cmccabe@apache.org>, David Arthur <mumrah@gmail.com>
2021-07-21 10:41:41 -07:00
Ismael Juma e0f8eda6fd
MINOR: Improve usage of LogManager.currentDefaultConfig (#11094)
In `deleteLogs`, we use a consistent value for `fileDeleteDelayMs`
for the whole method. In `DynamicBrokerConfig.reconfigure`, it's
a minor readability improvement, but there should be no change in
behavior.

Reviewers: David Arthur <mumrah@gmail.com>
2021-07-21 06:12:03 -07:00
Luke Chen ad59e3b622
MINOR: update doc to reflect the grace period change (#11100)
We removed default 24 hours grace period in KIP-633, and deprecate some grace methods, but we forgot to update the stream docs.

Reviewer: Bruno Cadonna <cadonna@apache.org>
2021-07-21 13:26:41 +02:00
Bruno Cadonna 9b3687e0ac
HOTFIX: Modify system test config to reduce time to stable task assignment. (#11090)
Currently, we verify the startup of a Streams client by checking the transition
from REBALANCING to RUNNING and if the client processed some records
in the EOS system test. However, if the Streams client only
has standby tasks assigned as it can happen if the client is catching 
up by using warm-up replicas, the client will never process
records within the timeout of the startup verification. Hence, the test 
will fail although everything is fine. This commit fixes this by reducing
the time to the next probing rebalance and by increasing the number of 
max warm-up replicas. In such a way, the catch up of the client and the 
following processing of records should still be within the startup verification 
timeout of the client.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2021-07-21 07:58:14 +02:00
Ryan Dielhenn 56ef910358
KAFKA-13104: Controller should notify raft client when it resigns #11082
When the active controller encounters an event exception it attempts to renounce leadership.
Unfortunately, this doesn't tell the RaftClient that it should attempt to give up leadership. This
will result in inconsistent state with the RaftClient as leader but with the controller as
inactive.  This PR changes the implementation so that the active controller asks the RaftClient
to resign.

Reviewers: Jose Sancio <jsancio@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2021-07-20 16:41:20 -07:00
Ron Dagostino d779ab08ae
MINOR: Test ReplicaManager Metric Names (#11066)
This patch closes a test gap where we do not check ReplicaManager metrics remain as expected. There
was a bug in 2.8 where the metrics moved under a different class name for the KRaft case. Having
such tests would have helped identify the bug.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-07-20 16:37:27 -07:00
Ron Dagostino 1e78dcda69
MINOR: Fix ZooKeeperAuthorizerTest for KRaft (#11095)
This patch fixes the ZooKeeperAuthorizerTest for KRaft. The system test was not
configuring/reconfiguring/restarting the remote controller quorum with the correct security settings.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-07-20 16:35:14 -07:00
Michael Carter 0314801a8e
KAFKA-9887 fix failed task or connector count on startup failure (#8844)
Moved the responsibility for recording task and connector startup and failure metrics from the invocation code
into the status listener. The reason behind this is that the WorkerTasks (and subclasses) were either not propagating exceptions upwards, or were unable to do so easily because they were running on completely different threads.

Also split out WorkerMetricsGroup from being an inner class into being a standard class. This was to make sure
the Data Abstraction Count checkStyle rule was not violated.

Author: Michael Carter <michael.carter@instaclustr.com>
Reviewers: Chris Egerton <chrise@confluent.io>, Randall Hauch <rhauch@gmail.com>
2021-07-20 17:39:26 -05:00
Walker Carlson eeb788d1b9
KAFKA-13010: Retry getting tasks incase of rebalance for TaskMetadata tests (#11083)
If there is a cooperative rebalance the tasks might not be assigned to a thread at all for a very short timeframe, causing this test to fail. We can just retry getting the metadata until the group has finished rebalancing and all tasks are assigned

Reviewers: Bruno Cadonna <cadonna@apache.org>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Josep Prat <josep.prat@aiven.io>
2021-07-20 14:56:07 -07:00
CHUN-HAO TANG 47a0974f5a
KAFKA-13082: Replace EasyMock with Mockito for ProcessorContextTest (#11045)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2021-07-20 11:28:11 -07:00
David Arthur 030caec096
MINOR: Handle some null cases in BrokerMetadataPublisher (#11029)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
2021-07-20 14:10:14 -04:00
José Armando García Sancio 69a4661d7a
KAFKA-13100: Create KRaft controller snapshot during promotion (#11084)
The leader assumes that there is always an in-memory snapshot at the last
committed offset. This means that the controller needs to generate an in-memory
snapshot when getting promoted from inactive to active.  This PR adds that
code. This fixes a bug where sometimes we would try to look for that in-memory
snapshot and not find it.

The controller always starts inactive, and there is no requirement that there
exists an in-memory snapshot at the last committed offset when the controller
is inactive. Therefore we can remove the initial snapshot at offset -1.

We should also optimize when a snapshot is cancelled or completes, by deleting
all in-memory snapshots less that the last committed offset.

SnapshotRegistry's createSnapshot should allow the creating of a snapshot if
the last snapshot's offset is the given offset. This allows for simpler client
code. Finally, this PR renames createSnapshot to getOrCreateSnapshot.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-07-20 10:13:01 -07:00
Ismael Juma b0c243f032
MINOR: Fix `testResolveDnsLookup` by using a mocked dns resolver (#11091)
This focuses on the currently failing test, #9315 is a more complete fix
that we should also review and merge.

Reviewers: David Jacot <djacot@confluent.io>
2021-07-20 07:40:33 -07:00
Luke Chen b6159042c0
KAFKA-12598: ConfigCommand should only support communication via ZooKeeper for a reduced set of cases (#10811)
Checked the documentation, we must use `--zookeeper` option in 3 places (alter and describe):
1. user configs where the config is a SCRAM mechanism name (i.e. a SCRAM credential for a user)
2. update broker configs for a particular broker when that broker is down
3. broker default configs when all brokers are down

Reference:
1. [config SCRAM Credentials](https://kafka.apache.org/documentation/#security_sasl_scram_credentials)
2. [Update config before broker started](https://kafka.apache.org/documentation/#dynamicbrokerconfigs)

So, after this PR, we only support `--zookeeper` on `users` and `brokers` entity. Add some argument parse rules and tests. 

Reviewers: Ron Dagostino <rdagostino@confluent.io>, Ismael Juma <ismael@juma.me.uk>
2021-07-19 18:53:14 -07:00
Colin Patrick McCabe 4423a54935
MINOR: log static broker configs in KRaft mode (#11067)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-07-19 12:34:42 -07:00
Colin Patrick McCabe bfc57aa4dd
MINOR: enable reassign_partitions_test.py for kraft (#11064)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-07-19 09:08:55 -07:00
Satish Duggana e8ce93bd53
KAFKA-9555 Added default RLMM implementation based on internal topic storage. (#10579)
KAFKA-9555 Added default RLMM implementation based on internal topic storage.

This is the initial version of the default RLMM implementation.
This includes changes containing default RLMM configs, RLMM implementation, producer/consumer managers.
Introduced TopicBasedRemoteLogMetadataManagerHarness which takes care of bringing up a Kafka cluster and create remote log metadata topic and initializes TopicBasedRemoteLogMetadataManager.
Refactored existing RemoteLogMetadataCacheTest to RemoteLogSegmentLifecycleTest to have parameterized tests to run both RemoteLogMetadataCache and also TopicBasedRemoteLogMetadataManager.
Refactored existing InmemoryRemoteLogMetadataManagerTest, RemoteLogMetadataManagerTest to have parameterized tests to run both InmemoryRemoteLogMetadataManager and also TopicBasedRemoteLogMetadataManager.

This is part of tiered storage KIP-405 efforts.

Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Cong Ding <cong@ccding.com>, Jun Rao <junrao@gmail.com>
2021-07-19 09:05:46 -07:00
David Jacot f7790a0675
KAFKA-13058; AlterConsumerGroupOffsetsHandler does not handle partition errors correctly. (#11016)
This patch updates `AlterConsumerGroupOffsetsHandler` to handle partition errors correctly. The issue is that any partition error fails the entire future instead of being passed as an error for its corresponding partition. 

Reviewers: Luke Chen <showuon@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
2021-07-19 16:48:03 +02:00
Ismael Juma a46b82bea9
KAFKA-12944: Assume message format version is 3.0 when inter-broker protocol is 3.0 or higher (KIP-724) (#11036)
Also:
* Deprecate `log.message.format.version` and `message.format.version`.
* Log broker warning if the deprecated config values are ignored due to
the inter-broker protocol version.
* Log warning if `message.format.version` is set via `ConfigCommand`.
* Always down-convert if fetch version is v3 or lower.
* Add tests to verify new message format version based on the
inter-broker protocol version.
* Adjust existing tests that create topics with an older message format to
have the inter-broker protocol set to 2.8.
* Add upgrade note.

Note that the log compaction change to always write new segments with
record format v2 if the IBP is 3.0 or higher will be done as part of
KAFKA-13093 (with Kafka 3.1 as the target release version).

Reviewers: David Jacot <djacot@confluent.io>, David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>
2021-07-19 05:37:16 -07:00
Luke Chen 597dd1359d
MINOR: improve the partitioner.class doc (#10987)
Reviewers: Josep Prat <josep.prat@aiven.io>, Justine Olshan <jolshan@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2021-07-19 13:55:18 +08:00
CHUN-HAO TANG 98bd590718
MINOR: Replace unused variable with underscore (#11037)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-07-17 16:36:52 +08:00
Chia-Ping Tsai a24dbba1e5
MINOR: make sure alterAclsPurgatory is closed when controller server … (#10868)
Reviewers:  Ismael Juma <ismael@juma.me.uk>
2021-07-17 14:52:37 +08:00
Ron Dagostino 762d11c13f
MINOR: ducktape should start brokers in parallel and support co-located kraft
This patch adds a sanity-check bounce system test for the case where we have 3
co-located KRaft controllers and fixes the system test code so that this case
will pass by starting brokers in parallel by default instead of serially. We
now also send SIGKILL to any running KRaft broker or controller nodes for the
co-located case when a majority of co-located controllers have been stopped --
otherwise they do not shutdown, and we spin for the 60 second timeout. Finally,
this patch adds the ability to specify that certain brokers should not be
started when starting the cluster, and then we can start those nodes at a later
time via the add_broker() method call; this is going to be helpful for KRaft
snapshot system testing.

We were not testing the 3 co-located KRaft controller case previously, and it
would not pass because the first Kafka node would never be considered started.
We were starting the Kafka nodes serially, and we decide that a node has
successfully started when it logs a particular message. This message is not
logged until the broker has identified the controller (i.e. the leader of the
KRaft quorum). There cannot be a leader until a majority of the KRaft quorum
has started, so with 3 co-located controllers the first node could never be
considered "started" by the system test.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-07-16 16:28:09 -07:00
José Armando García Sancio b5cb02b288 KAFKA-13090: Improve kraft snapshot integration test
Check and verify generated snapshots for the controllers and the
brokers. Assert reader state when reading last log append time.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-07-16 14:10:52 -07:00
Matthias J. Sax 3e38038278
HOTFIX: Init stream-stream left/outer join emit interval correctly (#11055)
Follow up to #10917

The fix from #10917 intended to reduce the emit frequency to save the creation cost of RocksDB iterators. However, we incorrectly initialized the "timer" with timestamp zero, and thus, the timer was always in the past and we did try to emit left/outer join result too often.

This PR fixes the initialization of the emit interval timer to current wall-clock time to effectively 'enable' the fix from #10917.

Reviewers: Sergio Peña <sergio@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2021-07-16 13:30:19 -07:00
David Arthur 061517212f
KAFKA-12777: Fix a potential NPE in AutoTopicCreationManager
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-07-16 13:17:32 -07:00
José Armando García Sancio 8e48212343
KAFKA-13098: Fix NoSuchFileException during snapshot recovery (#11071)
Fix a bug where if a snapshot file is deleted while we're running snapshot recovery,
a NoSuchFileException will be thrown and snapshot recovery will fail.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-07-16 12:56:15 -07:00
Guozhang Wang 13b2df733a
MINOR: Default GRACE with Old API should set as 24H minus window-size / inactivity-gap (#10953)
In 2.8 and before, we computed the default grace period with Math.max(maintainDurationMs - sizeMs, 0); in method gracePeriodMs() in TimeWindows, SessionWindows, and JoinWindows. That means that the default grace period has never been 24 hours but 24 hours - window size. Since gracePeriodMs() is used to compute the retention time of the changelog topic for the corresponding window state store and the segments for the window state store it is important to keep the same computation for the deprecated methods. Otherwise, Streams app that run with 2.8 and before might not be compatible with Streams 3.0 because the retention time of the changelog topics created with older Streams apps will be smaller than the assumed retention time for Streams apps in 3.0. For example, with a window size of 10 hours, an old Streams app would have created a changelog topic with retention time 10 hours (window size) + 14 hours (default grace period, 24 hours - 10 hours). A 3.0 Streams app would assume a retention time of 10 hours (window size) + 24 hours (deprecated default grace period as currently specified on trunk). In the presence of failures, where a state store needs to recreated, records might get lost, because before the failure the state store of a 3.0 Streams app contained 10 hours + 24 hours of records whereas the changelog topic that was created with the old Streams app would only contain 10 hours + 14 hours of records.

All this happened due to us always stating that the default grace period was 24 hours although it was not completely correct and a connected and unfortunate misunderstanding when we removed deprecated windows APIs (#10378).

Co-authors: Bruno Cadonna <cadonna@apache.org>
Reviewers: Luke Chen <showuon@gmail.com>, Matthias J. Sax <mjsax@apache.org>, Bruno Cadonna <cadonna@apache.org>
2021-07-16 11:22:26 -07:00
Chris Egerton e01c8a1cae
KAFKA-13017: Remove excessive logging for sink task deserialization errors (#10945)
[Jira](https://issues.apache.org/jira/browse/KAFKA-13017)

Reverts https://github.com/apache/kafka/pull/7496, which added `ERROR`-level logging for deserialization errors in sink tasks even when connectors had logging for these errors disabled.

No information is lost by this change that cannot be retained in an opt-in fashion by setting `errors.log.enable` and `errors.log.include.messages` to `true` in a connector config.

Reviewers: Arjun Satish <arjun@confluent.io>, Tom Bentley <tbentley@redhat.com>
2021-07-16 15:40:17 +01:00
Luke Chen 4fd6d2bec8
KAFKA-13064: Make ListConsumerGroupOffsetsHandler unmap for COORDINATOR_NOT_AVAILABLE error (#11026)
This patch improve the error handling in `ListConsumerGroupOffsetsHandler` and ensures that `COORDINATOR_NOT_AVAILABLE` is unmapped in order to look up the coordinator again.

Reviewers: David Jacot <djacot@confluent.io>
2021-07-16 09:58:57 +02:00
Ismael Juma fd66db4793
KAFKA-12930,KAFKA-12929: Deprecate Java 8 and Scala 2.12 (#11059)
Update the readme to note the deprecation. We will also mention the deprecation
in the downloads page when the release is done.

Reviewers: Konstantine Karantasis <konstantine@confluent.io>, David Jacot <djacot@confluent.io>, José Armando García Sancio <jsancio@users.noreply.github.com>
2021-07-15 15:29:18 -07:00
Justine Olshan 584213ed20
Fix perf regression on LISR requests by asynchronously flushing the partition.metadata file (#11056)
After noticing increased LISR times, we discovered a lot of time was spent synchronously flushing the partition metadata file. This PR changes the code so we asynchronously flush the files.

We ensure files are flushed before appending, renaming or closing the log to ensure we have the partition metadata information on disk. Three new tests have been added to address these cases.

Reviewers:  Lucas Bradstreet <lucas@confluent.io>, Jun Rao <junrao@gmail.com>
2021-07-15 14:00:32 -07:00
Colin Patrick McCabe e07de97a4c
KAFKA-12803: Support reassigning partitions when in KRaft mode (#10753)
Support the KIP-455 reassignment API when in KRaft mode. Reassignments
which merely rearrange partitions complete immediately. Those that only
remove a partition complete immediately if the ISR would be non-empty
after the specified removals. Reassignments that add one or more
partitions follow the KIP-455 pattern of adding all the adding replicas
to the replica set, and then waiting for the ISR to include all the new
partitions before completing. Changes to the partition sets are
accomplished via PartitionChangeRecord.

Reviewers: Jun Rao <junrao@gmail.com>
2021-07-15 11:41:51 -07:00
Guozhang Wang 3e3264760b
KAFKA-10847: Remove internal config for enabling the fix (#10941)
Also update the upgrade guide indicating about the grace period KIP and its indication on the fix with throughput impact.

Reviewers: Luke Chen <showuon@gmail.com>, Matthias J. Sax <mjsax@apache.org>
2021-07-15 10:58:15 -07:00
Luke Chen 921a3428a8
KAFKA-13072: Make RemoveMembersFromConsumerGroupHandler unmap for COORDINATOR_NOT_AVAILABLE error (#11035)
This patch improve the error handling in `RemoveMembersFromConsumerGroupHandler` and ensures that `COORDINATOR_NOT_AVAILABLE` is unmapped in order to look up the coordinator again.

Reviewers: David Jacot <djacot@confluent.io>
2021-07-15 18:03:46 +02:00