Commit Graph

206 Commits

Author SHA1 Message Date
Federico Valeri 50e0e3c257
KAFKA-14582: Move JmxTool to tools (#13136)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2023-02-02 11:23:26 +01:00
Mickael Maison 8b44237655
KAFKA-14575: Move ClusterTool to tools module (#13080)
Reviewers: dengziming <dengziming1993@gmail.com>, Federico Valeri  <fedevaleri@gmail.com>
2023-01-22 12:50:43 +01:00
Luke Chen 2575362639
KAFKA-14498: reduce the startup nodes to avoid timeout error (#13016)
In MetadataQuorumCommandTest, we sometimes got the error:

java.util.concurrent.ExecutionException: java.lang.RuntimeException: Received a fatal error while waiting for the broker to catch up with the current cluster metadata.

Since we tried to bring up 3 broker + 3 controllers at the same time, and the config initial.broker.registration.timeout.ms (default 1 min) is sometimes not enough for them to start up. Checking the tests, it doesn't require so many nodes. Reducing the nodes number to make these tests reliable.

Reviewers: dengziming <dengziming1993@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2022-12-21 11:19:22 +08:00
Ismael Juma c0b28fde66
MINOR: Use INFO logging for tools and trogdor tests (#13006)
`TRACE` is too noisy and makes the build slower.

Reviewers: David Jacot <djacot@confluent.io>
2022-12-17 10:22:40 -08:00
Ismael Juma 88725669e7
MINOR: Move MetadataQuorumCommand from `core` to `tools` (#12951)
`core` should only be  used for legacy cli tools and tools that require
access to `core` classes instead of communicating via the kafka protocol
(typically by using the client classes).

Summary of changes:
1. Convert the command implementation and tests to Java and move it to
    the `tools` module.
2. Introduce mechanism to capture stdout and stderr from tests.
3. Change `kafka-metadata-quorum.sh` to point to the new command class.
4. Adjusted the test classpath of the `tools` module so that it supports tests
    that rely on the `@ClusterTests` annotation.
5. Improved error handling when an exception different from `TerseFailure` is
    thrown.
6. Changed `ToolsUtils` to avoid usage of arrays in favor of `List`.

Reviewers: dengziming <dengziming1993@gmail.com>
2022-12-09 09:22:58 -08:00
runom b8754c074a
KAFKA-14355: Fix integer overflow in ProducerPerformance (#12822)
Change types from int to long to avoid overflow

Reviewers: Luke Chen <showuon@gmail.com>,  Igor Soarez <soarez@apple.com>
2022-11-05 20:19:08 +08:00
Kirk True 8e43548175
KAFKA-13725: KIP-768 OAuth code mixes public and internal classes in same package (#12039)
* KAFKA-13725: KIP-768 OAuth code mixes public and internal classes in same package

Move classes into a sub-package of "internal" named "secured" that
matches the layout more closely of the "unsecured" package.

Replaces the concrete implementations in the former packages with
sub-classes of the new package layout and marks them as deprecated. If
anyone is already using the newer OAuth code, this should still work.

* Fix checkstyle and spotbugs violations

Co-authored-by: Kirk True <kirk@mustardgrain.com>

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2022-09-23 13:15:15 +05:30
dengziming 150fd5b0b1
KAFKA-13914: Add command line tool kafka-metadata-quorum.sh (#12469)
Add `MetadataQuorumCommand` to describe quorum status, I'm trying to use arg4j style command format, currently, we only support one sub-command which is "describe" and we can specify 2 arguments which are --status and --replication.

```
# describe quorum status
kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe --replication

ReplicaId	LogEndOffset	Lag	LastFetchTimeMs	LastCaughtUpTimeMs	Status  	
0        	10          	        0  	-1             	        -1                	                 Leader  	
1        	10          	        0  	-1             	        -1                	                 Follower	
2        	10          	        0  	-1             	        -1                	                 Follower	

kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe --status
ClusterId:                             fMCL8kv1SWm87L_Md-I2hg
LeaderId:                             3002
LeaderEpoch:                      2
HighWatermark:                  10
MaxFollowerLag:                 0
MaxFollowerLagTimeMs:   -1
CurrentVoters:                    [3000,3001,3002]
CurrentObservers:              [0,1,2]

# specify AdminClient properties
kafka-metadata-quorum.sh --bootstrap-server localhost:9092 --command-config config.properties describe --status
```

Reviewers: Jason Gustafson <jason@confluent.io>
2022-08-20 08:37:26 -07:00
Jason Gustafson f0a09ea003
MINOR: Fix event output inconsistencies in TransactionalMessageCopier (#12098)
This patch fixes some strangeness and inconsistency in the messages written by `TransactionalMessageCopier` to stdout. Here is a sample of two messages.

Progress message:
```
{"consumed":33000,"stage":"ProcessLoop","totalProcessed":33000,"progress":"copier-0","time":"2022/04/24 05:40:31:649","remaining":333}
```
The `transactionalId` is set to the value of the `progress` key.

And a shutdown message:
```
{"consumed":33333,"shutdown_complete":"copier-0","totalProcessed":33333,"time":"2022/04/24 05:40:31:937","remaining":0}
```
The `transactionalId` this time is set to the `shutdown_complete` key and there is no `stage` key.

In this patch, we change the following:

1. Use a separate key for the `transactionalId`.
2. Drop the `progress` and `shutdown_complete` keys.
3. Use `stage=ShutdownComplete` in the shutdown message.
4. Modify `transactional_message_copier.py` system test service accordingly.

Reviewers: David Arthur <mumrah@gmail.com>
2022-04-29 10:02:25 -07:00
彭小漪 6145974fef
KAFKA-13728: fix PushHttpMetricsReporter no longer pushes metrics when network failure is recovered. (#11879)
The class PushHttpMetricsReporter no longer pushes metrics when network failure is recovered.

I debugged the code and found the problem here: when we submit a task to the ScheduledThreadPoolExecutor that needs to be executed periodically, if the task throws an exception and is not swallowed, the task will no longer be scheduled to execute.

So when an IO exception occasionally occurs on the network, we should swallow it rather than throw it in task HttpReporter.

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2022-03-19 21:09:28 -07:00
Justine Olshan 7afdb069bf
KAFKA-13750; Client Compatability KafkaTest uses invalid idempotency configs (#11909)
Reviewers: Luke Chen <showuon@gmail.com>, David Jacot <djacot@confluent.io>
2022-03-17 18:00:27 +01:00
Kirk True ec29b62e92
KAFKA-13444: Fix OAuthCompatibilityTool help and add SSL options (#11486)
Reviewers: Jun Rao <junrao@gmail.com>
2021-11-15 15:45:18 -08:00
Jorge Esteban Quilcate Otoya 214b59b3ec
KAFKA-13429: ignore bin on new modules (#11415)
Reviewers: John Roesler <vvcephei@apache.org>
2021-11-10 14:36:24 -06:00
Kirk True 7b379539a5
KAFKA-13202: KIP-768: Extend SASL/OAUTHBEARER with Support for OIDC (#11284)
This task is to provide a concrete implementation of the interfaces defined in KIP-255 to allow Kafka to connect to an OAuth/OIDC identity provider for authentication and token retrieval. While KIP-255 provides an unsecured JWT example for development, this will fill in the gap and provide a production-grade implementation.

The OAuth/OIDC work will allow out-of-the-box configuration by any Apache Kafka users to connect to an external identity provider service (e.g. Okta, Auth0, Azure, etc.). The code will implement the standard OAuth client credentials grant type.

The proposed change is largely composed of a pair of AuthenticateCallbackHandler implementations: one to login on the client and one to validate on the broker.

See the following for more detail:

KIP-768
KAFKA-13202

Reviewers: Yi Ding <dingyi.zj@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
2021-10-28 11:36:53 -07:00
Jason Gustafson c1c639db77
KAFKA-13288; Include internal topics when searching hanging transactions (#11319)
This patch ensures that internal topics are included when searching for hanging transactions with the `--broker-id` argument in `kafka-transactions.sh`.

Reviewers: David Jacot <djacot@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2021-09-10 14:33:37 -07:00
Yanwen(Jason) Lin 66a27af2f1
KAFKA-10038: Supports default client.id for ConsoleConsumer, ProducerPerformance, ConsumerPerformance (#11297)
Reviewers: Guozhang Wang <wangguoz@gmail.com>
2021-09-07 13:49:50 -07:00
dengziming 1d22b0d706
KAFKA-10774; Admin API for Describe topic using topic IDs (#9769)
Reviewers: Justine Olshan <jolshan@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Satish Duggana <satishd@apache.org>, Rajini Sivaram <rajinisivaram@googlemail.com>
2021-08-28 09:00:36 +01:00
Jason Gustafson ba47beec01
MINOR: Ensure transactional message copier failures are logged (#11268)
This patch has a couple small improvements to `TransactionalMessageCopier` logging:

- Log all fatal exceptions which cause the copier to shutdown unexpectedly
- Log all non-fatal exceptions which cause the copier to abort a transaction

Reviewers: David Jacot <djacot@confluent.io>
2021-08-27 11:02:47 -07:00
David Jacot 4e2f2b0674
MINOR: Update `TransactionalMessageCopier` to use the latest transaction pattern (#11265)
Reviewers: Jason Gustafson <jason@confluent.io>
2021-08-27 11:11:57 +02:00
Jason Gustafson a5daae20b5
KAFKA-13155; Fix concurrent modification in consumer shutdown (#11164)
The `TransactionalMessageCopier` tool, which is used in system tests attempts to close the consumer as part of a shutdown hook. Although the access is synchronized, there is no guarantee that the consumer has finished polling when shutdown is invoked. The patch fixes the problem by call `wakeup()` from the shutdown hook and pushing the call to `close()` to the main thread.

Reviewers: David Jacot <djacot@confluent.io>
2021-08-10 09:39:59 -07:00
Jason Gustafson f29c43bdbb
KAFKA-12979; Implement command to find hanging transactions (#10974)
This patch implements the `find-hanging` command described in KIP-664: https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions#KIP664:Providetoolingtodetectandaborthangingtransactions-FindingHangingTransactions.

Reviewers: Luke Chen <showuon@gmail.com>, David Jacot <djacot@confluent.io>
2021-07-06 10:39:59 -07:00
Jason Gustafson fce771579c
KAFKA-12888; Add transaction tool from KIP-664 (#10814)
This patch adds the transaction tool specified in KIP-664: https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions. This includes all of the logic for describing transactional state and for aborting transactions. The only thing that is left out is the `--find-hanging` implementation, which will be left for a subsequent patch.

Reviewers: Boyang Chen <boyang@apache.org>, David Jacot <djacot@confluent.io>
2021-06-22 09:47:30 -07:00
CHUN-HAO TANG 580c111258
KAFKA-12662: add unit test for ProducerPerformance (#10588)
Reviewers: Luke Chen <showuon@gmail.com>, wenbingshen <oliver.shen999@gmail.com>, dengziming <dengziming1993@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2021-06-17 20:07:12 +08:00
Luke Chen efb7cda178
MINOR: update java doc for deprecated methods (#10722)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2021-05-23 18:33:01 -07:00
A. Sophie Blee-Goldman 3805f3706f
KAFKA-12574: KIP-732, Deprecate eos-alpha and replace eos-beta with eos-v2 (#10573)
Deprecates the following 

1. StreamsConfig.EXACTLY_ONCE
2. StreamsConfig.EXACTLY_ONCE_BETA
3. Producer#sendOffsetsToTransaction(Map offsets, String consumerGroupId)

And introduces a new StreamsConfig.EXACTLY_ONCE_V2 config. Additionally, this PR replaces usages of the term "eos-beta" throughout the code with the term "eos-v2"

Reviewers: Matthias J. Sax <mjsax@confluent.io>
2021-04-28 13:22:15 -07:00
Shay Elkin fc405d792d
Minor: Move trogdor out of tools and into its own gradle module (#10539)
Move Trogdor out of tools and into its own gradle module.  This allows us to minimize
the dependencies of the tools module.  We still keep Trogdor in the CLASSPATH
created by kafka-run-class.sh.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-04-15 11:37:15 -07:00
lamberken 80de188727
KAFKA-12611: Fix using random payload in ProducerPerformance incorrectly (#10469)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-04-13 14:28:31 +08:00
Ismael Juma 2342ec1d1c
KAFKA-12600: Remove deprecated config value `default` for client config `client.dns.lookup` (#10458)
The config has been deprecated since Kafka 2.6 (released ~1 year before
3.0), but it was the default before it got deprecated. As such, it's
reasonably unlikely that people would have set it explicitly.

Given the confusing `default` name even though it's _not_ the default, I
think we should remove it in 3.0.

Also remove `ClientDnsLookup.DEFAULT` (not public API), which unlocks
a number of code simplications.

Reviewers: David Jacot <djacot@confluent.io>
2021-04-01 07:59:59 -07:00
dengziming e6f8ca80cd
MINOR: Fix log statement whose placeholders are inconsistent with arguments (#10312)
1. When the 2nd argument is an exception we don't need a placeholder
2. Placeholders should equal to arguments.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-03-22 13:39:05 +08:00
Lee Dongjin e6f8f5d0ae
MINOR: Remove unused variables, methods, parameters, unthrown exceptions, and fix typos (#9457)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com
2021-03-10 13:21:30 +08:00
dengziming 3769bc21b5
MINOR: replace hard-coding utf-8 with StandardCharsets.UTF_8 (#10079)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-02-09 10:06:01 +08:00
APaMio 1670362236
MINOR: Replace Collection.toArray(new T[size]) by Collection.toArray(new T[0]) (#9750)
This PR is based on the research of https://shipilev.net/blog/2016/arrays-wisdom-ancients

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-12-21 17:38:33 +08:00
Scott Hendricks baef516789
Add ConfigurableProducerSpec to Trogdor for improved E2E latency tracking. (#9736)
Reviewer: Colin P. McCabe <cmccabe@apache.org>
2020-12-18 13:03:59 -08:00
Geordie cc0247bf53
MINOR: Leaves lock() outside the try block (#9687)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-12-04 23:56:01 +08:00
Hamza Slama 43fd630d80
MINOR: remove unnecessary semicolon from Agent.java and AgentClient.java (#9625)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2020-11-20 15:34:27 +08:00
Jason Gustafson 927edfece3
KAFKA-10601; Add support for append linger to Raft implementation (#9418)
The patch adds `quorum.append.linger.ms` behavior to the raft implementation. This gives users a powerful knob to tune the impact of fsync.  When an append is accepted from the state machine, it is held in an accumulator (similar to the producer) until the configured linger time is exceeded. This allows the implementation to amortize fsync overhead at the expense of some write latency.

The patch also improves our methodology for testing performance. Up to now, we have relied on the producer performance test, but it is difficult to simulate expected controller loads because producer performance is limited by other factors such as the number of producer clients and head-of-line blocking. Instead, this patch adds a workload generator which runs on the leader after election.

Finally, this patch brings us nearer to the write semantics expected by the KIP-500 controller. It makes the following changes:

- Introduce `RecordSerde<T>` interface which abstracts the underlying log implementation from `RaftClient`. The generic type is carried over to `RaftClient<T>` and is exposed through the read/write APIs.
- `RaftClient.append` is changed to `RaftClient.scheduleAppend` and returns the last offset of the expected log append.
- `RaftClient.scheduleAppend` accepts a list of records and ensures that the full set are included in a single batch.
- Introduce `RaftClient.Listener` with a single `handleCommit` API which will eventually replace `RaftClient.read` in order to surface committed data to the controller state machine. Currently `handleCommit` is only used for records appended by the leader.

Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Guozhang Wang <wangguoz@gmail.com>
2020-10-27 12:10:13 -07:00
Lee Dongjin 8d4bbf22ad
MINOR: trivial cleanups, javadoc errors, omitted StateStore tests, etc. (#8130)
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Guozhang Wang <guozhang@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2020-10-07 19:08:31 -07:00
Matthias J. Sax a15387f34d
KAFKA-9274: Revert deprecation of `retries` for producer and admin clients (#9333)
Reviewer: John Roesler <john@confluent.io>
2020-09-30 12:13:34 -07:00
Ismael Juma 7d0086e0c3
KAFKA-10447: Migrate tools module to JUnit 5 (#9231)
This change sets the groundwork for migrating other modules incrementally.

Main changes:
- Replace `junit` 4.13 with `junit-jupiter` and `junit-vintage` 5.7.0-RC1.
- All modules except for `tools` depend on `junit-vintage`.
- `tools` depends on `junit-jupiter`.
- Convert `tools` tests to JUnit 5.
- Update `PushHttpMetricsReporterTest` to use `mockito` instead of `powermock` and `easymock`
(powermock doesn't seem to work well with JUnit 5 and we don't need it since mockito can mock
static methods).
- Update `mockito` to 3.5.7.
- Update `TestUtils` to use JUnit 5 assertions since `tools` depends on it.

Unrelated clean-ups:
- Remove `unit` from package names in a few `core` tests.
- Replace `try/catch/fail` with `assertThrows` in a number of places.
- Tag `CoordinatorTest` as integration test.
- Remove unnecessary type parameters when invoking methods and constructors.

Tested with IntelliJ and gradle. Verified that the following commands work as expected:
* ./gradlew tools:unitTest
* ./gradlew tools:integrationTest
* ./gradlew tools:test
* ./gradlew core:unitTest
* ./gradlew core:integrationTest
* ./gradlew clients:test

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2020-09-10 16:14:38 -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
Chia-Ping Tsai dccc82ea9d
KAFKA-10308: Fix flaky core/round_trip_fault_test.py (#9079)
Creating a topic may fail (due to timeout) in running system tests. However, `RoundTripWorker` does not ignore `TopicExistsException` which makes `round_trip_fault_test.py` be a flaky one.

More specifically, a network exception can cause the `CreateTopics` request to reach Kafka but Trogdor retry it
and hit a `TopicAlreadyExists` exception on the retry, failing the test.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2020-08-04 06:53:10 -07:00
Matthias J. Sax 194c56fce2
KAFKA-9274: Mark `retries` config as deprecated and add new `task.timeout.ms` config (#8864)
- part of KIP-572
 - deprecates producer config `retries` (still in use)
 - deprecates admin config `retries` (still in use)
 - deprecates Kafka Streams config `retries` (will be ignored)
 - adds new Kafka Streams config `task.timeout.ms` (follow up PRs will leverage this new config)

Reviewers: John Roesler <john@confluent.io>, Jason Gustafson <jason@confluent.io>, Randall Hauch <randall@confluent.io>
2020-07-21 12:19:13 -07:00
Manikumar Reddy c38825ab97 KAFKA-9432:(follow-up) Set `configKeys` to null in `describeConfigs()` to make it backward compatible with older Kafka versions.
- After #8312, older brokers are returning empty configs,  with latest `adminClient.describeConfigs`.  Old brokers  are receiving empty configNames in `AdminManageer.describeConfigs()` method. Older brokers does not handle empty configKeys. Due to this old brokers are filtering all the configs.
- Update ClientCompatibilityTest to verify describe configs
- Add test case to test describe configs with empty configuration Keys

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

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>

Closes #9046 from omkreddy/KAFKA-9432
2020-07-21 17:32:11 +05:30
Manikumar Reddy 17256abb23 MINOR: Create ChannelBuilder for each connection in ConnectionStressWorker workload
- Currently we create single channel builder and reuse it in ConnectStressor workload.  This will fail when testing with secure connections, as we close channel builder after first connection.  This PR creates  ChannelBuilder for each test connection.
- Also increase to connect ready wait timeout to 500ms.

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

Reviewers: Ismael Juma <ismael@juma.me.uk>, Rajini Sivaram <rajinisivaram@googlemail.com>

Closes #8937 from omkreddy/Connect
2020-07-10 00:51:50 +05:30
Cheng Tan 55b5b248cd
KAFKA-9893: Configurable TCP connection timeout and improve the initial metadata fetch (KIP-601) (#8683)
Reviewers: David Jacot <djacot@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
2020-06-30 12:15:17 +01:00
Scott Hendricks 5203ab0c5d
MINOR - Increase the number of Trogdor Histogram buckets to 10000 (#8627)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2020-05-06 21:10:19 -07:00
Nikolay c07db1c7d9
KAFKA-9573: Fix VerifiableProducer and VerifiableConsumer to work with older Kafka versions (#8197)
These classes are used by `upgrade_test.py` with old Kafka versions so they can
only use functionality that exists in all Kafka versions. This change fixes the test
for Kafka versions older than 0.11.0.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2020-03-01 16:31:15 -08:00
Mitch 96c69da8c1
KAFKA-8507; Unify connection name flag for command line tool [KIP-499] (#8023)
This change updates ConsoleProducer, ConsumerPerformance, VerifiableProducer, and VerifiableConsumer classes to add and prefer the --bootstrap-server flag for defining the connection point of the Kafka cluster. This change is part of KIP-499: https://cwiki.apache.org/confluence/display/KAFKA/KIP-499+-+Unify+connection+name+flag+for+command+line+tool.

Reviewers: Ron Dagostino <rdagostino@confluent.io>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>,  Chia-Ping Tsai <chia7712@gmail.com>, Jason Gustafson <jason@confluent.io>
2020-02-13 13:44:51 -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
Karan Kumar c8d97c6d51
KAFKA-9375: Add names to all Connect threads (#7901)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ryanne Dolan <ryannedolan@gmail.com>, gcsaba2
2020-01-31 18:21:21 +00:00