Commit Graph

11741 Commits

Author SHA1 Message Date
Luke Chen c17c9f0a6c KAFKA-15421: fix network thread leak in testThreadPoolResize (#14320)
In SocketServerTest, we create SocketServer and enableRequestProcessing on each test class initialization. That's fine since we shutdown it in @AfterEach. The issue we have is we disabled 2 tests in this test suite. And when running these disabled tests, we will go through class initialization, but without @AfterEach. That causes 2 network thread leaked.

Compared the error message in DynamicBrokerReconfigurationTest#testThreadPoolResize test here:

org.opentest4j.AssertionFailedError: Invalid threads: expected 6, got 8: List(data-plane-kafka-socket-acceptor-ListenerName(INTERNAL)-SSL-0, data-plane-kafka-socket-acceptor-ListenerName(PLAINTEXT)-PLAINTEXT-0, data-plane-kafka-socket-acceptor-ListenerName(INTERNAL)-SSL-0, data-plane-kafka-socket-acceptor-ListenerName(EXTERNAL)-SASL_SSL-0, data-plane-kafka-socket-acceptor-ListenerName(INTERNAL)-SSL-0, data-plane-kafka-socket-acceptor-ListenerName(EXTERNAL)-SASL_SSL-0, data-plane-kafka-socket-acceptor-ListenerName(PLAINTEXT)-PLAINTEXT-0, data-plane-kafka-socket-acceptor-ListenerName(EXTERNAL)-SASL_SSL-0) ==> expected: <true> but was: <false>

The 2 unexpected network threads are leaked from SocketServerTest.

Reviewers: Satish Duggana <satishd@apache.org>, Christo Lolov <lolovc@amazon.com>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash <kchandraprakash@uber.com>, Chris Egerton <chrise@aiven.io>
2023-09-03 16:17:39 +08:00
Rohan 3f1a459873 KAFKA-15429: reset transactionInFlight on StreamsProducer close (#14326)
Resets the value of transactionInFlight to false when closing the
StreamsProducer. This ensures we don't try to commit against a
closed producer

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2023-09-02 18:16:39 -07:00
Rohan ac4bdb2111 KAFKA-15429: catch+log errors from unsubscribe in streamthread shutdown (#14325)
Preliminary fix for KAFKA-15429 which updates StreamThread.completeShutdown to
catch-and-log errors from consumer.unsubscribe. Though this does not prevent
the exception, it does preserve the original exception that caused the stream
thread to exit.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2023-09-02 18:16:39 -07:00
Christo Lolov 8eb9105e51 KAFKA-15427: Fix resource leak in integration tests for tiered storage (#14319)
Co-authored-by: Nikhil Ramakrishnan <nikrmk@amazon.com>

Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
2023-09-01 23:14:50 +05:30
Kamal Chandraprakash 771f14ca38 KAFKA-15351: Ensure log-start-offset not updated to local-log-start-offset when remote storage enabled (#14301)
When tiered storage is enabled on the topic, and the last-standing-replica is restarted, then the log-start-offset should not reset its offset to first-local-log-segment-base-offset.

Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>
2023-09-01 06:34:05 +05:30
Kamal Chandraprakash b6c5ac0913 KAFKA-15404: Disable the flaky integration tests. (#14296)
Disabled the below tests to fix the thread leak:

1. kafka.server.DynamicBrokerReconfigurationTest.testThreadPoolResize() and
2. org.apache.kafka.tiered.storage.integration.OffloadAndConsumeFromLeaderTest

Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Justine Olshan <jolshan@confluent.io>
2023-09-01 06:20:11 +05:30
Luke Chen e0382dcd32 MINOR: Close topic based RLMM correctly in integration tests (#14315)
Reviewers: Divij Vaidya <diviv@amazon.com>
2023-08-31 08:47:14 +00:00
A. Sophie Blee-Goldman 0f3ebdac97 HOTFIX: avoid placement of unnecessary transient standby tasks & improve assignor logging (#14149)
Minor fix to avoid creating unnecessary standby tasks, especially when these may be surprising or unexpected as in the case of an application with num.standby.replicas = 0 and warmup replicas disabled.

The "bug" here was introduced during the fix for an issue with cooperative rebalancing and in-memory stores. The fundamental problem is that in-memory stores cannot be unassigned from a consumer for any period, however temporary, without being closed and losing all the accumulated state. This caused some grief when the new HA task assignor would assign an active task to a node based on the readiness of the standby version of that task, but would have to remove the active task from the initial assignment so it could first be revoked from its previous owner, as per the cooperative rebalancing protocol. This temporary gap in any version of that task among the consumer's assignment for that one intermediate rebalance would end up causing the consumer to lose all state for it, in the case of in-memory stores.

To fix this, we simply began to place standby tasks on the intended recipient of an active task awaiting revocation by another consumer. However, the fix was a bit of an overreach, as we assigned these temporary standby tasks in all cases, regardless of whether there had previously been a standby version of that task. We can narrow this down without sacrificing any of the intended functionality by only assigning this kind of standby task where the consumer had previously owned some version of it that would otherwise potentially be lost.

Also breaks up some of the long log lines in the StreamsPartitionAssignor and expands the summary info while moving it all to the front of the line (following reports of missing info due to truncation of long log lines in larger applications)
2023-08-30 13:31:06 -07:00
Vincent Jiang daea0fdff0 KAFKA-15375: fix broken clean shutdown detection logic in LogManager
When running in kraft mode, LogManager.startup is called in a different thread than the main broker (#14239)
startup thread (by BrokerMetadataPublisher when the first metadata update is received.) If a fatal
error happens during broker startup, before LogManager.startup is completed, LogManager.shutdown may
 mark log dirs as clean shutdown improperly.

This PR includes following change:
1. During LogManager startup time:
  - track hadCleanShutdwon info for each log dir
  - track loadLogsCompleted status for each log dir
2. During LogManager shutdown time:
  - do not write clean shutdown marker file for log dirs which have hadCleanShutdown==false and loadLogsCompleted==false

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-08-30 11:57:35 -07:00
Christo Lolov 67c3d966f5 KAFKA-15267: Do not allow Tiered Storage to be disabled while topics have remote.storage.enable property (#14161)
The purpose of this change is to not allow a broker to start up with Tiered Storage disabled (remote.log.storage.system.enable=false) while there are still topics that have 'remote.storage.enable' set.

Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
2023-08-30 05:35:09 +05:30
Kamal Chandraprakash de7ee8a2de MINOR: Fix the TBRLMMRestart test. (#14297)
Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>
2023-08-28 20:25:26 +05:30
Kamal Chandraprakash 6d077eca9f KAFKA-15399: Enable OffloadAndConsumeFromLeader test (#14285)
Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
2023-08-28 10:34:26 +00:00
Luke Chen b6da42319b Merge branch '3.6' of https://github.com/apache/kafka into 3.6 2023-08-28 18:00:34 +08:00
Calvin Liu bee6c41ff8 KAFKA-15353: make sure AlterPartitionRequest.build() is idempotent (#14236)
As described in https://issues.apache.org/jira/browse/KAFKA-15353
When the AlterPartitionRequest version is < 3 and its builder.build is called multiple times, both newIsrWithEpochs and newIsr will all be empty. This can happen if the sender retires on errors.

Reviewers: Luke Chen <showuon@gmail.com>
2023-08-28 18:00:20 +08:00
iit2009060 5fd03ee140 KAFKA-15256: Adding reviewer as part of release announcement email template (#14288)
Reviewers: Divij Vaidya <diviv@amazon.com>, Philip Nee <pnee@confluent.io>
2023-08-28 09:40:04 +00:00
Gantigmaa Selenge 57c7be9f22 KAFKA-15294: Publish remote storage configs (#14266)
This change does the following:

1. Make RemoteLogManagerConfigs that are implemented public

2. Add tasks to generate html docs for the configs

3. Include config docs in the main site

Reviewers: Divij Vaidya <diviv@amazon.com>, Luke Chen <showuon@gmail.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
2023-08-28 08:39:30 +00:00
Abhijeet Kumar 51d39c53b2 KAFKA-15181: Wait for RemoteLogMetadataCache to initialize after assigning partitions (#14127)
This PR adds the following changes to the `TopicBasedRemoteLogMetadataManager`

1. Added a guard in RemoteLogMetadataCache so that the incoming request can be served from the cache iff the corresponding user-topic-partition is initalized
2. Improve error handling in ConsumerTask thread so that is not killed when there are errors in reading the internal topic
3. ConsumerTask initialization should handle the case when there are no records to read
and some other minor changes

Added Unit Tests for the changes

Co-authored-by: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>

Reviewers: Luke Chen <showuon@gmail.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
2023-08-26 05:55:36 +05:30
David Arthur 843ede494f KAFKA-15389: Don't publish until we have replayed at least one record (#14282)
When starting up a controller for the first time (i.e., with an empty log), it is possible for
MetadataLoader to publish an empty MetadataImage before the activation records of the controller
have been written. While this is not a bug, it could be confusing. This patch closes that gap by
waiting for at least one controller record to be committed before the MetadataLoader starts publishing
images.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-08-25 10:44:30 -07:00
Luke Chen 28736badcf MINOR: Use "add-exports" only when jdk >= 16 in minikdc (#14232)
Use "add-exports" only when jdk >= 16 in minikdc

Reviewers: Greg Harris <greg.harris@aiven.io>
2023-08-25 11:53:46 +08:00
Satish Duggana 31227857ae KAFKA-14888: Added remote log segments retention mechanism based on time and size. (#13561)
This change introduces a remote log segment segment retention cleanup mechanism.

RemoteLogManager runs retention cleanup activity tasks on each leader replica. It assesses factors such as overall size and retention duration, subsequently removing qualified segments from remote storage. This process also involves adjusting the log-start-offset within the UnifiedLog accordingly. It also cleans up the segments which have epochs earlier than the earliest leader epoch in the current leader. 

Co-authored-by: Satish Duggana <satishd@apache.org>
Co-authored-by: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>

Reviewers: Jun Rao <junrao@gmail.com>, Divij Vaidya <diviv@amazon.com, Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Christo Lolov <lolovc@amazon.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>, Alexandre Dupriez <alexandre.dupriez@gmail.com>, Nikhil Ramakrishnan <ramakrishnan.nikhil@gmail.com>
2023-08-25 05:30:49 +05:30
Yash Mayya 0279c1be7f KAFKA-15377: Don't expose externalized secret values in tasks-config API endpoint (#14244)
Reviewers: Greg Harris <greg.harris@aiven.io>
2023-08-24 15:45:38 -07:00
Greg Harris ef8831a59a KAFKA-15393: Improve shutdown behavior in MM2 integration tests (#14278)
Reviewers: Yash Mayya <yash.mayya@gmail.com>, Chris Egerton <chrise@aiven.io>
2023-08-24 12:28:10 -07:00
Kamal Chandraprakash df9f405ec8 KAFKA-15290: Handle topic-level dynamic remote storage enable configuration (#14238)
* KAFKA-15290: Handle topic-level dynamic remote log storage enable configuration.

To onboard existing topics to tiered storage, bootstrap the remote-log-components when updating the dynamic `remote.storage.enable` config on the topic.

Reviewers: Christo Lolov <lolovc@amazon.com>, Divij Vaidya <diviv@amazon.com, Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>
2023-08-24 21:15:12 +05:30
Kamal Chandraprakash a0605c5e11 KAFKA-15400: Use readLock when removing an item from the RemoteIndexCache (#14283)
- Caffeine cache is thread safe, we want to hold the writeLock only during the close.
- Fix the flaky tests

Reviewers: Divij Vaidya <diviv@amazon.com>
2023-08-24 11:51:36 +00:00
Mehari Beyene f91cb6b87b KAFKA-14991: KIP-937-Improve message timestamp validation (#14135)
This implementation introduces two new configurations `log.message.timestamp.before.max.ms` and `log.message.timestamp.after.max.ms` and deprecates `log.message.timestamp.difference.max.ms`.

The default value for all these three configs is maintained to be Long.MAX_VALUE for backward compatibility but with the newly added configurations we can have a finer control when validating message timestamps that are in the past and the future compared to the broker's timestamp.

To maintain backward compatibility if the default value of `log.message.timestamp.before.max.ms` is not changed, we are assuming users are still using the deprecated config `log.message.timestamp.difference.max.ms` and validation is done using its value. This ensures that existing customers who have customized the value of `log.message.timestamp.difference.max.ms` will continue to see no change in behavior.

Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>
2023-08-24 11:50:39 +00:00
Okada Haruki 09c074ad8a KAFKA-15391: Handle concurrent dir rename which makes log-dir to be offline unexpectedly (#14280)
A race condition between async flush and segment rename (for deletion purpose) might cause the entire log directory to be marked offline when we delete a topic. This PR fixes the bug by ignoring NoSuchFileException when we flush a directory.

Reviewers: Divij Vaidya <diviv@amazon.com>
2023-08-24 08:53:37 +00:00
Divij Vaidya 8ecbd0bc63 KAFKA-9926: Fix flaky PlaintextAdminIntegrationTest.testCreatePartitions (#14273)
Reviewers: Luke Chen <showuon@gmail.com>
2023-08-23 07:36:04 +00:00
Ron Dagostino 64087fd9f3 MINOR: Move delegation token support to Metadata Version 3.6-IV2 (#14270)
#14083 added support for delegation tokens in KRaft and attached that support to the existing
MetadataVersion 3.6-IV1. This patch moves that support into a separate MetadataVersion 3.6-IV2.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-08-22 16:20:27 -07:00
David Arthur 732b7b25b0 KAFKA-14538 Metadata transactions in MetadataLoader (#14208)
This PR contains three main changes:

- Support for transactions in MetadataLoader
- Abort in-progress transaction during controller failover
- Utilize transactions for ZK to KRaft migration

A new MetadataBatchLoader class is added to decouple the loading of record batches from the
publishing of metadata in MetadataLoader. Since a transaction can span across multiple batches (or
multiple transactions could exist within one batch), some buffering of metadata updates was needed
before publishing out to the MetadataPublishers. MetadataBatchLoader accumulates changes into a
MetadataDelta, and uses a callback to publish to the publishers when needed.

One small oddity with this approach is that since we can "splitting" batches in some cases, the
number of bytes returned in the LogDeltaManifest has new semantics. The number of bytes included in
a batch is now only included in the last metadata update that is published as a result of a batch.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-08-22 16:17:18 -07:00
Walker Carlson bae7ea5693 KAFKA-14936: fix grace period partition issue (#14269)
Move the store creation to builder pattern and recover mintimestamp

Reviewers: John Roesler<vvcephei@apache.org>, Bill Bejeck <bbejeck@gmail.com>
2023-08-21 16:10:40 -05:00
Ron Dagostino 01917450ac
MINOR: Enable delegation token system test for KRaft (#14268)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-08-21 13:04:00 -04:00
Justine Olshan f207a61e62 KAFKA-15380: Execute action queue after callback request (#14197)
KIP-890 part 1 introduced the callback request type. It is used to execute a callback after KafkaApis.handle has returned. We did not account for tryCompleteActions at the end of handle when making this change.

In tests, we saw produce p99 increase dramatically (likely because we have to wait for another request before we can complete DelayedProduce). As a result, we should add the tryCompleteActions after the callback as well. In testing, this improved the produce performance.

Reviewers: Artem Livshits <alivshits@confluent.io>, Jason Gustafson <jason@confluent.io>
2023-08-21 08:46:53 -07:00
David Mao eefa812453
MINOR: Delete unused class, LogOffsetMetadata toString formatting (#14246)
Noticed that there was a dangling unused class (LongRef, replaced by PrimitiveRef.LongRef), and the LogOffsetMetadata toString was a little oddly formatted.

Reviewers: Justine Olshan <jolshan@confluent.io>
2023-08-20 15:16:27 -07:00
Kamal Chandraprakash 6492164d9c
KAFKA-15167: Tiered Storage Test Harness Framework (#14116)
`TieredStorageTestHarness` is a base class for integration tests exercising the tiered storage functionality. This uses  `LocalTieredStorage` instance as the second-tier storage system and `TopicBasedRemoteLogMetadataManager` as the remote log metadata manager.

Co-authored-by: Alexandre Dupriez <alexandre.dupriez@gmail.com>
Co-authored-by: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2023-08-20 17:15:52 +05:30
Proven Provenzano c2759df067
KAFKA-15219: KRaft support for DelegationTokens (#14083)
Reviewers: David Arthur <mumrah@gmail.com>, Ron Dagostino <rndgstn@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Viktor Somogyi <viktor.somogyi@cloudera.com>
2023-08-19 14:01:08 -04:00
Bruno Cadonna 05c329e61f
KAFKA-10199: Change to RUNNING if no pending task to init exist (#14249)
A stream thread should only change to RUNNING if there are no
active tasks in restoration in the state updater and if there
are no pending tasks to recycle and to init.

Usually all pending tasks to init are added to the state updater
in the same poll iteration that handles the assignment. However,
if during an initialization of a task a LockException the task
is re-added to the tasks to init and initialization is retried
in the next poll iteration.

A LockException might occur when a state directory is still locked
by another thread, when the rebalance just happened.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Walker Carlson <wcarlson@confluent.io>
2023-08-19 19:00:23 +02:00
Bruno Cadonna 4c7e0a9fa6
MINOR: Decouple purging committed records from committing (#14227)
Currently, Kafka Streams only tries to purge records whose
offset are committed from a repartition topic when at
least one offset was committed in the current commit.
The coupling between committing some offsets and purging
records is not needed and might delay purging of records.
For example, if a in-flight call for purging records has not
completed yet when a commit happens, a new call
is not issued.
If then the earlier in-flight call for purging records
finally completes but the next commit does not commit any
offsets, Streams does not issue the call for purging records
whose offset were committed in the previous commit
because the purging call was still in-flight.

This change issues calls for purging records during any commit
if the purge interval passed, even if no offsets were committed
in the current commit.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Walker Carlson <wcarlson@confluent.io>
2023-08-19 12:13:30 +02:00
Walker Carlson d0b7677c2c
KAFKA-14936: Add restore logic (3/N) (#14027)
Added restore logic for the buffer in grace period joins.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bbejeck@gmail.com>
2023-08-18 22:00:04 -05:00
Greg Harris 82ae77f945
KAFKA-15226: Add connect-plugin-path and plugin.discovery system test (#14230)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-08-18 15:28:43 -07:00
Matthias J. Sax b36cf4ef97
HOTIFX: fix Kafka Streams upgrade path from 3.4 to 3.5 (#14103)
KIP-904 introduced a backward incompatible change that requires a 2-bounce rolling upgrade.
The new "3.4" upgrade config value is not recognized by `AssignorConfiguration` though and thus crashed Kafka Streams if use.

Reviewers: Farooq Qaiser <fqaiser94@gmail.com>, Bruno Cadonna <bruno@confluent.io>
2023-08-18 11:06:08 -07:00
David Arthur 3ad5f42f59
Handle case of default broker in config migration (#14237)
When collecting the set of broker IDs during the migration, don't try to parse the default broker resource `""` as a broker ID.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-08-18 12:44:01 -04:00
Lucas Brutschy ee036ed9ef
KAFKA-15319: Upgrade rocksdb to fix CVE-2022-37434 (#14216)
Rocksdbjni<7.9.2 is vulnerable to CVE-2022-37434 due to zlib 1.2.12

Reviewers: Divij Vaidya <diviv@amazon.com>, Bruno Cadonna <cadonna@apache.org>
2023-08-18 18:31:27 +02:00
DL1231 4f88fb28f3
KAFKA-15130: Delete remote segments when deleting a topic (#13947)
* Delete remote segments when deleting a topic

Co-authored-by: Kamal Chandraprakash <kchandraprakash@uber.com>
Co-authored-by: d00791190 <dinglan6@huawei.com>
2023-08-18 18:21:09 +05:30
José Armando García Sancio 3f4816dd3e
KAFKA-15345; KRaft leader notifies leadership when listener reaches epoch start (#14213)
In a non-empty log the KRaft leader only notifies the listener of leadership when it has read to the leader's epoch start offset. This guarantees that the leader epoch has been committed and that the listener has read all committed offsets/records.

Unfortunately, the KRaft leader doesn't do this when the log is empty. When the log is empty the listener is notified immediately when it has become leader. This makes the API inconsistent and harder to program against.

This change fixes that by having the KRaft leader wait for the listener's nextOffset to be greater than the leader's epochStartOffset before calling handleLeaderChange.

The RecordsBatchReader implementation is also changed to include control records. This makes it possible for the state machine learn about committed control records. This additional information can be used to compute the committed offset or for counting those bytes when determining when to snapshot the partition.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Jason Gustafson <jason@confluent.io>
2023-08-17 18:40:17 -07:00
Yash Mayya 7802c264c9
MINOR: Allow writing tombstone offsets for arbitrary partitions in the FileStreamSourceConnector (#14234)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-08-17 14:13:53 -04:00
Chris Egerton a253dc6643
KAFKA-15102: Add release notes about the replication.policy.internal.topic.separator.enabled property for MirrorMaker 2 (#14220)
Reviewers: Greg Harris <greg.harris@aiven.io>
2023-08-17 14:11:01 -04:00
Lucas Brutschy d85a700813
MINOR: Do not reuse admin client across tests (#14225)
Reusing an admin client across tests can cause false positives in leak checkers, so don't do it.

Reviewers: Divij Vaidya <diviv@amazon.com>, Matthias J. Sax <matthias@confluent.io>
2023-08-17 10:53:58 -07:00
Chris Egerton de409b389d
KAFKA-15177: Implement KIP-875 SourceConnector::alterOffset API in MirrorMaker 2 connectors (#14005)
Reviewers: Yash Mayya <yash.mayya@gmail.com>, Greg Harris <greg.harris@aiven.io>
2023-08-17 09:33:59 -04:00
Greg Harris 6bd17419b7
KAFKA-15228: Add sync-manifests command to connect-plugin-path (KIP-898) (#14195)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-08-16 11:37:33 -07:00
Greg Harris a9efca0bf6
KAFKA-14759: Move Mock, Schema, and Verifiable connectors to new test-plugins module (#13302)
Reviewers: Hector Geraldino <hgeraldino@gmail.com>, Chris Egerton <chrise@aiven.io>
2023-08-16 10:30:24 -07:00