Skip to content

Commit

Permalink
[improve][broker] GetPartitionMetadata fail also can produce messages (
Browse files Browse the repository at this point in the history
…apache#23050)

### Motivation

GetPartitionMetadata fail also can produce messages

- 'autoUpdatePartitionsInterval' will get partition metadata and will regularly detect partition changes

- if GetPartitionMetadata will return ServiceNotReady, client receive ServiceNotReady will close cnx

- if close the current cnx, all producers and consumers witch use this cnx will close and reconnect

(https://github.com/apache/pulsar/blob/5c6602cbb3660a696bf960f2847aac1a2ae037d2/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java#L323-L345)
- this will influence a lot of producers and consumers and if current time the zk not available and bundle cache not exist the topic's bundle metadata, the client can't send messages to broker because the producer lookup will fail

### Modifications

GetPartitionMetadata return MetadataError when throw MetadataStoreException

(cherry picked from commit 6fa3bcf)
  • Loading branch information
congbobo184 authored and lhotari committed Jul 29, 2024
1 parent 55d32f2 commit 3b3e90b
Show file tree
Hide file tree
Showing 2 changed files with 91 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -171,6 +171,7 @@
import org.apache.pulsar.common.util.netty.NettyChannelUtil;
import org.apache.pulsar.common.util.netty.NettyFutureUtil;
import org.apache.pulsar.functions.utils.Exceptions;
import org.apache.pulsar.metadata.api.MetadataStoreException;
import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID;
import org.apache.pulsar.transaction.coordinator.exceptions.CoordinatorException;
import org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStore;
Expand Down Expand Up @@ -629,7 +630,9 @@ protected void handlePartitionMetadataRequest(CommandPartitionedTopicMetadata pa
log.warn("Failed to get Partitioned Metadata [{}] {}: {}", remoteAddress,
topicName, ex.getMessage(), ex);
ServerError error = ServerError.ServiceNotReady;
if (ex instanceof RestException restException){
if (ex instanceof MetadataStoreException) {
error = ServerError.MetadataError;
} else if (ex instanceof RestException restException){
int responseCode = restException.getResponse().getStatus();
if (responseCode == NOT_FOUND.getStatusCode()){
error = ServerError.TopicNotFound;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,87 @@
/*
* 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.
*/
package org.apache.pulsar.broker.zookeeper;

import com.google.common.collect.Sets;
import org.apache.pulsar.broker.MetadataSessionExpiredPolicy;
import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.TenantInfoImpl;
import org.apache.zookeeper.KeeperException;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;

import java.util.concurrent.TimeUnit;


@Test
public class ZKReconnectTest extends MockedPulsarServiceBaseTest {

@BeforeMethod(alwaysRun = true)
@Override
protected void setup() throws Exception {
this.conf.setZookeeperSessionExpiredPolicy(MetadataSessionExpiredPolicy.reconnect);
this.internalSetup();
admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build());
admin.tenants().createTenant("public",
new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test")));
admin.namespaces().createNamespace("public/default");
admin.namespaces().setNamespaceReplicationClusters("public/default", Sets.newHashSet("test"));
}

@Test
public void testGetPartitionMetadataFailAlsoCanProduceMessage() throws Exception {

pulsarClient = PulsarClient.builder().
serviceUrl(pulsar.getBrokerServiceUrl())
.build();

String topic = "testGetPartitionMetadataFailAlsoCanProduceMessage";
admin.topics().createPartitionedTopic(topic, 5);
Producer<byte[]> producer = pulsarClient.newProducer()
.autoUpdatePartitionsInterval(1, TimeUnit.SECONDS).topic(topic).create();

this.mockZooKeeper.setAlwaysFail(KeeperException.Code.SESSIONEXPIRED);

// clear cache
pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources()
.getCache().delete("/admin/partitioned-topics/public/default/persistent"
+ "/testGetPartitionMetadataFailAlsoCanProduceMessage");
pulsar.getNamespaceService().getOwnershipCache().invalidateLocalOwnerCache();

// autoUpdatePartitions 1 second
TimeUnit.SECONDS.sleep(3);

// also can send message
producer.send("test".getBytes());
this.mockZooKeeper.unsetAlwaysFail();
producer.send("test".getBytes());
producer.close();
}


@AfterMethod(alwaysRun = true)
@Override
protected void cleanup() throws Exception {
this.internalCleanup();
}
}

0 comments on commit 3b3e90b

Please sign in to comment.