Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

In some instances, producers fail to create after a topic is scheduled onto previously used broker #6416

Closed
addisonj opened this issue Feb 25, 2020 · 4 comments · Fixed by #6881
Labels
type/bug The PR fixed a bug or issue reported a bug

Comments

@addisonj
Copy link
Contributor

Describe the bug
When a topic is unloaded from a broker, a producer can attempt to reconnect before the topic is scheduled to a new broker but after the topic is unloaded. In some instances (discussed more below) this re-connection results in a producer that fails to create or error out and leaves a danging producerFuture that is never resolved or removed (unless the client terminates it's connection). If the topic ends up back on the same broker (either immediately or after another unload occurs from another broker) any producers that try and connect will immediately fail as the existing producerFuture is found and responds to the client with an error.

In #5571, an epoch was added to help address some related issues, however, in this case, as seen in

CompletableFuture<Producer> producerFuture = new CompletableFuture<>();
CompletableFuture<Producer> existingProducerFuture = producers.putIfAbsent(producerId,
producerFuture);
if (existingProducerFuture != null) {
if (existingProducerFuture.isDone() && !existingProducerFuture.isCompletedExceptionally()) {
Producer producer = existingProducerFuture.getNow(null);
log.info("[{}] Producer with the same id {} is already created: {}", remoteAddress,
producerId, producer);
ctx.writeAndFlush(Commands.newProducerSuccess(requestId, producer.getProducerName(),
producer.getSchemaVersion()));
return null;
} else {
// There was an early request to create a producer with
// same producerId. This can happen when
// client
// timeout is lower the broker timeouts. We need to wait
// until the previous producer creation
// request
// either complete or fails.
ServerError error = null;
if(!existingProducerFuture.isDone()) {
error = ServerError.ServiceNotReady;
}else {
error = getErrorCode(existingProducerFuture);
// remove producer with producerId as it's already completed with exception
producers.remove(producerId);
}
log.warn("[{}][{}] Producer with id {} is already present on the connection", remoteAddress,
producerId, topicName);
ctx.writeAndFlush(Commands.newError(requestId, error,
"Producer is already present on the connection"));
return null;
}
}
, the producerFuture is retrieved based solely on the producerId which the client provides. Because of this, the epoch isn't useful to solve the problem.

As to the details of why the producer fails initially, what we see is the following:

  1. A topic is unloaded
  2. The producer tries to re-connect
  3. We call into org.apache.pulsar.broker.service.BrokerService#checkTopicNsOwnership (
    public void checkTopicNsOwnership(final String topic) throws RuntimeException {
    ) and an exception is thrown (we see this warning in our logs)
  4. In some instances, this exception doesn't appear to bubble up to the following code
    return topics.computeIfAbsent(topic, (topicName) -> {
    return isPersistentTopic ? this.loadOrCreatePersistentTopic(topicName, createIfMissing)
    : createNonPersistentTopic(topicName);
    });
    } catch (IllegalArgumentException e) {
    log.warn("[{}] Illegalargument exception when loading topic", topic, e);
    return failedFuture(e);
    } catch (RuntimeException e) {
    Throwable cause = e.getCause();
    if (cause instanceof ServiceUnitNotReadyException) {
    log.warn("[{}] Service unit is not ready when loading the topic", topic);
    (we often don't see the "Service unit is not ready" log when it breaks, however, in some cases we don't see this logs and still have a proper exception sent to the client)
  5. This apparent break in exception handling (or control flow in general) appears to cause the getOrCreateTopic at
    service.getOrCreateTopic(topicName.toString()).thenAccept((Topic topic) -> {
    to never return and the producerFuture to never be resolved. We see this on the client side by a timeout to create a producer in cases where it fails, in other instances the timeout doesn't occur and we instead get a proper error message to the client, indicating that the namespace bundle isn't owned. As best as I can tell, this doesn't have anything to do with the pendingTopicLoadingQueue as we don't see enough topics being loaded that would put us into needing to use that queue.
  6. Now, with the dangling producerFuture in place, once the topic is re-scheduled back to this broker, it will be unable to ever create a producer.

I honestly have no idea why or how the call to getOrCreateTopic appears to never return, but we have about 10 instances we have validated with our logs (attached an instance below) that all show this same pattern and also have heap dumps (can share if desired) that also support this.

We have seen some other issues like #6054 (they haven't re-occurred often enough to gather enough data) but as best as we can tell it may be a related issue, where some call to load the topic never completes.

While there is likely some easy workarounds for this specific issue (such as putting a timeout on the producerFuture or by using the epoch in conjunction with the producerId for the producers map), this is really a spooky one, and I wonder if related issues could have a similar root cause!

To Reproduce
Since this appears to be down to a timing issue, it is hard to reliably reproduce, however, the following should work with enough tries:

  1. Have a producer, producing to a topic with a low reconnect timeout
  2. Unload the topic/bundle
  3. Observe that the producer gets a timeout on creating a producer
  4. Unload the topic/bundle again until it makes it back to the initial broker
  5. Observe that you can't get a new producer

Expected behavior
We expect to be able to create a producer after an unload.

Logs
These aren't full logs, just the relevant lines for when a producerFuture is left behind

Bad Instance

Broker Logs

{"msg":"17:11:16.978 [bookkeeper-ml-workers-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.BrokerService - Created topic persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails - dedup is disabled\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-2r8zr","container_name":"broker"},"docker":{"container_id":"6cc06889852e17619b2520562cdb2eba70b37d5e318210f0df33ee868378fe8d"}}
{"msg":"17:11:16.978 [bookkeeper-ml-workers-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.persistent.PersistentTopic - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails] There are no replicated subscriptions on the topic\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-2r8zr","container_name":"broker"},"docker":{"container_id":"6cc06889852e17619b2520562cdb2eba70b37d5e318210f0df33ee868378fe8d"}}
{"msg":"17:11:16.978 [bookkeeper-ml-workers-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.persistent.SubscribeRateLimiter - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails] configured subscribe-dispatch rate at broker SubscribeRate{subscribeThrottlingRatePerConsumer=300, ratePeriodInSecond=30}\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-2r8zr","container_name":"broker"},"docker":{"container_id":"6cc06889852e17619b2520562cdb2eba70b37d5e318210f0df33ee868378fe8d"}}
{"msg":"17:11:16.978 [bookkeeper-ml-workers-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.persistent.DispatchRateLimiter - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails] configured TOPIC message-dispatch rate at broker DispatchRate{dispatchThrottlingRateInMsg=500000, dispatchThrottlingRateInByte=52428800, ratePeriodInSecond=1}\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-2r8zr","container_name":"broker"},"docker":{"container_id":"6cc06889852e17619b2520562cdb2eba70b37d5e318210f0df33ee868378fe8d"}}
{"msg":"17:11:16.978 [bookkeeper-ml-workers-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.AbstractTopic - Disabling publish throttling for persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-2r8zr","container_name":"broker"},"docker":{"container_id":"6cc06889852e17619b2520562cdb2eba70b37d5e318210f0df33ee868378fe8d"}}
{"msg":"17:11:16.978 [bookkeeper-ml-workers-OrderedExecutor-2-0] WARN  org.apache.pulsar.broker.service.AbstractTopic - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails] Error getting policies null and publish throttling will be disabled\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-2r8zr","container_name":"broker"},"docker":{"container_id":"6cc06889852e17619b2520562cdb2eba70b37d5e318210f0df33ee868378fe8d"}}
{"msg":"17:11:16.380 [Thread-5203370] INFO  org.apache.pulsar.broker.service.ServerCnx - [/10.9.29.28:37282] Subscribing on topic persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails / canvas-cdc/clean-prod-iad/cdc-transform-71\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-2r8zr","container_name":"broker"},"docker":{"container_id":"6cc06889852e17619b2520562cdb2eba70b37d5e318210f0df33ee868378fe8d"}}
{"msg":"17:11:16.143 [Thread-492744] WARN  org.apache.pulsar.broker.service.BrokerService - Namespace bundle for topic (persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails) not served by this instance. Please redo the lookup. Request is denied: namespace=canvas-cdc/prod-iad\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:16.143 [Thread-492744] INFO  org.apache.pulsar.broker.service.ServerCnx - [/10.9.27.47:40828] persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails configured with schema false\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:16.073 [bookkeeper-ml-workers-OrderedExecutor-2-0] INFO  org.apache.pulsar.broker.service.persistent.PersistentTopic - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails] Topic closed\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:16.065 [Thread-492726] WARN  org.apache.pulsar.broker.service.ServerCnx - [/10.9.29.28:52860][persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails][canvas-cdc/clean-prod-iad/cdc-transform-71] Failed to create consumer: Topic is temporarily unavailable\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:16.065 [Thread-492726] WARN  org.apache.pulsar.broker.service.persistent.PersistentTopic - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails] Attempting to subscribe to a fenced topic\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:16.064 [Thread-492726] INFO  org.apache.pulsar.broker.service.ServerCnx - [/10.9.29.28:52860] Subscribing on topic persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails / canvas-cdc/clean-prod-iad/cdc-transform-71\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:16.024 [Thread-492716] INFO  org.apache.pulsar.broker.service.ServerCnx - [/10.9.27.47:40828][persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails] Creating producer. producerId=73\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:15.629 [pulsar-web-33-11] INFO  org.apache.pulsar.broker.service.persistent.PersistentSubscription - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails][canvas-cdc/clean-prod-iad/cdc-transform-71] Successfully disconnected and closed subscription\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:15.629 [pulsar-web-33-11] INFO  org.apache.pulsar.broker.service.persistent.PersistentSubscription - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails][canvas-cdc/clean-prod-iad/cdc-transform-71] Successfully closed subscription [ManagedCursorImpl{ledger=canvas-cdc/prod-iad/persistent/cluster71-canvas.cluster67_shard_1072.thumbnails, name=canvas-cdc%2Fclean-prod-iad%2Fcdc-transform-71, ackPos=522049:0, readPos=522049:1}]\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:15.628 [pulsar-web-33-11] INFO  org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails / canvas-cdc/clean-prod-iad/cdc-transform-71] All consumers removed. Subscription is disconnected\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:15.627 [pulsar-web-33-11] INFO  org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers - Removed consumer Consumer{subscription=PersistentSubscription{topic=persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails, name=canvas-cdc/clean-prod-iad/cdc-transform-71}, consumerId=229, consumerName=5525b, address=/10.9.29.28:52860} with pending 0 acks\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:15.627 [pulsar-web-33-11] INFO  org.apache.pulsar.broker.service.Consumer - Disconnecting consumer: Consumer{subscription=PersistentSubscription{topic=persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails, name=canvas-cdc/clean-prod-iad/cdc-transform-71}, consumerId=229, consumerName=5525b, address=/10.9.29.28:52860}\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:15.627 [pulsar-web-33-11] INFO  org.apache.pulsar.broker.service.Producer - Disconnecting producer: Producer{topic=PersistentTopic{topic=persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails}, client=/10.9.27.47:40828, producerName=pulsar-prod-iad-199-193, producerId=73}\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:15.627 [pulsar-web-33-11] INFO  org.apache.pulsar.broker.service.BrokerService - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails] Unloading topic\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}

Client Logs

{"msg":"17:12:02.508 [pulsar-timer-4-1] INFO  org.apache.pulsar.client.impl.ConnectionHandler - [canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails] [pulsar-prod-iad-199-193] Reconnecting after connection was closed\n","kube":{"namespace":"pulsar-cdc","pod_name":"pf-canvas-cdc-prod-iad-cluster71-0","container_name":"pulsarfunction"},"docker":{"container_id":"c4e6640923ec8e00f50ca85375c7b37c0de7b59491dd7cdb68800b84f8ac5740"}}
{"msg":"17:12:02.406 [pulsar-client-io-1-1] WARN  org.apache.pulsar.client.impl.ConnectionHandler - [canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails] [pulsar-prod-iad-199-193] Could not get connection to broker: 14588 lookup request timedout after ms 30000 -- Will try again in 0.1 s\n","kube":{"namespace":"pulsar-cdc","pod_name":"pf-canvas-cdc-prod-iad-cluster71-0","container_name":"pulsarfunction"},"docker":{"container_id":"c4e6640923ec8e00f50ca85375c7b37c0de7b59491dd7cdb68800b84f8ac5740"}}
{"msg":"17:12:02.406 [pulsar-client-io-1-1] ERROR org.apache.pulsar.client.impl.ProducerImpl - [canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails] [pulsar-prod-iad-199-193] Failed to create producer: 14588 lookup request timedout after ms 30000\n","kube":{"namespace":"pulsar-cdc","pod_name":"pf-canvas-cdc-prod-iad-cluster71-0","container_name":"pulsarfunction"},"docker":{"container_id":"c4e6640923ec8e00f50ca85375c7b37c0de7b59491dd7cdb68800b84f8ac5740"}}
{"msg":"17:11:15.970 [pulsar-client-io-1-1] INFO  org.apache.pulsar.client.impl.ProducerImpl - [canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails] [pulsar-prod-iad-199-193] Creating producer on cnx [id: 0x37f41d66, L:/10.9.27.47:40828 - R:/10.9.36.202:6651]\n","kube":{"namespace":"pulsar-cdc","pod_name":"pf-canvas-cdc-prod-iad-cluster71-0","container_name":"pulsarfunction"},"docker":{"container_id":"c4e6640923ec8e00f50ca85375c7b37c0de7b59491dd7cdb68800b84f8ac5740"}}
{"msg":"17:11:15.968 [pulsar-timer-4-1] INFO  org.apache.pulsar.client.impl.ConnectionHandler - [canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails] [pulsar-prod-iad-199-193] Reconnecting after timeout\n","kube":{"namespace":"pulsar-cdc","pod_name":"pf-canvas-cdc-prod-iad-cluster71-0","container_name":"pulsarfunction"},"docker":{"container_id":"c4e6640923ec8e00f50ca85375c7b37c0de7b59491dd7cdb68800b84f8ac5740"}}
{"msg":"17:11:15.866 [pulsar-client-io-1-1] INFO  org.apache.pulsar.client.impl.ConnectionHandler - [canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.thumbnails] [pulsar-prod-iad-199-193] Closed connection [id: 0x37f41d66, L:/10.9.27.47:40828 - R:/10.9.36.202:6651] -- Will try again in 0.1 s\n","kube":{"namespace":"pulsar-cdc","pod_name":"pf-canvas-cdc-prod-iad-cluster71-0","container_name":"pulsarfunction"},"docker":{"container_id":"c4e6640923ec8e00f50ca85375c7b37c0de7b59491dd7cdb68800b84f8ac5740"}}

Good Instance

Broker Logs

{"msg":"17:11:17.631 [bookkeeper-ml-workers-OrderedExecutor-7-0] INFO  org.apache.pulsar.broker.service.BrokerService - Created topic persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories - dedup is disabled\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-2r8zr","container_name":"broker"},"docker":{"container_id":"6cc06889852e17619b2520562cdb2eba70b37d5e318210f0df33ee868378fe8d"}}
{"msg":"17:11:17.631 [bookkeeper-ml-workers-OrderedExecutor-7-0] INFO  org.apache.pulsar.broker.service.persistent.PersistentTopic - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] There are no replicated subscriptions on the topic\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-2r8zr","container_name":"broker"},"docker":{"container_id":"6cc06889852e17619b2520562cdb2eba70b37d5e318210f0df33ee868378fe8d"}}
{"msg":"17:11:17.630 [bookkeeper-ml-workers-OrderedExecutor-7-0] INFO  org.apache.pulsar.broker.service.persistent.SubscribeRateLimiter - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] configured subscribe-dispatch rate at broker SubscribeRate{subscribeThrottlingRatePerConsumer=300, ratePeriodInSecond=30}\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-2r8zr","container_name":"broker"},"docker":{"container_id":"6cc06889852e17619b2520562cdb2eba70b37d5e318210f0df33ee868378fe8d"}}
{"msg":"17:11:17.630 [bookkeeper-ml-workers-OrderedExecutor-7-0] INFO  org.apache.pulsar.broker.service.persistent.DispatchRateLimiter - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] configured TOPIC message-dispatch rate at broker DispatchRate{dispatchThrottlingRateInMsg=500000, dispatchThrottlingRateInByte=52428800, ratePeriodInSecond=1}\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-2r8zr","container_name":"broker"},"docker":{"container_id":"6cc06889852e17619b2520562cdb2eba70b37d5e318210f0df33ee868378fe8d"}}
{"msg":"17:11:17.630 [bookkeeper-ml-workers-OrderedExecutor-7-0] INFO  org.apache.pulsar.broker.service.AbstractTopic - Disabling publish throttling for persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-2r8zr","container_name":"broker"},"docker":{"container_id":"6cc06889852e17619b2520562cdb2eba70b37d5e318210f0df33ee868378fe8d"}}
{"msg":"17:11:17.630 [bookkeeper-ml-workers-OrderedExecutor-7-0] WARN  org.apache.pulsar.broker.service.AbstractTopic - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] Error getting policies null and publish throttling will be disabled\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-2r8zr","container_name":"broker"},"docker":{"container_id":"6cc06889852e17619b2520562cdb2eba70b37d5e318210f0df33ee868378fe8d"}}
{"msg":"17:11:17.369 [Thread-5203753] INFO  org.apache.pulsar.broker.service.ServerCnx - [/10.9.27.47:60270][persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] Creating producer. producerId=97\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-2r8zr","container_name":"broker"},"docker":{"container_id":"6cc06889852e17619b2520562cdb2eba70b37d5e318210f0df33ee868378fe8d"}}
{"msg":"17:11:17.218 [ForkJoinPool.commonPool-worker-1] INFO  org.apache.pulsar.broker.service.ServerCnx - [/10.9.29.28:37282] Subscribing on topic persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories / canvas-cdc/clean-prod-iad/cdc-transform-71\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-2r8zr","container_name":"broker"},"docker":{"container_id":"6cc06889852e17619b2520562cdb2eba70b37d5e318210f0df33ee868378fe8d"}}
{"msg":"17:11:16.989 [Thread-493004] WARN  org.apache.pulsar.broker.service.BrokerService - Namespace is being unloaded, cannot add topic persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:16.988 [Thread-493004] INFO  org.apache.pulsar.broker.service.ServerCnx - [/10.9.27.47:40828][persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] Creating producer. producerId=97\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:16.967 [bookkeeper-ml-workers-OrderedExecutor-7-0] INFO  org.apache.pulsar.broker.service.persistent.PersistentTopic - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] Topic closed\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:16.880 [pulsar-web-33-2] INFO  org.apache.pulsar.broker.service.persistent.PersistentSubscription - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories][canvas-cdc/clean-prod-iad/cdc-transform-71] Successfully disconnected and closed subscription\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:16.880 [pulsar-web-33-2] INFO  org.apache.pulsar.broker.service.persistent.PersistentSubscription - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories][canvas-cdc/clean-prod-iad/cdc-transform-71] Successfully closed subscription [ManagedCursorImpl{ledger=canvas-cdc/prod-iad/persistent/cluster71-canvas.cluster67_shard_1072.group_categories, name=canvas-cdc%2Fclean-prod-iad%2Fcdc-transform-71, ackPos=521792:0, readPos=521792:1}]\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:16.880 [pulsar-web-33-2] INFO  org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories / canvas-cdc/clean-prod-iad/cdc-transform-71] All consumers removed. Subscription is disconnected\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:16.880 [pulsar-web-33-2] INFO  org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers - Removed consumer Consumer{subscription=PersistentSubscription{topic=persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories, name=canvas-cdc/clean-prod-iad/cdc-transform-71}, consumerId=81, consumerName=5525b, address=/10.9.29.28:52860} with pending 0 acks\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:16.879 [pulsar-web-33-2] INFO  org.apache.pulsar.broker.service.Consumer - Disconnecting consumer: Consumer{subscription=PersistentSubscription{topic=persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories, name=canvas-cdc/clean-prod-iad/cdc-transform-71}, consumerId=81, consumerName=5525b, address=/10.9.29.28:52860}\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:16.879 [pulsar-web-33-2] INFO  org.apache.pulsar.broker.service.Producer - Disconnecting producer: Producer{topic=PersistentTopic{topic=persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories}, client=/10.9.27.47:40828, producerName=pulsar-prod-iad-196-161, producerId=97}\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}
{"msg":"17:11:16.879 [pulsar-web-33-2] INFO  org.apache.pulsar.broker.service.BrokerService - [persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] Unloading topic\n","kube":{"namespace":"pulsar","pod_name":"pulsar-prod-broker-564968ff9c-zljhk","container_name":"broker"},"docker":{"container_id":"c2bc11c1db743603d128641575963a36d3c5ee2f6abca2994df97a7e8720359e"}}

Client Logs

{"msg":"17:11:17.727 [pulsar-client-io-1-1] INFO  org.apache.pulsar.client.impl.ProducerImpl - [canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] [pulsar-prod-iad-196-161] Created producer on cnx [id: 0xd2d7a3f2, L:/10.9.27.47:60270 - R:/10.9.47.166:6651]\n","kube":{"namespace":"pulsar-cdc","pod_name":"pf-canvas-cdc-prod-iad-cluster71-0","container_name":"pulsarfunction"},"docker":{"container_id":"c4e6640923ec8e00f50ca85375c7b37c0de7b59491dd7cdb68800b84f8ac5740"}}
{"msg":"17:11:17.365 [pulsar-client-io-1-1] INFO  org.apache.pulsar.client.impl.ProducerImpl - [canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] [pulsar-prod-iad-196-161] Creating producer on cnx [id: 0xd2d7a3f2, L:/10.9.27.47:60270 - R:/10.9.47.166:6651]\n","kube":{"namespace":"pulsar-cdc","pod_name":"pf-canvas-cdc-prod-iad-cluster71-0","container_name":"pulsarfunction"},"docker":{"container_id":"c4e6640923ec8e00f50ca85375c7b37c0de7b59491dd7cdb68800b84f8ac5740"}}
{"msg":"17:11:17.362 [pulsar-timer-4-1] INFO  org.apache.pulsar.client.impl.ConnectionHandler - [canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] [pulsar-prod-iad-196-161] Reconnecting after connection was closed\n","kube":{"namespace":"pulsar-cdc","pod_name":"pf-canvas-cdc-prod-iad-cluster71-0","container_name":"pulsarfunction"},"docker":{"container_id":"c4e6640923ec8e00f50ca85375c7b37c0de7b59491dd7cdb68800b84f8ac5740"}}
{"msg":"17:11:17.262 [pulsar-client-io-1-1] WARN  org.apache.pulsar.client.impl.ConnectionHandler - [canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] [pulsar-prod-iad-196-161] Could not get connection to broker: Namespace is being unloaded, cannot add topic persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories -- Will try again in 0.1 s\n","kube":{"namespace":"pulsar-cdc","pod_name":"pf-canvas-cdc-prod-iad-cluster71-0","container_name":"pulsarfunction"},"docker":{"container_id":"c4e6640923ec8e00f50ca85375c7b37c0de7b59491dd7cdb68800b84f8ac5740"}}
{"msg":"17:11:17.261 [pulsar-client-io-1-1] ERROR org.apache.pulsar.client.impl.ProducerImpl - [canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] [pulsar-prod-iad-196-161] Failed to create producer: Namespace is being unloaded, cannot add topic persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories\n","kube":{"namespace":"pulsar-cdc","pod_name":"pf-canvas-cdc-prod-iad-cluster71-0","container_name":"pulsarfunction"},"docker":{"container_id":"c4e6640923ec8e00f50ca85375c7b37c0de7b59491dd7cdb68800b84f8ac5740"}}
{"msg":"17:11:17.261 [pulsar-client-io-1-1] WARN  org.apache.pulsar.client.impl.ClientCnx - [id: 0x37f41d66, L:/10.9.27.47:40828 - R:/10.9.36.202:6651] Received error from server: Namespace is being unloaded, cannot add topic persistent://canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories\n","kube":{"namespace":"pulsar-cdc","pod_name":"pf-canvas-cdc-prod-iad-cluster71-0","container_name":"pulsarfunction"},"docker":{"container_id":"c4e6640923ec8e00f50ca85375c7b37c0de7b59491dd7cdb68800b84f8ac5740"}}
{"msg":"17:11:16.982 [pulsar-client-io-1-1] INFO  org.apache.pulsar.client.impl.ProducerImpl - [canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] [pulsar-prod-iad-196-161] Creating producer on cnx [id: 0x37f41d66, L:/10.9.27.47:40828 - R:/10.9.36.202:6651]\n","kube":{"namespace":"pulsar-cdc","pod_name":"pf-canvas-cdc-prod-iad-cluster71-0","container_name":"pulsarfunction"},"docker":{"container_id":"c4e6640923ec8e00f50ca85375c7b37c0de7b59491dd7cdb68800b84f8ac5740"}}
{"msg":"17:11:16.980 [pulsar-timer-4-1] INFO  org.apache.pulsar.client.impl.ConnectionHandler - [canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] [pulsar-prod-iad-196-161] Reconnecting after timeout\n","kube":{"namespace":"pulsar-cdc","pod_name":"pf-canvas-cdc-prod-iad-cluster71-0","container_name":"pulsarfunction"},"docker":{"container_id":"c4e6640923ec8e00f50ca85375c7b37c0de7b59491dd7cdb68800b84f8ac5740"}}
{"msg":"17:11:16.880 [pulsar-client-io-1-1] INFO  org.apache.pulsar.client.impl.ConnectionHandler - [canvas-cdc/prod-iad/cluster71-canvas.cluster67_shard_1072.group_categories] [pulsar-prod-iad-196-161] Closed connection [id: 0x37f41d66, L:/10.9.27.47:40828 - R:/10.9.36.202:6651] -- Will try again in 0.1 s\n","kube":{"namespace":"pulsar-cdc","pod_name":"pf-canvas-cdc-prod-iad-cluster71-0","container_name":"pulsarfunction"},"docker":{"container_id":"c4e6640923ec8e00f50ca85375c7b37c0de7b59491dd7cdb68800b84f8ac5740"}}

The most important thing to note here is just the difference in when we get a client exception (in the good case) and when we don't (in the bad case). We see this pattern pretty much universally, but, AFAICT from the code, there isn't any reason to believe that that the future shouldn't immediately resolve in both cases.

Additional context
We saw this in 2.4.x but are also seeing it in 2.5.0 using the official docker images. We also are connecting directly to the brokers (this occurs in a pulsar IO source). We see this occur when we have lots of load shedding when we are back-filling data into Pulsar and don't have as well distributed load.

@addisonj addisonj added the type/bug The PR fixed a bug or issue reported a bug label Feb 25, 2020
@sijie
Copy link
Member

sijie commented Feb 25, 2020

/cc @codelipenghui since he introduced epoch to solve some of the create-producer timeouts.

@skyrocknroll
Copy link
Contributor

We are facing the same issue in our environment when auto unloads kicks in.

@codelipenghui
Copy link
Contributor

@skyrocknroll Which broker version are you used?

@skyrocknroll
Copy link
Contributor

@codelipenghui 2.4.2

addisonj pushed a commit to instructure/pulsar that referenced this issue Mar 6, 2020
See apache#6416. This change ensures that all futures within BrokerService
have a guranteed timeout. As stated in apache#6416, we see cases where it
appears that loading or creating a topic fails to resolve the future for
unknown reasons. It appears that these futures *may* not be returning.
This seems like a sane change to make to ensure that these futures
finish, however, it still isn't understood under what conditions these
futures may not be returning, so this fix is mostly a workaround for
some underlying issues
sijie pushed a commit that referenced this issue Mar 18, 2020
See #6416. This change ensures that all futures within BrokerService
have a guranteed timeout. As stated in #6416, we see cases where it
appears that loading or creating a topic fails to resolve the future for
unknown reasons. It appears that these futures *may* not be returning.
This seems like a sane change to make to ensure that these futures
finish, however, it still isn't understood under what conditions these
futures may not be returning, so this fix is mostly a workaround for
some underlying issues

Co-authored-by: Addison Higham <[email protected]>
addisonj pushed a commit to instructure/pulsar that referenced this issue Mar 18, 2020
See apache#6416. This change ensures that all futures within BrokerService
have a guranteed timeout. As stated in apache#6416, we see cases where it
appears that loading or creating a topic fails to resolve the future for
unknown reasons. It appears that these futures *may* not be returning.
This seems like a sane change to make to ensure that these futures
finish, however, it still isn't understood under what conditions these
futures may not be returning, so this fix is mostly a workaround for
some underlying issues
tuteng pushed a commit to AmateurEvents/pulsar that referenced this issue Mar 21, 2020
See apache#6416. This change ensures that all futures within BrokerService
have a guranteed timeout. As stated in apache#6416, we see cases where it
appears that loading or creating a topic fails to resolve the future for
unknown reasons. It appears that these futures *may* not be returning.
This seems like a sane change to make to ensure that these futures
finish, however, it still isn't understood under what conditions these
futures may not be returning, so this fix is mostly a workaround for
some underlying issues

Co-authored-by: Addison Higham <[email protected]>
(cherry picked from commit 4a4cce9)
tuteng pushed a commit that referenced this issue Apr 6, 2020
See #6416. This change ensures that all futures within BrokerService
have a guranteed timeout. As stated in #6416, we see cases where it
appears that loading or creating a topic fails to resolve the future for
unknown reasons. It appears that these futures *may* not be returning.
This seems like a sane change to make to ensure that these futures
finish, however, it still isn't understood under what conditions these
futures may not be returning, so this fix is mostly a workaround for
some underlying issues

Co-authored-by: Addison Higham <[email protected]>
(cherry picked from commit 4a4cce9)
tuteng pushed a commit that referenced this issue Apr 13, 2020
See #6416. This change ensures that all futures within BrokerService
have a guranteed timeout. As stated in #6416, we see cases where it
appears that loading or creating a topic fails to resolve the future for
unknown reasons. It appears that these futures *may* not be returning.
This seems like a sane change to make to ensure that these futures
finish, however, it still isn't understood under what conditions these
futures may not be returning, so this fix is mostly a workaround for
some underlying issues

Co-authored-by: Addison Higham <[email protected]>
(cherry picked from commit 4a4cce9)
addisonj pushed a commit to instructure/pulsar that referenced this issue May 5, 2020
Fixes apache#6872
Fixes apache#6416

If a producer tries to create a producer to a topic that is currently
unloading, we can get a `RuntimeException` from
`BrokerService.checkTopicNsOwnership` which is bubbled up through
`topic.addProducer`. By only handling a `BrokerServiceException` this
results in a future that never completes and results in producers not
being able to be created if this topic is scheduled back to this broker.
codelipenghui pushed a commit that referenced this issue May 6, 2020
Fixes #6872
Fixes #6416

If a producer tries to create a producer to a topic that is currently
unloading, we can get a `RuntimeException` from
`BrokerService.checkTopicNsOwnership` which is bubbled up through
`topic.addProducer`. By only handling a `BrokerServiceException` this
results in a future that never completes and results in producers not
being able to be created if this topic is scheduled back to this broker.
addisonj pushed a commit to instructure/pulsar that referenced this issue May 7, 2020
Fixes apache#6872
Fixes apache#6416

If a producer tries to create a producer to a topic that is currently
unloading, we can get a `RuntimeException` from
`BrokerService.checkTopicNsOwnership` which is bubbled up through
`topic.addProducer`. By only handling a `BrokerServiceException` this
results in a future that never completes and results in producers not
being able to be created if this topic is scheduled back to this broker.
jiazhai pushed a commit that referenced this issue May 8, 2020
Fixes #6872
Fixes #6416

If a producer tries to create a producer to a topic that is currently
unloading, we can get a `RuntimeException` from
`BrokerService.checkTopicNsOwnership` which is bubbled up through
`topic.addProducer`. By only handling a `BrokerServiceException` this
results in a future that never completes and results in producers not
being able to be created if this topic is scheduled back to this broker.
(cherry picked from commit 30e26f8)
cdbartholomew pushed a commit to kafkaesque-io/pulsar that referenced this issue May 12, 2020
Fixes apache#6872
Fixes apache#6416

If a producer tries to create a producer to a topic that is currently
unloading, we can get a `RuntimeException` from
`BrokerService.checkTopicNsOwnership` which is bubbled up through
`topic.addProducer`. By only handling a `BrokerServiceException` this
results in a future that never completes and results in producers not
being able to be created if this topic is scheduled back to this broker.
jiazhai pushed a commit to jiazhai/pulsar that referenced this issue May 18, 2020
See apache#6416. This change ensures that all futures within BrokerService
have a guranteed timeout. As stated in apache#6416, we see cases where it
appears that loading or creating a topic fails to resolve the future for
unknown reasons. It appears that these futures *may* not be returning.
This seems like a sane change to make to ensure that these futures
finish, however, it still isn't understood under what conditions these
futures may not be returning, so this fix is mostly a workaround for
some underlying issues

Co-authored-by: Addison Higham <[email protected]>(cherry picked from commit 4a4cce9)
Huanli-Meng pushed a commit to Huanli-Meng/pulsar that referenced this issue May 27, 2020
Fixes apache#6872
Fixes apache#6416

If a producer tries to create a producer to a topic that is currently
unloading, we can get a `RuntimeException` from
`BrokerService.checkTopicNsOwnership` which is bubbled up through
`topic.addProducer`. By only handling a `BrokerServiceException` this
results in a future that never completes and results in producers not
being able to be created if this topic is scheduled back to this broker.
huangdx0726 pushed a commit to huangdx0726/pulsar that referenced this issue Aug 24, 2020
See apache#6416. This change ensures that all futures within BrokerService
have a guranteed timeout. As stated in apache#6416, we see cases where it
appears that loading or creating a topic fails to resolve the future for
unknown reasons. It appears that these futures *may* not be returning.
This seems like a sane change to make to ensure that these futures
finish, however, it still isn't understood under what conditions these
futures may not be returning, so this fix is mostly a workaround for
some underlying issues

Co-authored-by: Addison Higham <[email protected]>
huangdx0726 pushed a commit to huangdx0726/pulsar that referenced this issue Aug 24, 2020
Fixes apache#6872
Fixes apache#6416

If a producer tries to create a producer to a topic that is currently
unloading, we can get a `RuntimeException` from
`BrokerService.checkTopicNsOwnership` which is bubbled up through
`topic.addProducer`. By only handling a `BrokerServiceException` this
results in a future that never completes and results in producers not
being able to be created if this topic is scheduled back to this broker.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging a pull request may close this issue.

5 participants