Commit Graph

260 Commits

Author SHA1 Message Date
Federico Valeri 07e2f6cd4d
KAFKA-14578: Move ConsumerPerformance to tools (#13215)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Alexandre Dupriez <alexandre.dupriez@gmail.com>
2023-03-06 18:16:55 +01:00
vamossagar12 bb3111f472
KAFKA-14580: Moving EndToEndLatency from core to tools module (#13095)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>, Ismael Juma <mlists@juma.me.uk>
2023-03-02 12:05:22 +01:00
Gantigmaa Selenge ea30ec4b56
KAFKA-14590: Move DelegationTokenCommand to tools (#13172)
KAFKA-14590: Move DelegationTokenCommand to tools

Reviewers: Luke Chen <showuon@gmail.com>, Christo Lolov <christo_lolov@yahoo.com>, Federico Valeri <fvaleri@redhat.com>
2023-03-02 14:30:07 +08:00
Ron Dagostino 631e6be3a0
KAFKA-14711: kafaka-metadata-quorum.sh does not honor --command-confi… (#13241)
…g option

https://github.com/apache/kafka/pull/12951 accidentally changed the behavior of the `kafaka-metadata-quorum.sh` CLI by making it silently ignore a `--command-config <filename>` properties file that exists. This was an undetected regression in the 3.4.0 release.  This patch fixes the issue such that any such specified file will be honored.

Reviewers: José Armando García Sancio <jsancio@apache.org>, Ismael Juma <ismael@juma.me.uk>
2023-02-13 18:33:20 -05:00
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
Ron Dagostino a3509c0870 MINOR: MiniKdc JVM shutdown hook fix (#7946)
Also made all shutdown hooks consistent and added tests

Reviewers: Ismael Juma <ismael@juma.me.uk>, Rajini Sivaram <rajinisivaram@googlemail.com>
2020-01-24 22:21:12 +00:00
Jason Gustafson 2ac78ff621
MINOR: Propagate LogContext to channel builders and SASL authenticator (#7867)
The log context is useful when debugging applications which have multiple clients. This patch propagates the context to the channel builders and the SASL authenticator.

Reviewers: Ron Dagostino <rndgstn@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
2020-01-06 14:27:30 -08:00
Ismael Juma 6dc6f6a60d
KAFKA-9324: Drop support for Scala 2.11 (KIP-531) (#7859)
* Adjust build and documentation.
* Use lambda syntax for SAM types in `core`, `streams-scala` and
`connect-runtime` modules.
* Remove `runnable` and `newThread` from `CoreUtils` as lambda
syntax for SAM types make them unnecessary.
* Remove stale comment in `FunctionsCompatConversions`,
`KGroupedStream`, `KGroupedTable' and `KStream` about Scala 2.11,
the conversions are needed for Scala 2.12 too.
* Deprecate `org.apache.kafka.streams.scala.kstream.Suppressed`
and use `org.apache.kafka.streams.kstream.Suppressed` instead.
* Use `Admin.create` instead of `AdminClient.create`. Static methods
in Java interfaces can be invoked since Scala 2.12. I noticed that
MirrorMaker 2 uses `AdminClient.create`, but I did not change them
as Connectors have restrictions on newer client APIs.
* Improve efficiency in a few `Gauge` implementations by avoiding
unnecessary intermediate collections.
* Remove pointless `Option.apply` in `ZookeeperClient`
`SessionState` metric.
* Fix unused import/variable and other compiler warnings.
* Reduce visibility of some vals/defs.

Reviewers: Manikumar Reddy <manikumar@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, Gwen Shapira <gwen@confluent.io>
2020-01-06 19:51:01 +01:00
Jason Gustafson 9d8ab3a1a2
KAFKA-8509; Add downgrade system test (#7724)
This patch adds a basic downgrade system test. It verifies that producing and consuming continues to work before and after the downgrade.

Reviewers: Ismael Juma <ismael@juma.me.uk>, David Arthur <mumrah@gmail.com>
2019-11-22 10:09:13 -08:00
Colin Patrick McCabe fe7543abd3
Revert "KAFKA-9165: Fix jersey warnings in Trogdor (#7669)" (#7721)
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Jason Gustafson <jason@confluent.io>
2019-11-20 12:49:25 -08:00
David Arthur d04699486d
KAFKA-8981 Add rate limiting to NetworkDegradeSpec (#7446)
* Add rate limiting to tc

* Feedback from PR

* Add a sanity test for tc

* Add iperf to vagrant scripts

* Dynamically determine the network interface

* Add some temp code for testing on AWS

* Temp: use hostname instead of external IP

* Temp: more AWS debugging

* More AWS WIP

* More AWS temp

* Lower latency some

* AWS wip

* Trying this again now that ping should work

* Add cluster decorator to tests

* Fix broken import

* Fix device name

* Fix decorator arg

* Remove errant import

* Increase timeouts

* Fix tbf command, relax assertion on latency test

* Fix log line

* Final bit of cleanup

* Newline

* Revert Trogdor retry count

* PR feedback

* More PR feedback

* Feedback from PR

* Remove unused argument
2019-11-18 20:36:00 -05:00
Colin Patrick McCabe 69a63304de
KAFKA-9165: Fix jersey warnings in Trogdor (#7669)
Reviewers: David Arthur <mumrah@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
2019-11-15 10:41:12 -08:00
Jason Gustafson 903d66e2f9 KAFKA-9079: Fix reset logic in transactional message copier
The consumer's `committed` API does not return an entry in the response map for a requested partition if there is no committed offset. The transactional message copier, which is used in the transaction system test, did not account for this. If the first transaction attempted by the copier was randomly aborted, then we would not seek to the beginning as expected, which means we would fail to copy some of the records.

This patch fixes the problem by iterating over the assignment rather than the result of `committed` when resetting offsets. It also adds enables additional logging in the transaction message copier service to make finding problems easier in the future.

Author: Jason Gustafson <jason@confluent.io>

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

Closes #7653 from hachikuji/fix-transaction-system-test
2019-11-06 15:59:51 +05:30
Guozhang Wang bcc023773f
KAFKA-8880: Add overloaded function of Consumer.committed (#7304)
1. Add the overloaded functions.
2. Update the code in Streams to use the batch API for better latency (this applies to both active StreamsTask for initialize the offsets, as well as the StandbyTasks for updating offset limits).
3. Also update all unit test to replace the deprecated APIs.

Reviewers: Christopher Pettitt <cpettitt@confluent.io>, Kamal Chandraprakash  <kamal.chandraprakash@gmail.com>, Bill Bejeck <bill@confluent.io>
2019-09-24 13:23:27 -07:00
Scott Hendricks 39afe9fe0e KAFKA-8853; Create sustained connections test for Trogdor
This creates a test that generates sustained connections against Kafka.  There
are three different components we can stress with this, KafkaConsumer,
KafkaProducer, and AdminClient.  This test tries use minimal bandwidth per
connection to reduce overhead impacts.

This test works by creating a threadpool that creates connections and then
maintains a central pool of connections at a specified keepalive rate.  The
keepalive action varies by which component is being stressed:

  * KafkaProducer:  Sends one single produce record.  The configuration for
    the produce request uses the same key/value generator as the ProduceBench
    test.

  * KafkaConsumer: Subscribes to a single partition, seeks to the end, and
    then polls a minimal number of records.  Each consumer connection is its
    own consumer group, and defaults to 1024 bytes as FETCH_MAX_BYTES to keep
    traffic to a minimum.

  * AdminClient: Makes an API call to get the nodes in the cluster.

NOTE: This test is designed to be run alongside a ProduceBench test for a
specific topic, due to the way the Consumer test polls a single partition.
There may be no data returned by the consumer test if this is run on its own.
The connection should still be kept alive, but with no data returned.

Author: Scott Hendricks <scott.hendricks@confluent.io>

Reviewers: Stanislav Kozlovski, Gwen Shapira

Closes #7289 from scott-hendricks/trunk
2019-09-08 19:49:13 -07:00
Robert Yokota add662907d MINOR: fix ProduceBenchWorker not to fail on final produce (#7254)
When sending bad records, the Trogdor task will fail if the final record produced is bad. Instead we should catch the exception to allow the task to finish since sending bad records is a valid use case.

Reviewers: Tu V. Tran <tuvtran97@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
2019-08-27 09:55:51 -07:00
jolshan 2c2b30d96b MINOR: Add RandomComponentPayloadGenerator and update Trogdor documentation (#7103)
Add a new RandomComponentPayloadGenerator that gives a payload based on random selection of another PayloadGenerator.  Additionally, add an example that uses a non-default PayloadGenerator configuration to TROGDOR.md.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-07-31 14:00:49 -07:00
Andy Coates 2a133ba656 KAFKA-8454; Add Java AdminClient Interface (KIP-476) (#7087)
Adds an `Admin` interface as specified in [KIP-476](https://cwiki.apache.org/confluence/display/KAFKA/KIP-476%3A+Add+Java+AdminClient+Interface).

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
2019-07-22 15:47:34 -07:00
jolshan 442d36241b MINOR: add useConfiguredPartitioner and skipFlush options for ProduceBench
Add a "useConfiguredPartitioner" boolean to specify testing with the configured partitioner, rather than overriding the partitioner in the test.

Add a "skipFlush" boolean to specify skipping the flush operation when producing.  This is helpful when testing some scenarios where linger.ms is greater than 0.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-07-03 17:23:36 -07:00
Colin Patrick McCabe 822abe47db
MINOR: WorkerUtils#topicDescriptions must unwrap exceptions properly (#6937)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
2019-07-03 16:08:39 -07:00
David Arthur d7a5e31ca2
KAFKA-8519 Add trogdor action to slow down a network (#6912)
This adds a new Trogdor fault spec for inducing network latency on a network device for system testing. It operates very similarly to the existing network partition spec by executing the `tc` linux utility.
2019-06-21 11:30:05 -04:00
Stanislav Kozlovski 58aa04f91e MINOR: Improve Trogdor external command worker docs (#6438)
Reviewers: Colin McCabe <cmccabe@apache.org>, Xi Yang <xi@confluent.io>
2019-06-06 10:04:05 -07:00
Lee Dongjin b43f5446ac KAFKA-8316; Remove deprecated usage of Slf4jRequestLog, SslContextFactory (#6668)
* Remove deprecated class Slf4jRequestLog: use Slf4jRequestLogWriter, CustomRequestLog instread.

1. Remove '@SuppressWarnings("deprecation")' from RestServer#initializeResources, JsonRestServer#start.
2. Remove unused JsonRestServer#httpRequest.

* Fix deprecated class usage: SslContextFactory -> SslContextFactory.[Server, Client]

1. Split SSLUtils#createSslContextFactory into SSLUtils#create[Server, Client]SideSslContextFactory: each method instantiates SslContextFactory.[Server, Client], respectively.
2. SSLUtils#configureSslContextFactoryAuthentication is called from SSLUtils#createServerSideSslContextFactory only.
3. Update SSLUtilsTest following splittion; for client-side SSL Context Factory, SslContextFactory#get[Need, Want]ClientAuth is always false. (SSLUtilsTest#testCreateClientSideSslContextFactory)

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
2019-05-20 10:15:15 -07:00
Boyang Chen 9fa331b811 KAFKA-8225 & KIP-345 part-2: fencing static member instances with conflicting group.instance.id (#6650)
For static members join/rejoin, we encode the current timestamp in the new member.id. The format looks like group.instance.id-timestamp.

During consumer/broker interaction logic (Join, Sync, Heartbeat, Commit), we shall check the whether group.instance.id is known on group. If yes, we shall match the member.id stored on static membership map with the request member.id. If mismatching, this indicates a conflict consumer has used same group.instance.id, and it will receive a fatal exception to shut down.

Right now the only missing part is the system test. Will work on it offline while getting the major logic changes reviewed.

Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2019-05-18 07:28:36 -07:00
Ismael Juma a37282415e
MINOR: Upgrade dependencies for Kafka 2.3 (#6665)
Many patch and minor updates.

Scalatest and Jetty deprecated classes that we
use. I removed usages for the former and filed KAFKA-8316 for the latter (I
suppressed the relevant deprecation warnings until the JIRA is fixed). As
part of the scalatest fixes, I also removed `TestUtils.fail` since it duplicates
`Assertions.fail`.

I also fixed a few compiler warnings that have crept in since my last sweep.

Updates of note:
- Jetty: 9.4.14 -> 9.4.18
  * https://github.com/eclipse/jetty.project/releases/tag/jetty-9.4.15.v20190215
  * https://github.com/eclipse/jetty.project/releases/tag/jetty-9.4.16.v20190411
  * https://github.com/eclipse/jetty.project/releases/tag/jetty-9.4.17.v20190418
  * https://github.com/eclipse/jetty.project/releases/tag/jetty-9.4.17.v20190418
  * https://github.com/eclipse/jetty.project/releases/tag/jetty-9.4.18.v20190429
- zstd: 1.3.8-1 -> 1.4.0-1
  * https://github.com/facebook/zstd/releases/tag/v1.4.0
  * zstd's fastest strategy, 6-8% faster in most scenarios
- zookeeper: 3.4.13 -> 3.4.14
  * https://zookeeper.apache.org/doc/r3.4.14/releasenotes.html

### Committer Checklist (excluded from commit message)
- [ ] Verify design and implementation 
- [ ] Verify test coverage and CI build status
- [ ] Verify documentation (including upgrade notes)

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2019-05-03 10:35:07 -07:00
Boyang Chen 0f995ba6be KAFKA-7862 & KIP-345 part-one: Add static membership logic to JoinGroup protocol (#6177)
This is the first diff for the implementation of JoinGroup logic for static membership. The goal of this diff contains:

* Add group.instance.id to be unique identifier for consumer instances, provided by end user;
Modify group coordinator to accept JoinGroupRequest with/without static membership, refactor the logic for readability and code reusability.
* Add client side support for incorporating static membership changes, including new config for group.instance.id, apply stream thread client id by default, and new join group exception handling.
* Increase max session timeout to 30 min for more user flexibility if they are inclined to tolerate partial unavailability than burdening rebalance.
* Unit tests for each module changes, especially on the group coordinator logic. Crossing the possibilities like:
6.1 Dynamic/Static member
6.2 Known/Unknown member id
6.3 Group stable/unstable
6.4 Leader/Follower

The rest of the 345 change will be broken down to 4 separate diffs:

* Avoid kicking out members through rebalance.timeout, only do the kick out through session timeout.
* Changes around LeaveGroup logic, including version bumping, broker logic, client logic, etc.
* Admin client changes to add ability to batch remove static members
* Deprecate group.initial.rebalance.delay

Reviewers: Liquan Pei <liquanpei@gmail.com>, Stanislav Kozlovski <familyguyuser192@windowslive.com>, Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2019-04-26 11:44:38 -07:00
Bob Barrett 964e90a725 MINOR: Remove errant lock.unlock() call from RoundTripWorker (#6612)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-04-20 10:43:35 -07:00
Colin Patrick McCabe 04a023e302
MINOR: ConnectionStressWorker: add missing executor shutdown (#6558) 2019-04-11 11:16:06 -07:00
Colin P. Mccabe a674ded0b3 MINOR: fix throttling and status in ConnectionStressWorker
Each separate thread should have its own throttle, so that it can sleep
for an appropriate amount of time when needed.

ConnectionStressWorker should avoid recalculating the status after
shutting down the runnables.  Otherwise, if one runnable is slow to
stop, it will skew the average down in a way that doesn't reflect
reality.  This change moves the status calculation into a separate
periodic runnable that gets shut down cleanly before the other ones.

Author: Colin P. Mccabe <cmccabe@confluent.io>

Reviewers: Gwen Shapira, Stanislav Kozlovski

Closes #6533 from cmccabe/fix_connection_stress_worker
2019-04-04 14:16:56 -07:00
Stanislav Kozlovski 4825bc47a0 KAFKA-8183: Add retries to WorkerUtils#verifyTopics (#6532)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-04-02 17:09:40 -07:00
Colin P. Mccabe 219c22113e MINOR: Optimize ConnectionStressWorker
Optimize ConnectionStressWorker by avoiding creating a new
ChannelBuilder each time we want to open a new connection.

Author: Colin P. Mccabe <cmccabe@confluent.io>

Reviewers: Gwen Shapira

Closes #6518 from cmccabe/optimize-connection-stress-worker
2019-03-29 15:02:10 -07:00
Colin Patrick McCabe b25974c387
MINOR: WorkerUtils#abort: fix bug in abort logic (#6516)
doneFuture is supposed to be completed with an empty string (meaning success) or a non-empty string which is the error message.  Currently, due to exception.getMessage sometimes returning null or an empty string, this is not working correctly.  This patch fixes that.

Reviewers: David Arthur <mumrah@gmail.com>
2019-03-28 14:47:37 -07:00
Stanislav Kozlovski 0d55f0f3ec KAFKA-8102: Add an interval-based Trogdor transaction generator (#6444)
This patch adds a TimeIntervalTransactionsGenerator class which enables the Trogdor ProduceBench worker to commit transactions based on a configurable millisecond time interval.

Also, we now handle 409 create task responses in the coordinator command-line client by printing a more informative message

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-03-25 09:58:11 -07:00
Stanislav Kozlovski 6217178139 KAFKA-7819: Improve RoundTripWorker (#6187)
RoundTripWorker to should use a long field for maxMessages rather than an int.  The consumer group used should unique as well.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-03-21 10:03:09 -07:00
Rajini Sivaram ca6ac9393b
MINOR: Retain public constructors of classes from public API (#6455)
TopicDescription and ConsumerGroupDescription in org.apache.kafka.clients.admin. are part of the public API, so we should retain the existing public constructor. Changed the new constructor with authorized operations to be package-private to avoid maintaining more public constructors since we only expect admin client to use this.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2019-03-18 08:51:50 +00:00
Stanislav Kozlovski f20f3c1a97 MINOR: Update Trogdor ConnectionStressWorker status at the end of execution (#6445)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-03-15 09:53:21 -07:00
Stanislav Kozlovski 9a384daf0e MINOR: Change Trogdor agent's cleanup executor to a cached thread pool (#6309)
It is best to use a growing thread pool for worker cleanups. This lets us ensure that we close workers as fast as possible and not get slowed down on blocking cleanups.

Reviewers: Colin McCabe <cmccabe@apache.org>, Jason Gustafson <jason@confluent.io>
2019-03-12 08:31:50 -07:00
Manikumar Reddy a42f16f980 KAFKA-7922: Return authorized operations in Metadata request response (KIP-430 Part-2)
-  Use automatic RPC generation in Metadata Request/Response classes
-  https://cwiki.apache.org/confluence/display/KAFKA/KIP-430+-+Return+Authorized+Operations+in+Describe+Responses

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

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>

Closes #6352 from omkreddy/KIP-430-METADATA
2019-03-10 17:30:16 +05:30
Bob Barrett 5e16173659 MINOR: Log exception thrown by consumer.poll() in VerifiableConsumer (#6368)
SecurityTest.test_client_ssl_endpoint_validation_failure is failing because it greps for 'SSLHandshakeException in the consumer and producer log files. With the fix for KAKFA-7773, the test uses the VerifiableConsumer instead of the ConsoleConsumer, which does not log the exception stack trace to the service log. This patch catches exceptions in the VerifiableConsumer and logs them in order to fix the test. Tested by running the test locally.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
2019-03-05 18:12:47 -08:00
Chia-Ping Tsai 35a0de32ee KAFKA-6161 Add default implementation to close() and configure() for Serdes (#5348)
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2019-02-21 09:05:13 -08:00
Sönke Liebau 08dbec7429 MINOR: Add sentence to command line documentation of producer performance tool on how to disable throttling. (#6205) 2019-02-07 07:59:32 -08:00
Colin Patrick McCabe 4be68c58da
KAFKA-7828: Add ExternalCommandWorker to Trogdor (#6219)
Allow the Trogdor agent to execute external commands. The agent communicates with the external commands via stdin, stdout, and stderr.

Based on a patch by Xi Yang <xi@confluent.io>

Reviewers: David Arthur <mumrah@gmail.com>
2019-02-06 16:42:02 -08:00
Colin Patrick McCabe a79d6dcdb6
KAFKA-7793: Improve the Trogdor command line. (#6133)
* Allow the Trogdor agent to be started in "exec mode", where it simply
runs a single task and exits after it is complete.

* For AgentClient and CoordinatorClient, allow the user to pass the path
to a file containing JSON, instead of specifying the JSON object in the
command-line text itself.  This means that we can get rid of the bash
scripts whose only function was to load task specs into a bash string
and run a Trogdor command.

* Print dates and times in a human-readable way, rather than as numbers
of milliseconds.

* When listing tasks or workers, output human-readable tables of
information.

* Allow the user to filter on task ID name, task ID pattern, or task
state.

* Support a --json flag to provide raw JSON output if desired.

Reviewed-by: David Arthur <mumrah@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
2019-01-24 09:26:51 -08:00
Stanislav Kozlovski 2e53fa08af KAFKA-7792: Add simple /agent/uptime and /coordinator/uptime health check endpoints (#6130)
Reviewed-by: Colin P. McCabe <cmccabe@apache.org>
2019-01-15 11:52:48 -08:00
Stanislav Kozlovski 13f679013a MINOR: Update Trogdor StringExpander regex to handle an epilogue (#6123)
Update the Trogdor StringExpander regex to handle an epilogue.  Previously the regex would use a lazy quantifier at the end, which meant it would not catch anything after the range expression.  Add a unit test.

Reviewed-by: Colin P. McCabe <cmccabe@apache.org>
2019-01-14 20:49:24 -08:00
Lee Dongjin 7df3e8cd38 KAFKA-7808: AdminClient#describeTopics should not throw InvalidTopic if topic name is not found (#6124)
* Update KafkaAdminClient#describeTopics to throw UnknownTopicOrPartitionException.

* Remove unused method: WorkerUtils#getMatchingTopicPartitions.

* Add some JavaDoc.

Reviewed-by: Colin P. McCabe <cmccabe@apache.org>, Ryanne Dolan <ryannedolan@gmail.com>
2019-01-11 14:56:06 -08:00
Stanislav Kozlovski 625e0d8829 KAFKA-7790: Fix Bugs in Trogdor Task Expiration (#6103)
The Trogdor Coordinator now overwrites a task's startMs to the time it received it if startMs is in the past.

The Trogdor Agent now correctly expires a task after the expiry time (startMs + durationMs) passes. Previously, it would ignore startMs and expire after durationMs milliseconds of local start of the task.

Reviewed-by: Colin P. McCabe <cmccabe@apache.org>
2019-01-11 13:38:00 -08:00
Srinivas Reddy 85906d3d2b MINOR: Switch anonymous classes to lambda expressions in tools module
Switch to lambda when ever possible instead of old anonymous way
in tools module

Author: Srinivas Reddy <srinivas96alluri@gmail.com>
Author: Srinivas Reddy <mrsrinivas@users.noreply.github.com>

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

Closes #6013 from mrsrinivas/tools-switch-to-java8
2018-12-21 14:20:57 +05:30
Arjun Satish 3ef6b0fce0 MINOR: Add unit for max latency in ProducerPerformance output (#6014)
Reviewers: Jason Gustafson <jason@confluent.io>
2018-12-12 08:28:58 -08:00
seayoun 4fb5520901 MINOR: Fix ProducerPerformance bug when numRecords > Integer.MAX (#5956)
Current code will fall into non-stop loop and send more message to broker, and Stat in PerfCallback method record will throw ArrayIndexOutOfBoundsException
2018-11-29 12:18:56 +05:30
Stanislav Kozlovski ec668180d7 MINOR: Support long maxMessages in Trogdor consume/produce bench workers (#5957)
Reivewers: Colin McCabe <cmccabe@apache.org>
2018-11-28 09:13:21 -08:00