You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/08/29 20:06:27 UTC

[GitHub] [kafka] cmccabe opened a new pull request, #12571: KAFKA-14187: kafka-featuressh: add support for --metadata

cmccabe opened a new pull request, #12571:
URL: https://github.com/apache/kafka/pull/12571

   This PR adds support to kafka-features.sh for the --metadata flag, as specified in KIP-778.  This
   flag makes it possible to upgrade to a new metadata version without consulting a table mapping
   version names to short integers.
   
   FeatureCommandTest.scala: make most tests here true unit tests (that don't start brokers) in order
   to improve test run time, and allow us to test more cases. For the integration test part, test both
   KRaft and ZK-based clusters. Add support for mocking feature operations in MockAdminClient.java.
   
   upgrade.html: add a section describing how the metadata.version should be upgraded in KRaft
   clusters.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jsancio commented on a diff in pull request #12571: KAFKA-14187: kafka-featuressh: add support for --metadata

Posted by GitBox <gi...@apache.org>.
jsancio commented on code in PR #12571:
URL: https://github.com/apache/kafka/pull/12571#discussion_r957849143


##########
docs/upgrade.html:
##########
@@ -2040,6 +2059,6 @@ <h4><a id="upgrade_7" href="#upgrade_7">Upgrading from 0.7</a></h4>
 
 Release 0.7 is incompatible with newer releases. Major changes were made to the API, ZooKeeper data structures, and protocol, and configuration in order to add replication (Which was missing in 0.7). The upgrade from 0.7 to later versions requires a <a href="https://cwiki.apache.org/confluence/display/KAFKA/Migrating+from+0.7+to+0.8">special tool</a> for migration. This migration can be done without downtime.
 
-</script>

Review Comment:
   Thanks for this fix. I think we need to close the `<script>` element on line 20.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dengziming commented on a diff in pull request #12571: KAFKA-14187: kafka-featuressh: add support for --metadata

Posted by GitBox <gi...@apache.org>.
dengziming commented on code in PR #12571:
URL: https://github.com/apache/kafka/pull/12571#discussion_r957966172


##########
docs/upgrade.html:
##########
@@ -61,8 +61,27 @@ <h4><a id="upgrade_3_3_0" href="#upgrade_3_3_0">Upgrading to 3.3.0 from any vers
     </li>
 </ol>
 
+<h4><a id="upgrade_3_3_0" href="#upgrade_3_3_0">Upgrading a KRaft-based cluster to 3.3.0 from any version 3.0.x through 3.2.x</a></h4>
+
+<p><b>If you are upgrading from a version prior to 3.3.0, please see the note below. Once you have changed the metadata.version to the latest version, it will not be possible to downgrade to a version prior to 3.3-IV0.</b></p>
+
+<p><b>For a rolling upgrade:</b></p>
+
+<ol>
+    <li>Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the
+        brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations.
+    </li>
+    <li>Once the cluster's behavior and performance has been verified, bump the metadata.version by running
+        <code>
+        ./bin/kafka-features.sh upgrade --metadata 3.3

Review Comment:
   We can use `bin/kafka-features.sh upgrade --feature metadata.version --version 7` to upgrade metadata, are we adding --metadata because it's more readable?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mumrah commented on a diff in pull request #12571: KAFKA-14187: kafka-featuressh: add support for --metadata

Posted by GitBox <gi...@apache.org>.
mumrah commented on code in PR #12571:
URL: https://github.com/apache/kafka/pull/12571#discussion_r957804548


##########
clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java:
##########
@@ -995,12 +1036,78 @@ public DescribeMetadataQuorumResult describeMetadataQuorum(DescribeMetadataQuoru
 
     @Override
     public DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options) {
-        throw new UnsupportedOperationException("Not implemented yet");
+        Map<String, FinalizedVersionRange> finalizedFeatures = new HashMap<>();
+        Map<String, SupportedVersionRange> supportedFeatures = new HashMap<>();
+        for (Map.Entry<String, Short> entry : featureLevels.entrySet()) {
+            finalizedFeatures.put(entry.getKey(), new FinalizedVersionRange(
+                    entry.getValue(), entry.getValue()));
+            supportedFeatures.put(entry.getKey(), new SupportedVersionRange(
+                    minSupportedFeatureLevels.get(entry.getKey()),
+                    maxSupportedFeatureLevels.get(entry.getKey())));
+        }
+        return new DescribeFeaturesResult(KafkaFuture.completedFuture(
+                new FeatureMetadata(finalizedFeatures,
+                    Optional.of(123L),
+                    supportedFeatures)));
     }
 
     @Override
-    public UpdateFeaturesResult updateFeatures(Map<String, FeatureUpdate> featureUpdates, UpdateFeaturesOptions options) {
-        throw new UnsupportedOperationException("Not implemented yet");
+    public UpdateFeaturesResult updateFeatures(
+        Map<String, FeatureUpdate> featureUpdates,
+        UpdateFeaturesOptions options
+    ) {
+        Map<String, KafkaFuture<Void>> results = new HashMap<>();
+        for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            KafkaFutureImpl<Void> future = new KafkaFutureImpl<Void>();
+            String feature = entry.getKey();
+            try {
+                short cur = featureLevels.getOrDefault(feature, (short) 0);
+                short next = entry.getValue().maxVersionLevel();
+                short min = minSupportedFeatureLevels.getOrDefault(feature, (short) 0);
+                short max = maxSupportedFeatureLevels.getOrDefault(feature, (short) 0);
+                switch (entry.getValue().upgradeType()) {
+                    case UNKNOWN:
+                        throw new InvalidRequestException("Invalid upgrade type.");
+                    case UPGRADE:
+                        if (cur > next) {
+                            throw new InvalidUpdateVersionException("Can't upgrade to lower version.");
+                        }
+                        break;
+                    case SAFE_DOWNGRADE:
+                        if (cur < next) {
+                            throw new InvalidUpdateVersionException("Can't downgrade to newer version.");
+                        }
+                        break;
+                    case UNSAFE_DOWNGRADE:
+                        if (cur < next) {
+                            throw new InvalidUpdateVersionException("Can't downgrade to newer version.");
+                        }
+                        while (next != cur) {

Review Comment:
   Can you help me understand this bit? Is it just here to create some artificial error for the test?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] cmccabe commented on a diff in pull request #12571: KAFKA-14187: kafka-featuressh: add support for --metadata

Posted by GitBox <gi...@apache.org>.
cmccabe commented on code in PR #12571:
URL: https://github.com/apache/kafka/pull/12571#discussion_r958914091


##########
clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java:
##########
@@ -36,10 +36,10 @@ public class SupportedVersionRange {
      * @throws IllegalArgumentException   Raised when the condition described above is not met.
      */
     SupportedVersionRange(final short minVersion, final short maxVersion) {
-        if (minVersion < 1 || maxVersion < 1 || maxVersion < minVersion) {
+        if (minVersion < 0 || maxVersion < 0 || maxVersion < minVersion) {

Review Comment:
   It is meaningful. If SupportedVersionRange includes 0, that means the feature can be turned off. If SupportedVersionRange does not include 0, the feature can't be turned off (for example, metadata.version on KRaft brokers cannot be 0)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] cmccabe commented on a diff in pull request #12571: KAFKA-14187: kafka-featuressh: add support for --metadata

Posted by GitBox <gi...@apache.org>.
cmccabe commented on code in PR #12571:
URL: https://github.com/apache/kafka/pull/12571#discussion_r958710277


##########
docs/upgrade.html:
##########
@@ -61,8 +61,27 @@ <h4><a id="upgrade_3_3_0" href="#upgrade_3_3_0">Upgrading to 3.3.0 from any vers
     </li>
 </ol>
 
+<h4><a id="upgrade_3_3_0" href="#upgrade_3_3_0">Upgrading a KRaft-based cluster to 3.3.0 from any version 3.0.x through 3.2.x</a></h4>
+
+<p><b>If you are upgrading from a version prior to 3.3.0, please see the note below. Once you have changed the metadata.version to the latest version, it will not be possible to downgrade to a version prior to 3.3-IV0.</b></p>
+
+<p><b>For a rolling upgrade:</b></p>
+
+<ol>
+    <li>Upgrade the brokers one at a time: shut down the broker, update the code, and restart it. Once you have done so, the
+        brokers will be running the latest version and you can verify that the cluster's behavior and performance meets expectations.
+    </li>
+    <li>Once the cluster's behavior and performance has been verified, bump the metadata.version by running
+        <code>
+        ./bin/kafka-features.sh upgrade --metadata 3.3

Review Comment:
   Yes, `--metadata` is more readable / usable.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] cmccabe commented on a diff in pull request #12571: KAFKA-14187: kafka-featuressh: add support for --metadata

Posted by GitBox <gi...@apache.org>.
cmccabe commented on code in PR #12571:
URL: https://github.com/apache/kafka/pull/12571#discussion_r957807838


##########
clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java:
##########
@@ -995,12 +1036,78 @@ public DescribeMetadataQuorumResult describeMetadataQuorum(DescribeMetadataQuoru
 
     @Override
     public DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options) {
-        throw new UnsupportedOperationException("Not implemented yet");
+        Map<String, FinalizedVersionRange> finalizedFeatures = new HashMap<>();
+        Map<String, SupportedVersionRange> supportedFeatures = new HashMap<>();
+        for (Map.Entry<String, Short> entry : featureLevels.entrySet()) {
+            finalizedFeatures.put(entry.getKey(), new FinalizedVersionRange(
+                    entry.getValue(), entry.getValue()));
+            supportedFeatures.put(entry.getKey(), new SupportedVersionRange(
+                    minSupportedFeatureLevels.get(entry.getKey()),
+                    maxSupportedFeatureLevels.get(entry.getKey())));
+        }
+        return new DescribeFeaturesResult(KafkaFuture.completedFuture(
+                new FeatureMetadata(finalizedFeatures,
+                    Optional.of(123L),
+                    supportedFeatures)));
     }
 
     @Override
-    public UpdateFeaturesResult updateFeatures(Map<String, FeatureUpdate> featureUpdates, UpdateFeaturesOptions options) {
-        throw new UnsupportedOperationException("Not implemented yet");
+    public UpdateFeaturesResult updateFeatures(
+        Map<String, FeatureUpdate> featureUpdates,
+        UpdateFeaturesOptions options
+    ) {
+        Map<String, KafkaFuture<Void>> results = new HashMap<>();
+        for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            KafkaFutureImpl<Void> future = new KafkaFutureImpl<Void>();
+            String feature = entry.getKey();
+            try {
+                short cur = featureLevels.getOrDefault(feature, (short) 0);
+                short next = entry.getValue().maxVersionLevel();
+                short min = minSupportedFeatureLevels.getOrDefault(feature, (short) 0);
+                short max = maxSupportedFeatureLevels.getOrDefault(feature, (short) 0);
+                switch (entry.getValue().upgradeType()) {
+                    case UNKNOWN:
+                        throw new InvalidRequestException("Invalid upgrade type.");
+                    case UPGRADE:
+                        if (cur > next) {
+                            throw new InvalidUpdateVersionException("Can't upgrade to lower version.");
+                        }
+                        break;
+                    case SAFE_DOWNGRADE:
+                        if (cur < next) {
+                            throw new InvalidUpdateVersionException("Can't downgrade to newer version.");
+                        }
+                        break;
+                    case UNSAFE_DOWNGRADE:
+                        if (cur < next) {
+                            throw new InvalidUpdateVersionException("Can't downgrade to newer version.");
+                        }
+                        while (next != cur) {

Review Comment:
   yes, this is to artificially mark some versions as requiring an unsafe downgrade.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mumrah commented on a diff in pull request #12571: KAFKA-14187: kafka-featuressh: add support for --metadata

Posted by GitBox <gi...@apache.org>.
mumrah commented on code in PR #12571:
URL: https://github.com/apache/kafka/pull/12571#discussion_r958851048


##########
clients/src/main/java/org/apache/kafka/clients/admin/SupportedVersionRange.java:
##########
@@ -36,10 +36,10 @@ public class SupportedVersionRange {
      * @throws IllegalArgumentException   Raised when the condition described above is not met.
      */
     SupportedVersionRange(final short minVersion, final short maxVersion) {
-        if (minVersion < 1 || maxVersion < 1 || maxVersion < minVersion) {
+        if (minVersion < 0 || maxVersion < 0 || maxVersion < minVersion) {

Review Comment:
   Do we need to expose version zero for some reason?
   
   If I understand correctly, this will let a feature define its range of supported versions to include zero which means "disabled". If the feature is not present, I think this could let a user "upgrade" to version zero which would have no effect. 
   
   In practice, we just have `metadata.version` which defines its supported versions starting with 1
   



##########
tests/kafkatest/tests/core/kraft_upgrade_test.py:
##########
@@ -0,0 +1,121 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from ducktape.mark import parametrize
+from ducktape.mark.resource import cluster
+from ducktape.utils.util import wait_until
+from kafkatest.services.console_consumer import ConsoleConsumer
+from kafkatest.services.kafka import KafkaService
+from kafkatest.services.kafka.quorum import remote_kraft, colocated_kraft
+from kafkatest.services.verifiable_producer import VerifiableProducer
+from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
+from kafkatest.utils import is_int
+from kafkatest.version import LATEST_3_0, LATEST_3_1, LATEST_3_2, DEV_BRANCH, \
+    KafkaVersion, LATEST_METADATA_VERSION
+
+#
+# Test upgrading between different KRaft versions.
+#
+# Note that the earliest supported KRaft version is 3.0, not 0.8 as it is for
+# ZK mode. The upgrade process is also somewhat different for KRaft because we
+# use metadata.version instead of inter.broker.protocol.
+#
+class TestKRaftUpgrade(ProduceConsumeValidateTest):
+
+    def __init__(self, test_context):
+        super(TestKRaftUpgrade, self).__init__(test_context=test_context)
+        self.may_truncate_acked_records = False
+
+    def setUp(self):
+        self.topic = "test_topic"
+        self.partitions = 3
+        self.replication_factor = 3
+
+        # Producer and consumer
+        self.producer_throughput = 1000
+        self.num_producers = 1
+        self.num_consumers = 1
+
+    def wait_until_rejoin(self):
+        for partition in range(0, self.partitions):
+            wait_until(lambda: len(self.kafka.isr_idx_list(self.topic, partition)) == self.replication_factor, timeout_sec=60,
+                    backoff_sec=1, err_msg="Replicas did not rejoin the ISR in a reasonable amount of time")
+
+    def perform_version_change(self, from_kafka_version):
+        self.logger.info("Performing rolling upgrade.")
+        for node in self.kafka.controller_quorum.nodes:
+            self.logger.info("Stopping controller node %s" % node.account.hostname)
+            self.kafka.controller_quorum.stop_node(node)
+            node.version = DEV_BRANCH
+            self.logger.info("Restarting controller node %s" % node.account.hostname)
+            self.kafka.controller_quorum.start_node(node)
+            self.wait_until_rejoin()
+            self.logger.info("Successfully restarted controller node %s" % node.account.hostname)
+        for node in self.kafka.nodes:
+            self.logger.info("Stopping broker node %s" % node.account.hostname)
+            self.kafka.stop_node(node)
+            node.version = DEV_BRANCH
+            self.logger.info("Restarting broker node %s" % node.account.hostname)
+            self.kafka.start_node(node)
+            self.wait_until_rejoin()
+            self.logger.info("Successfully restarted broker node %s" % node.account.hostname)
+        self.logger.info("Changing metadata.version to %s" % LATEST_METADATA_VERSION)
+        self.kafka.upgrade_metadata_version(LATEST_METADATA_VERSION)
+
+    def run_upgrade(self, from_kafka_version):
+        """Test upgrade of Kafka broker cluster from various versions to the current version
+
+        from_kafka_version is a Kafka version to upgrade from.
+
+        - Start 3 node broker cluster on version 'from_kafka_version'.
+        - Start producer and consumer in the background.
+        - Perform rolling upgrade.
+        - Upgrade cluster to the latest metadata.version.
+        - Finally, validate that every message acked by the producer was consumed by the consumer.
+        """
+        fromKafkaVersion = KafkaVersion(from_kafka_version)
+        self.kafka = KafkaService(self.test_context,
+                                  num_nodes=3,
+                                  zk=None,
+                                  version=fromKafkaVersion,
+                                  topics={self.topic: {"partitions": self.partitions,
+                                                       "replication-factor": self.replication_factor,
+                                                       'configs': {"min.insync.replicas": 2}}})
+        self.kafka.start()
+        self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka,
+                                           self.topic, throughput=self.producer_throughput,
+                                           message_validator=is_int,
+                                           compression_types=["none"],
+                                           version=KafkaVersion(from_kafka_version))
+        self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka,
+                                        self.topic, new_consumer=True, consumer_timeout_ms=30000,
+                                        message_validator=is_int, version=KafkaVersion(from_kafka_version))
+        self.run_produce_consume_validate(core_test_action=lambda: self.perform_version_change(from_kafka_version))
+        cluster_id = self.kafka.cluster_id()
+        assert cluster_id is not None
+        assert len(cluster_id) == 22
+        assert self.kafka.check_protocol_errors(self)
+
+    @cluster(num_nodes=5)
+    @parametrize(from_kafka_version=str(LATEST_3_1), metadata_quorum=colocated_kraft)
+    @parametrize(from_kafka_version=str(LATEST_3_2), metadata_quorum=colocated_kraft)
+    def test_colocated_upgrade(self, from_kafka_version, metadata_quorum):
+        self.run_upgrade(from_kafka_version)
+
+    @cluster(num_nodes=8)
+    @parametrize(from_kafka_version=str(LATEST_3_1), metadata_quorum=remote_kraft)
+    @parametrize(from_kafka_version=str(LATEST_3_2), metadata_quorum=remote_kraft)
+    def test_non_colocated_upgrade(self, from_kafka_version, metadata_quorum):
+        self.run_upgrade(from_kafka_version)

Review Comment:
   nit: newline



##########
clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java:
##########
@@ -995,12 +1036,78 @@ public DescribeMetadataQuorumResult describeMetadataQuorum(DescribeMetadataQuoru
 
     @Override
     public DescribeFeaturesResult describeFeatures(DescribeFeaturesOptions options) {
-        throw new UnsupportedOperationException("Not implemented yet");
+        Map<String, FinalizedVersionRange> finalizedFeatures = new HashMap<>();
+        Map<String, SupportedVersionRange> supportedFeatures = new HashMap<>();
+        for (Map.Entry<String, Short> entry : featureLevels.entrySet()) {
+            finalizedFeatures.put(entry.getKey(), new FinalizedVersionRange(
+                    entry.getValue(), entry.getValue()));
+            supportedFeatures.put(entry.getKey(), new SupportedVersionRange(
+                    minSupportedFeatureLevels.get(entry.getKey()),
+                    maxSupportedFeatureLevels.get(entry.getKey())));
+        }
+        return new DescribeFeaturesResult(KafkaFuture.completedFuture(
+                new FeatureMetadata(finalizedFeatures,
+                    Optional.of(123L),
+                    supportedFeatures)));
     }
 
     @Override
-    public UpdateFeaturesResult updateFeatures(Map<String, FeatureUpdate> featureUpdates, UpdateFeaturesOptions options) {
-        throw new UnsupportedOperationException("Not implemented yet");
+    public UpdateFeaturesResult updateFeatures(
+        Map<String, FeatureUpdate> featureUpdates,
+        UpdateFeaturesOptions options
+    ) {
+        Map<String, KafkaFuture<Void>> results = new HashMap<>();
+        for (Map.Entry<String, FeatureUpdate> entry : featureUpdates.entrySet()) {
+            KafkaFutureImpl<Void> future = new KafkaFutureImpl<Void>();
+            String feature = entry.getKey();
+            try {
+                short cur = featureLevels.getOrDefault(feature, (short) 0);
+                short next = entry.getValue().maxVersionLevel();
+                short min = minSupportedFeatureLevels.getOrDefault(feature, (short) 0);
+                short max = maxSupportedFeatureLevels.getOrDefault(feature, (short) 0);
+                switch (entry.getValue().upgradeType()) {
+                    case UNKNOWN:
+                        throw new InvalidRequestException("Invalid upgrade type.");
+                    case UPGRADE:
+                        if (cur > next) {
+                            throw new InvalidUpdateVersionException("Can't upgrade to lower version.");
+                        }
+                        break;
+                    case SAFE_DOWNGRADE:
+                        if (cur < next) {
+                            throw new InvalidUpdateVersionException("Can't downgrade to newer version.");
+                        }
+                        break;
+                    case UNSAFE_DOWNGRADE:
+                        if (cur < next) {
+                            throw new InvalidUpdateVersionException("Can't downgrade to newer version.");
+                        }
+                        while (next != cur) {
+                            if (cur % 2 == 0) {

Review Comment:
   Could you add a small comment explaining this bit?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] cmccabe merged pull request #12571: KAFKA-14187: kafka-featuressh: add support for --metadata

Posted by GitBox <gi...@apache.org>.
cmccabe merged PR #12571:
URL: https://github.com/apache/kafka/pull/12571


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org