Commit Graph

59 Commits

Author SHA1 Message Date
Omnia G.H Ibrahim f9fde0eec1
KAFKA-15102: Add replication.policy.internal.topic.separator.enabled property to MirrorMaker 2 (KIP-949) (#14082)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-08-15 20:47:30 -04:00
Ed B 742de94e03 KAFKA-13255: Use config.properties.exclude when mirroring topics (#11401)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2022-04-29 08:33:26 +01:00
Lee Dongjin 417630d9be KAFKA-13397: MirrorMaker should not mirror topics ending with `.internal` (#11431)
When running in dedicated mode, Connect runtimes are configured to use the `.internal` suffix for their topics. 

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>
2021-11-24 11:21:48 +01:00
Omnia G H Ibrahim 400d39bb0e
KAFKA-10777: Add additional configuration to control MirrorMaker 2 internal topics naming convention - KIP-690 (#11220)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2021-10-11 16:35:12 +02: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
Ryanne Dolan 93f57370c6
KAFKA-9726: Add IdentityReplicationPolicy to MirrorMaker2 (#10652)
This new policy enables active/passive, one-way replication without renaming topics, similar to MM1. This implementation is described in KIP-382 (adopted), originally as "LegacyReplicationPolicy".

This enables operators to migrate from MM1 to MM2 without re-architecting their replication flows, and enables some additional use-cases for MM2. For example, operators may wish to "upgrade" their Kafka clusters by mirroring everything to a completely new cluster. Such a migration would have been difficult with either MM1 or MM2 previously.

When using IdentityReplicationPolicy, operators should be aware that MM2 will not be able to detect cycles among replicated topics. A misconfigured topology may result in replicating the same records back-and-forth or in an infinite loop. However, we don't prevent this behavior, as some use-cases involve filtering records (via SMTs) to prevent cycles.

Reviewers: Mickael Maison <mickael.maison@gmail.com>

Co-authored-by: Ryanne Dolan <rdolan@twitter.com>
Co-authored-by: Matthew de Detrich <mdedetrich@gmail.com>
Co-authored-by: Ivan Yurchenko <ivanyu@aiven.io>
2021-07-01 09:21:27 +01:00
Mickael Maison 3c4be0b57a
KAFKA-12379: Allow configuring the location of the offset-syncs topic with MirrorMaker2 (#10221)
This commit implements KIP-716. It introduces a new setting `offset-syncs.topic.location` that allows specifying where the offset-syncs topic is created.

Reviewers: Tom Bentley <tbentley@redhat.com>, Edoardo Comar <ecomar@uk.ibm.com>
2021-06-29 21:33:59 +01:00
Ismael Juma d27a84f70c
KAFKA-12945: Remove port, host.name and related configs in 3.0 (#10872)
They have been deprecated since 0.10.0. Full list of removes configs:
* port
* host.name
* advertised.port
* advertised.host.name

Also adjust tests to take the removals into account. Some tests were
no longer relevant and have been removed.

Finally, took the chance to:
* Clean up unnecessary usage of `KafkaConfig$.MODULE$` in
related files.
* Add missing `Test` annotations to `AdvertiseBrokerTest` and
make necessary changes for the tests to pass.

Reviewers: David Jacot <djacot@confluent.io>, Luke Chen <showuon@gmail.com>
2021-06-17 05:32:34 -07:00
Viktor Somogyi-Vass 8a50a5204d
KAFKA-12922: MirrorCheckpointTask should close topic filter (#10849)
Reviewers: Mickael Maison <mickael.maison@gmail.com>

Co-authored-by: Daniel Urban <durban@cloudera.com>
2021-06-10 17:17:42 +01:00
Matthew de Detrich 56d9482462
KAFKA-12819: Add assert messages to MirrorMaker tests plus other quality of life improvements (#10762)
Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Konstantine Karantasis <k.karantasis@gmail.com>
2021-05-26 23:21:45 -07:00
Luke Chen ab06aef2b8
KAFKA-12284: increase request timeout to make tests reliable (#10547)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-04-28 23:56:48 +08:00
Lee Dongjin d3538ed0ab
MINOR: Remove unthrown exceptions, fix typo, etc. (#10402)
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Ryanne Dolan <ryannedolan@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2021-04-21 10:40:01 +08:00
Ismael Juma 89933f21f2
KAFKA-12612: Remove `checksum` from ConsumerRecord/RecordMetadata for 3.0 (#10470)
The methods have been deprecated since 0.11 without replacement since
message format 2 moved the checksum to the record batch (instead of the
record).

Unfortunately, we did not deprecate the constructors that take a checksum
(even though we intended to) so we cannot remove them. I have deprecated
them for removal in 4.0 and added a single non deprecated constructor to
`ConsumerRecord` and `RecordMetadata` that take all remaining parameters.
`ConsumerRecord` could do with one additional convenience constructor, but
that requires a KIP and hence should be done separately.

Also:
* Removed `ChecksumMessageFormatter`, which is technically not public
API, but may have been used with the console consumer.
* Updated all usages of `ConsumerRecord`/`RecordMetadata` constructors
to use the non deprecated ones.
* Added tests for deprecated `ConsumerRecord/`RecordMetadata`
constructors.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>
2021-04-14 14:38:37 -07:00
Luke Chen 4e60ad72fb
MINOR: Improve error message in MirrorConnectorsIntegrationBaseTest (#10268)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2021-03-12 12:14:53 +00:00
bmaidics 0dbafce8dd
MINOR: Add missing unit tests for Mirror Connect (#10192)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
2021-03-12 11:32:17 +00:00
Luke Chen f6929637b9
KAFKA-12284: Wait for mirrored topics to be created (#10185)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2021-03-03 19:20:08 +00:00
Dhruvil Shah cc088c5abe
KAFKA-12254: Ensure MM2 creates topics with source topic configs (#10217)
MM2 creates new topics on the destination cluster with default configurations. It has an async periodic task to refresh topic configurations from the source to destination. However, this opens up a window where the destination cluster has data produced to it with default configurations. In the worst case, this could cause data loss if the destination topic is created without the right cleanup.policy. This commit fixes the above issue by ensuring that the right configurations are supplied to AdminClient#createTopics when MM2 creates topics on the destination cluster.

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
2021-03-01 09:30:30 +00:00
iczellion 38ec258c66
MINOR: Fix Typo in MirrorMaker README file (#10144)
Fix Typo in metric name of MirrorMaker README file from 'replication-latecny-ms' to 'replication-latency-ms'

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Eric Beaudet <galpha@gmail.com>
2021-02-18 09:54:02 +00:00
Randall Hauch 1d4f69be34
KAFKA-12326: Corrected regresion in MirrorMaker 2 executable introduced with KAFKA-10021 (#10122)
Fixes the recent change to the `MirrorMaker` class (used only in the MirrorMaker 2 executable) that uses a `SharedTopicAdmin` client as part of Connect, so that the correct properties into the `SharedTopicAdmin`.

Author: Randall Hauch <rhauch@gmail.com>
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
2021-02-12 16:53:29 -06:00
Randall Hauch 982ea2f6a4
KAFKA-10021: Changed Kafka backing stores to use shared admin client to get end offsets and create topics (#9780)
The existing `Kafka*BackingStore` classes used by Connect all use `KafkaBasedLog`, which needs to frequently get the end offsets for the internal topic to know whether they are caught up. `KafkaBasedLog` uses its consumer to get the end offsets and to consume the records from the topic.

However, the Connect internal topics are often written very infrequently. This means that when the `KafkaBasedLog` used in the `Kafka*BackingStore` classes is already caught up and its last consumer poll is waiting for new records to appear, the call to the consumer to fetch end offsets will block until the consumer returns after a new record is written (unlikely) or the consumer’s `fetch.max.wait.ms` setting (defaults to 500ms) ends and the consumer returns no more records. IOW, the call to `KafkaBasedLog.readToEnd()` may block for some period of time even though it’s already caught up to the end.

Instead, we want the `KafkaBasedLog.readToEnd()` to always return quickly when the log is already caught up. The best way to do this is to have the `KafkaBackingStore` use the admin client (rather than the consumer) to fetch end offsets for the internal topic. The consumer and the admin API both use the same `ListOffset` broker API, so the functionality is ultimately the same but we don't have to block for any ongoing consumer activity.

Each Connect distributed runtime includes three instances of the `Kafka*BackingStore` classes, which means we have three instances of `KafkaBasedLog`. We don't want three instances of the admin client, and should have all three instances of the `KafkaBasedLog` share a single admin client instance. In fact, each `Kafka*BackingStore` instance currently creates, uses and closes an admin client instance when it checks and initializes that store's internal topic. If we change `Kafka*BackingStores` to share one admin client instance, we can change that initialization logic to also reuse the supplied admin client instance.

The final challenge is that `KafkaBasedLog` has been used by projects outside of Apache Kafka. While `KafkaBasedLog` is definitely not in the public API for Connect, we can make these changes in ways that are backward compatible: create new constructors and deprecate the old constructors. Connect can be changed to only use the new constructors, and this will give time for any downstream users to make changes.

These changes are implemented as follows:
1. Add a `KafkaBasedLog` constructor to accept in its parameters a supplier from which it can get an admin instance, and deprecate the old constructor. We need a supplier rather than just passing an instance because `KafkaBasedLog` is instantiated before Connect starts up, so we need to create the admin instance only when needed. At the same time, we'll change the existing init function parameter from a no-arg function to accept an admin instance as an argument, allowing that init function to reuse the shared admin instance used by the `KafkaBasedLog`. Note: if no admin supplier is provided (in deprecated constructor that is no longer used in AK), the consumer is still used to get latest offsets.
2. Add to the `Kafka*BackingStore` classes a new constructor with the same parameters but with an admin supplier, and deprecate the old constructor. When the classes instantiate its `KafkaBasedLog` instance, it would pass the admin supplier and pass an init function that takes an admin instance.
3. Create a new `SharedTopicAdmin` that lazily creates the `TopicAdmin` (and underlying Admin client) when required, and closes the admin objects when the `SharedTopicAdmin` is closed.
4. Modify the existing `TopicAdmin` (used only in Connect) to encapsulate the logic of fetching end offsets using the admin client, simplifying the logic in `KafkaBasedLog` mentioned in #1 above. Doing this also makes it easier to test that logic.
5. Change `ConnectDistributed` to create a `SharedTopicAdmin` instance (that is `AutoCloseable`) before creating the `Kafka*BackingStore` instances, passing the `SharedTopicAdmin` (which is an admin supplier) to all three `Kafka*BackingStore objects`, and finally always closing the `SharedTopicAdmin` upon termination. (Shutdown of the worker occurs outside of the `ConnectDistributed` code, so modify `DistributedHerder` to take in its constructor additional `AutoCloseable` objects that should be closed when the herder is closed, and then modify `ConnectDistributed` to pass the `SharedTopicAdmin` as one of those `AutoCloseable` instances.)
6. Change `MirrorMaker` similarly to `ConnectDistributed`.
7. Change existing unit tests to no longer use deprecated constructors.
8. Add unit tests for new functionality.

Author: Randall Hauch <rhauch@gmail.com>
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
2021-02-09 11:09:41 -06:00
Lev Zemlyanov 07843cfbf4
KAFKA-10834: Remove redundant type casts in Connect (#10053)
Cleanup up to remove redundant type casts in Connect and use the diamond operator when needed 

Reviewers: Konstantine Karantasis <k.karantasis@gmail.com>
2021-02-04 17:08:56 -08:00
Lev Zemlyanov c19a35d1b7
KAFKA-10835: Replace Runnable and Callable overrides with lambdas in Connect (#9867)
Reviewers: Konstantine Karantasis <k.karantasis@gmail.com>
2021-02-04 09:15:49 -08:00
Ismael Juma bf4afae8f5 MINOR: Fix MirrorMakerConfigTest build failures 2021-01-28 18:32:35 -08:00
Julien Chanaud a63094930f
KAFKA-10710; MM2 - Create herders only if source->target.enabled=true and heartbeats are disabled (#9589)
By default Mirror Maker 2 creates herders for all the possible combinations even if the "links" are not enabled.

This is because the beats are emitted from the "opposite" herder.
If there is a replication flow from A to B and heartbeats are required, 2 herders are needed :

- A->B for the MirrorSourceConnector
- B->A for the MirrorHeartbeatConnector

The MirrorHeartbeatConnector on B->A emits beats into topic heartbeats on cluster A.
The MirrorSourceConnector on A->B then replicates whichever topic is configured as well as heartbeats.

In cases with multiple clusters (10 and more), this leads to an incredible amount of connections, file descriptors and configuration topics created in every target clusters that are not necessary.

With this code change, we will leverage the top level property "emit.heartbeats.enabled" which defaults to "true".
We skip creating the A->B herder whenever A->B.emit.heartbeats.enabled=false (defaults to true) and A->B.enabled=false (defaults to false). 

Existing users will not see any change and if they depend on these "opposites" herders for their monitoring, it will still work.
New users with more complex use case can change this property and fine tune their heartbeat generation.

Reviewers: Ryanne Dolan <ryannedolan@gmail.com>,  Sanjana Kaundinya <skaundinya@gmail.com>, Jason Gustafson <jason@confluent.io>
2021-01-28 14:52:51 -08:00
Luke Chen 45b7a0a334
KAFKA-12229: Restore original class loader in integration tests using EmbeddedConnectCluster during shutdown (#9942)
Fixes errors such as: 
```
java.lang.NullPointerException at 
org.apache.kafka.connect.mirror.MirrorSourceConnector.listTopics(MirrorSourceConnector.java:348) at 
org.apache.kafka.connect.mirror.MirrorSourceConnector.findSourceTopicPartitions(MirrorSourceConnector.java:192) at 
org.apache.kafka.connect.mirror.MirrorSourceConnectorTest.testRefreshTopicPartitionsTopicOnTargetFirst(MirrorSourceConnectorTest.java:222)
```

It was a difficult to debug issue due to class loading interference between the Connect worker and Mockito. Digging into the Mockito, found it's not about JUnit 5, it's because of the class loader. In Mockito, we rely on the class loader to generate the proxy instance ([source](https://github.com/mockito/mockito/blob/release/3.x/src/main/java/org/mockito/internal/creation/bytebuddy/SubclassBytecodeGenerator.java#L91)) to intercept the method call, and if the class loader is not expected, we'll generate the wrong proxy instance (with wrong class path). We set the class loader during connector start to resolve conflicting dependencies ([KIP-146](https://cwiki.apache.org/confluence/display/KAFKA/KIP-146+-+Classloading+Isolation+in+Connect)), so we should set it back to the original class loader after connector stop in tests (`EmbeddedConnectCluster` is only used in tests) for the following Mockito works as expected.

So, there's an interference of integration tests with unit tests when Connect integration tests run before the MM2 unit tests, and that will cause the Mockito used in unit tests not work as expected.

Reviewers: Konstantine Karantasis <k.karantasis@gmail.com>
2021-01-25 19:36:57 -08:00
Chia-Ping Tsai e0ebb1d202
KAFKA-12202 Migrate connect:mirror module to JUnit 5 (#9894)
1. Replace junit 4 APIs by junit 5
2. Remove the dependencies of junit 4 from `EmbeddedKafkaCluster`

Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-01-16 09:30:04 -08:00
Ning Zhang 2cde6f61b8
KAFKA-10304: Refactor MM2 integration tests (#9224)
Co-authored-by: Ning Zhang <nzhang1220@fb.com>
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2021-01-14 14:48:17 +00:00
Randall Hauch 8db3b1a09a
KAFKA-10811: Correct the MirrorConnectorsIntegrationTest to correctly mask the exit procedures (#9698)
Normally the `EmbeddedConnectCluster` class masks the `Exit` procedures using within the Connect worker. This normally works great when a single instance of the embedded cluster is used. However, the `MirrorConnectorsIntegrationTest` uses two `EmbeddedConnectCluster` instances, and when the first one is stopped it would reset the (static) exit procedures, and any problems during shutdown of the second embedded Connect cluster would cause the worker to shut down the JVM running the tests.

Instead, the `MirrorConnectorsIntegrationTest` class should mask the `Exit` procedures and instruct the `EmbeddedConnectClusters` instances (via the existing builder method) to not mask the procedures.

Author: Randall Hauch <rhauch@gmail.com>
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2020-12-07 09:34:34 -06:00
Samuel Cantero 1625984149
MINOR: Allow Checkpoints for consumers using static partition assignments (#9545)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2020-11-19 15:01:22 +00:00
Xavier Léauté 60d002f9eb
KAFKA-10572 mirror-maker config changes for KIP-629 (#9429)
Author: Xavier Léauté <xavier@confluent.io>
Reviewer: Randall Hauch <rhauch@gmail.com>
2020-10-20 09:15:14 -05:00
Mickael Maison 270881cd65
KAFKA-10332: Update MM2 refreshTopicPartitions() logic (#9343)
Trigger task reconfiguration when:
- topic-partitions are created or deleted on source cluster
- topic-partitions are missing on target cluster

Authors: Mickael Maison <mickael.maison@gmail.com>, Edoardo Comar <ecomar@uk.ibm.com>
Reviewer: Randall Hauch <rhauch@gmail.com>
2020-10-19 10:51:44 -05:00
Samuel Cantero cf202cb6ac
MINOR: Fix consumer/producer properties override (#9313)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ryanne Dolan <ryannedolan@gmail.com>
2020-10-16 18:17:03 +02:00
Ivan Yurchenko b5aa2691e9
MINOR: Fix MirrorConnectorsIntegrationTest (#9341)
In `setup()`, `primary` was checked to be running twice, instead of `backup`.

Reviewers: Mickael Maison <mickael.maison@gmail.com>
2020-10-02 18:50:22 +02:00
Andre Araujo 6b95f1ec57
KAFKA-10048: Possible data gap for a consumer after a failover when using MM2 (#8730)
Ensure that the MM2 checkpoint mirror task replicates consumer offsets even when they are
zero to avoid issues with consumers after failovers.

Author: Andre Araujo <asdaraujo@gmail.com>

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ryanne Dolan <ryannedolan@gmail.com>, Edoardo Comar <ecomar@uk.ibm.com>, heritamas
2020-10-02 18:34:50 +02:00
Lucas Bradstreet 8e9cf0e0cb
MINOR: mirror integration tests should not call System.exit (#9200)
Reviewers: Ron Dagostino <rdagostino@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
2020-09-09 16:07:36 +01:00
Ning Zhang c6d135e21f
KAFKA-10133: MM2 readme update on config (#9215)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ryanne Dolan <ryannedolan@gmail.com>
2020-09-04 19:26:23 +02:00
showuon 43a8b4458d
KAFKA-10255: Fix flaky testOneWayReplicationWithAutoOffsetSync test (#9029)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ning Zhang
2020-07-30 22:01:20 +01:00
Mickael Maison caa806cd82
KAFKA-10232: MirrorMaker2 internal topics Formatters KIP-597 (#8604)
This PR includes 3 MessageFormatters for MirrorMaker2 internal topics:
- HeartbeatFormatter
- CheckpointFormatter
- OffsetSyncFormatter

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

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

Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
Co-authored-by: Edoardo Comar <ecomar@uk.ibm.com>
2020-07-03 10:41:45 +01:00
showuon 3624daef2a
KAFKA-9509: Increase timeout when consuming records to fix flaky test in MM2 (#8894)
A simple increase in the timeout of the consumer that verifies that records have been replicated seems to fix the integration tests in `MirrorConnectorsIntegrationTest` that have been failing more often recently. 

Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Sanjana Kaundinya <skaundinya@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>
2020-06-29 13:22:08 -07:00
Ning Zhang 9c9a79b459
KAFKA-9076: support consumer sync across clusters in MM 2.0 (#7577)
In order to make the Kafka consumer and stream application migrate from source to target cluster
transparently and conveniently, e.g. in event of source cluster failure, a background job is proposed
to periodically sync the consumer offsets from the source to target cluster, so that when the
consumer and stream applications switche to the target cluster, they will resume to consume from
where they left off at source cluster.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ryanne Dolan <ryannedolan@gmail.com>, Thiago Pinto, Srinivas Boga
2020-06-26 16:17:29 +01:00
Mandar Tillu bdae26d047
Fixing KAFKA-10094 (#8797) 2020-06-05 15:56:32 -07:00
xiaodongdu 9c833f665f
KAFKA-9960: implement KIP-606 to add metadata context to MetricsReporter (#8691)
Implemented KIP-606 to add metadata context to MetricsReporter.

Author: Xiaodong Du <xdu@confluent.io>
Reviewers: David Arthur <mumrah@gmail.com>, Randall Hauch <rhauch@gmail.com>, Xavier Léauté <xavier@confluent.io>, Ryan Pridgeon <ryan.n.pridgeon@gmail.com>
2020-05-27 20:18:36 -05:00
Mario Molina 856e366512
KAFKA-9780: Deprecate commit records without record metadata (#8379)
Author: Mario Molina <mmolimar@gmail.com>
Reviewer: Randall Hauch <rhauch@gmail.com>
2020-05-21 14:18:09 -05:00
Randall Hauch d4ffc24a02
MINOR: Correct MirrorMaker2 integration test configs for Connect internal topics (#8653)
The replication factor for the Connect workers’ internal topics were incorrectly named and not actually used.
2020-05-21 10:56:04 -05:00
Chris Egerton ab4b4d7372
KAFKA-9950: Construct new ConfigDef for MirrorTaskConfig before defining new properties (#8608)
`MirrorTaskConfig` class mutates the `ConfigDef` by defining additional properties, which leads to a potential `ConcurrentModificationException` during worker configuration validation and unintended inclusion of those new properties in the `ConfigDef` for the connectors which in turn is then visible via the REST API's `/connectors/{name}/config/validate` endpoint.

The fix here is a one-liner that just creates a copy of the `ConfigDef` before defining new properties.

Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>
2020-05-20 21:14:40 -07:00
Jeff Widman 9886d67999
MINOR: Fix typos in config properties in MM2 test (#8561)
Fixed typos in two MM2 configs that define the replication factor for internal Connect topics.
Only a single test was affected. 

Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>
2020-04-27 23:10:08 -07:00
Mickael Maison 3df5464fca
MINOR: Fix a number of warnings in mirror/mirror-client (#8074)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Ryanne Dolan <ryannedolan@gmail.com>, Andrew Choi <a24choi@edu.uwaterloo.ca>
2020-03-26 11:59:21 +00:00
Konstantine Karantasis 406635bcc9
MINOR: Use Exit.exit instead of System.exit in MM2 (#8321)
Exit.exit needs to be used in code instead of System.exit.

Particularly in integration tests using System.exit is disrupting because it exits the jvm process and does not just fail the test correctly. Integration tests override procedures in Exit to protect against such cases.

Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Randall Hauch <rhauch@gmail.com>
2020-03-20 16:39:17 -07:00
Xavier Léauté 7e1c39f75a
KAFKA-9106 make metrics exposed via jmx configurable (#7674)
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Rajini Sivaram <rajinisivaram@googlemail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
2020-02-13 10:21:14 -08:00
Konstantine Karantasis 97d2c726f1
MINOR: Small Connect integration test fixes (#8100)
Author: Konstantine Karantasis <konstantine@confluent.io>
Reviewer: Randall Hauch <rhauch@gmail.com>
2020-02-12 17:40:37 -06:00