You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/05/04 13:27:36 UTC

[GitHub] [pulsar] zbentley opened a new issue, #15433: Terminating a partitioned topic that has just been created often fails with "PreconditionFailedException: Failed to find ownership for topic..."

zbentley opened a new issue, #15433:
URL: https://github.com/apache/pulsar/issues/15433

   **Describe the bug**
   Using pulsar standalone, when running integration tests, I often create tenants, namespaces, and topics, then rapidly take actions on those topics (like publishing, termination, deletion). When I attempt to terminate a topic right after it has been created, it sometimes fails with "PreconditionFailedException: Failed to find ownership for topic" errors (see stacktrace below).
   
   This condition persists for quite awhile; if I retry once a second for up to 5 seconds, sometimes it goes away, but sometimes it's still occurring after 5 seconds.
   
   In 2.9.1 the error occurs with a 412 response code. In 2.10 it has a 500 response code. 
   
   It shouldn't happen regardless; topics should be terminateable once the management API reports that they've been successfully created.
   
   **To Reproduce**
   
   On a standalone 2.10 or 2.9 broker, in rapid succession, do the following:
   
   1. Using the management API, create a tenant.
   2. Using the management API, create a namespace.
   3. Using the management API, create a persistent, partitioned topic.
   4. `POST` to the `/terminate/partitions` endpoint on that topic.
   
   Doing this repeatedly will eventually encounter an error with the below stacktrace.
   
   **Expected behavior**
   Once created, topics should be "done", in that they're ready for use from anywhere in the cluster.
   
   If using autocreated topics, I can understand why this would be tricky (autocreating a topic via the binary protocol can be more eventually consistent, since only the thing issuing the create needs to use it right away).
   
   But when using the management API to create a topic, the API shouldn't return 200 for creation until all backend operations report success. That might mean topic creation has to take a long time, or might fail if all brokers can't be reached to confirm metadata update, but that's preferable to topics breaking when users try to do things with them later on.
   
   **Desktop (please complete the following information):**
    - MacOS 12, Intel
    - Pulsar standalone 2.9 and 2.10 via Homebrew
   
   **Example Stacktrace**
   ```
   E               chariot.pulsar.client.exceptions.PiecemealDeletionError: ClientResponseError: 500, message='Internal Server Error', url=URL('http://localhost:8080/admin/v2/persistent/chariot-test-obj-ChariotSite-251-gqtb_1/chariot_namespace_integration_test_topic/chariot_topic_integration_test_topic/terminate/partitions')
   E               Response code: 500
   E               URL: POST http://localhost:8080/admin/v2/persistent/chariot-test-obj-ChariotSite-251-gqtb_1/chariot_namespace_integration_test_topic/chariot_topic_integration_test_topic/terminate/partitions
   E
   E                --- An unexpected error occurred in the server ---
   E
   E               Message: Failed to find ownership for topic:persistent://chariot-test-obj-ChariotSite-251-gqtb_1/chariot_namespace_integration_test_topic/chariot_topic_integration_test_topic-partition-0
   E
   E               Stacktrace:
   E
   E               org.apache.pulsar.client.admin.PulsarAdminException$PreconditionFailedException: Failed to find ownership for topic:persistent://chariot-test-obj-ChariotSite-251-gqtb_1/chariot_namespace_integration_test_topic/chariot_topic_integration_test_topic-partition-0
   E               	at org.apache.pulsar.client.admin.internal.BaseResource.getApiException(BaseResource.java:240)
   E               	at org.apache.pulsar.client.admin.internal.TopicsImpl$8.failed(TopicsImpl.java:536)
   E               	at org.glassfish.jersey.client.JerseyInvocation$1.failed(JerseyInvocation.java:882)
   E               	at org.glassfish.jersey.client.JerseyInvocation$1.completed(JerseyInvocation.java:863)
   E               	at org.glassfish.jersey.client.ClientRuntime.processResponse(ClientRuntime.java:229)
   E               	at org.glassfish.jersey.client.ClientRuntime.access$200(ClientRuntime.java:62)
   E               	at org.glassfish.jersey.client.ClientRuntime$2.lambda$response$0(ClientRuntime.java:173)
   E               	at org.glassfish.jersey.internal.Errors$1.call(Errors.java:248)
   E               	at org.glassfish.jersey.internal.Errors$1.call(Errors.java:244)
   E               	at org.glassfish.jersey.internal.Errors.process(Errors.java:292)
   E               	at org.glassfish.jersey.internal.Errors.process(Errors.java:274)
   E               	at org.glassfish.jersey.internal.Errors.process(Errors.java:244)
   E               	at org.glassfish.jersey.process.internal.RequestScope.runInScope(RequestScope.java:288)
   E               	at org.glassfish.jersey.client.ClientRuntime$2.response(ClientRuntime.java:173)
   E               	at org.apache.pulsar.client.admin.internal.http.AsyncHttpConnector.lambda$apply$1(AsyncHttpConnector.java:228)
   E               	at java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:859)
   E               	at java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:837)
   E               	at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506)
   E               	at java.base/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2073)
   E               	at org.apache.pulsar.client.admin.internal.http.AsyncHttpConnector.lambda$retryOperation$4(AsyncHttpConnector.java:270)
   E               	at java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:859)
   E               	at java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:837)
   E               	at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506)
   E               	at java.base/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2073)
   E               	at org.asynchttpclient.netty.NettyResponseFuture.loadContent(NettyResponseFuture.java:222)
   E               	at org.asynchttpclient.netty.NettyResponseFuture.done(NettyResponseFuture.java:257)
   E               	at org.asynchttpclient.netty.handler.AsyncHttpClientHandler.finishUpdate(AsyncHttpClientHandler.java:241)
   E               	at org.asynchttpclient.netty.handler.HttpHandler.handleChunk(HttpHandler.java:114)
   E               	at org.asynchttpclient.netty.handler.HttpHandler.handleRead(HttpHandler.java:143)
   E               	at org.asynchttpclient.netty.handler.AsyncHttpClientHandler.channelRead(AsyncHttpClientHandler.java:78)
   E               	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
   E               	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
   E               	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
   E               	at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
   E               	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
   E               	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
   E               	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
   E               	at io.netty.channel.CombinedChannelDuplexHandler$DelegatingChannelHandlerContext.fireChannelRead(CombinedChannelDuplexHandler.java:436)
   E               	at io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:327)
   E               	at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:299)
   E               	at io.netty.channel.CombinedChannelDuplexHandler.channelRead(CombinedChannelDuplexHandler.java:251)
   E               	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
   E               	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
   E               	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
   E               	at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
   E               	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
   E               	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
   E               	at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
   E               	at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:166)
   E               	at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:722)
   E               	at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:658)
   E               	at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:584)
   E               	at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:496)
   E               	at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:986)
   E               	at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
   E               	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
   E               	at java.base/java.lang.Thread.run(Thread.java:829)
   E               Caused by: javax.ws.rs.ClientErrorException: HTTP 412 Precondition Failed
   E               	at org.glassfish.jersey.client.JerseyInvocation.createExceptionForFamily(JerseyInvocation.java:985)
   E               	at org.glassfish.jersey.client.JerseyInvocation.convertToException(JerseyInvocation.java:967)
   E               	at org.glassfish.jersey.client.JerseyInvocation.access$700(JerseyInvocation.java:82)
   E               	... 54 more
   ```
   


-- 
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: commits-unsubscribe@pulsar.apache.org.apache.org

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


[GitHub] [pulsar] zbentley commented on issue #15433: Terminating a partitioned topic that has just been created often fails with "PreconditionFailedException: Failed to find ownership for topic..."

Posted by GitBox <gi...@apache.org>.
zbentley commented on issue #15433:
URL: https://github.com/apache/pulsar/issues/15433#issuecomment-1126984325

   @mattisonchao I suspect that the use of `pulsar-admin` is inducing too much latency to reproduce the issue, as `pulsar-admin` takes some time to start. We use a custom ORM to interact with Pulsar admin objects; below I have captured the HTTP calls made by the ORM in a situation that reproduces this bug. The steps are roughly the same (create namespace with various configs, create a topic in it, immediately attempt to terminate it.
   
   ```
   http://localhost:8080/admin/v2/namespaces/chariot1/chariot_namespace_techtalk_topic -> 404; history: ['http://localhost:8080/admin/v2/namespaces/chariot1/chariot_namespace_techtalk_topic']; kwargs: {}; response: {"reason":"Namespace does not exist"}
   http://localhost:8080/admin/v2/bookies/all -> 200; history: ['http://localhost:8080/admin/v2/bookies/all']; kwargs: {}; response: {"bookies":[{"bookieId":"127.0.0.1:3181"}]}
   http://localhost:8080/admin/v2/racks-info -> 404; history: ['http://localhost:8080/admin/v2/racks-info']; kwargs: {}; response:
   http://localhost:8080/admin/v2/namespaces/chariot1/chariot_namespace_techtalk_topic -> 204; history: ['http://localhost:8080/admin/v2/namespaces/chariot1/chariot_namespace_techtalk_topic']; kwargs: {'json': {'anti_affinity_group': None, 'backlog_quota_map': {'destination_storage': {'policy': 'producer_exception', 'limitSize': 4096, 'limitTime': -1}}, 'retention_policies': {'retentionTimeInMinutes': 60, 'retentionSizeInMB': -1}, 'persistence': {'bookkeeperEnsemble': 1, 'bookkeeperWriteQuorum': 1, 'bookkeeperAckQuorum': 1, 'managedLedgerMaxMarkDeleteRate': 0}, 'deduplicationEnabled': False, 'autoSubscriptionCreationOverride': {'allowAutoSubscriptionCreation': False}, 'autoTopicCreationOverride': {'allowAutoTopicCreation': False, 'defaultNumPartitions': 0, 'topicType': 'non-partitioned'}, 'message_ttl_in_seconds': 0, 'subscription_expiration_time_minutes': 0, 'encryption_required': False, 'inactive_topic_policies': {'maxInactiveDurationSeconds': -1, 'inactiveTopicDeleteMode': 'delete
 _when_no_subscriptions', 'deleteWhileInactive': False}, 'delayed_delivery_policies': {'tickTime': 30.0}, 'max_producers_per_topic': 100, 'max_consumers_per_topic': 10, 'max_consumers_per_subscription': 0, 'max_unacked_messages_per_subscription': 10000, 'max_unacked_messages_per_consumer': 100, 'properties': {'chariot_primary': True, 'chariot_topic_config': '{"consumer_groups": [], "deduplicationEnabled": false, "delay_granularity": 30.0, "durability": "3", "envelope_extra_size": 3145728, "folder": "sre", "max_consumers": 10, "max_message_size": 102400, "max_producers": 100, "partitions": 4, "payload": {"extra_settings": {}, "file_descriptor_set": "file {\\n  name: \\"techtalk_topic_envelope.proto\\"\\n  package: \\"techtalk_topic\\"\\n  dependency: \\"chariot/proto/common/tag.proto\\"\\n  message_type {\\n    name: \\"Heartbeat\\"\\n    field {\\n      name: \\"message\\"\\n      number: 1\\n      label: LABEL_REQUIRED\\n      type: TYPE_STRING\\n    }\\n  }\\n  message_type {\\n   
  name: \\"ChariotMessage\\"\\n    field {\\n      name: \\"schema_version\\"\\n      number: 1\\n      label: LABEL_OPTIONAL\\n      type: TYPE_ENUM\\n      type_name: \\".techtalk_topic.ChariotMessage.ChariotMessageSchemaVersion\\"\\n      default_value: \\"V1\\"\\n    }\\n    field {\\n      name: \\"tags\\"\\n      number: 2\\n      label: LABEL_REPEATED\\n      type: TYPE_MESSAGE\\n      type_name: \\".chariot.proto.common.ChariotMessageTag\\"\\n    }\\n    field {\\n      name: \\"retries\\"\\n      number: 3\\n      label: LABEL_OPTIONAL\\n      type: TYPE_UINT32\\n      default_value: \\"0\\"\\n    }\\n    field {\\n      name: \\"data\\"\\n      number: 5\\n      label: LABEL_OPTIONAL\\n      type: TYPE_MESSAGE\\n      type_name: \\".techtalk_topic.Heartbeat\\"\\n      oneof_index: 0\\n    }\\n    field {\\n      name: \\"noop\\"\\n      number: 6\\n      label: LABEL_OPTIONAL\\n      type: TYPE_MESSAGE\\n      type_name: \\".techtalk_topic.ChariotMessage.ChariotNoopMessage\
 \"\\n      oneof_index: 0\\n    }\\n    nested_type {\\n      name: \\"ChariotNoopMessage\\"\\n      field {\\n        name: \\"data\\"\\n        number: 1\\n        label: LABEL_OPTIONAL\\n        type: TYPE_STRING\\n      }\\n    }\\n    enum_type {\\n      name: \\"ChariotMessageSchemaVersion\\"\\n      value {\\n        name: \\"V1\\"\\n        number: 1\\n      }\\n    }\\n    oneof_decl {\\n      name: \\"payload\\"\\n    }\\n    reserved_range {\\n      start: 4\\n      end: 5\\n    }\\n  }\\n}\\n"}, "rate_limits": [{"actions": ["3"], "bytes": null, "messages": 1000, "period": 60.0}], "send_timeout": 1.0, "thresholds": [{"actions": ["2"], "age": 3600.0, "kind": "1", "size": null}, {"actions": ["4"], "age": null, "kind": "2", "size": 4096}]}'}, 'is_allow_auto_update_schema': True, 'schema_validation_enforced': False}}; response:
   http://localhost:8080/admin/v2/persistent/chariot1/chariot_namespace_techtalk_topic/chariot_topic_techtalk_topic/partitions -> 204; history: ['http://localhost:8080/admin/v2/persistent/chariot1/chariot_namespace_techtalk_topic/chariot_topic_techtalk_topic/partitions']; kwargs: {'json': 4}; response:
   http://localhost:8080/admin/v2/persistent/chariot1/chariot_namespace_techtalk_topic -> 200; history: ['http://localhost:8080/admin/v2/persistent/chariot1/chariot_namespace_techtalk_topic']; kwargs: {}; response: ["persistent://chariot1/chariot_namespace_techtalk_topic/chariot_topic_techtalk_topic-partition-0","persistent://chariot1/chariot_namespace_techtalk_topic/chariot_topic_techtalk_topic-partition-1","persistent://chariot1/chariot_namespace_techtalk_topic/chariot_topic_techtalk_topic-partition-2","persistent://chariot1/chariot_namespace_techtalk_topic/chariot_topic_techtalk_topic-partition-3"]
   http://localhost:8080/admin/v2/persistent/chariot1/chariot_namespace_techtalk_topic/chariot_topic_techtalk_topic/partitions -> 200; history: ['http://localhost:8080/admin/v2/persistent/chariot1/chariot_namespace_techtalk_topic/chariot_topic_techtalk_topic/partitions']; kwargs: {}; response: {"partitions":4}
   http://localhost:8080/admin/v2/persistent/chariot1/chariot_namespace_techtalk_topic/chariot_topic_techtalk_topic/terminate/partitions
   ```


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] mattisonchao commented on issue #15433: Terminating a partitioned topic that has just been created often fails with "PreconditionFailedException: Failed to find ownership for topic..."

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on issue #15433:
URL: https://github.com/apache/pulsar/issues/15433#issuecomment-1122347743

   Hi, @zbentley 
   
   I try to use your step to reproduce this problem but looks like it doesn't work well. Could you share your code to help re-produce it?
   
   ## Reproduce step
   
   1.  `./pulsar-admin tenants create my-property`
   2.  `./pulsar-admin namespaces create my-property/my-ns`
   3.  `./pulsar-admin topics create-partitioned-topic persistent://my-property/my-ns/test -p 200`
   4.  Use `httpie` POST to the `/terminate/partitions` endpoint on that topic.
   
   ```
   http POST http://localhost:8080/admin/v2/persistent/my-property/my-ns/test/terminate/partitions
   HTTP/1.1 200 OK
   Content-Type: application/json
   Date: Tue, 10 May 2022 12:43:50 GMT
   Server: Jetty(9.4.44.v20210927)
   Transfer-Encoding: chunked
   broker-address: localhost
   {
   	... omit some data
   }
   ```
   
   
   
   


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] mattisonchao commented on issue #15433: Terminating a partitioned topic that has just been created often fails with "PreconditionFailedException: Failed to find ownership for topic..."

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on issue #15433:
URL: https://github.com/apache/pulsar/issues/15433#issuecomment-1141526313

   @zbentley  Could you provide the full log?


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] mattisonchao closed issue #15433: Terminating a partitioned topic that has just been created often fails with "PreconditionFailedException: Failed to find ownership for topic..."

Posted by GitBox <gi...@apache.org>.
mattisonchao closed issue #15433: Terminating a partitioned topic that has just been created often fails with "PreconditionFailedException: Failed to find ownership for topic..."
URL: https://github.com/apache/pulsar/issues/15433


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] mattisonchao commented on issue #15433: Terminating a partitioned topic that has just been created often fails with "PreconditionFailedException: Failed to find ownership for topic..."

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on issue #15433:
URL: https://github.com/apache/pulsar/issues/15433#issuecomment-1284945191

   Close this issue because it's already stale.


-- 
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: commits-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] github-actions[bot] commented on issue #15433: Terminating a partitioned topic that has just been created often fails with "PreconditionFailedException: Failed to find ownership for topic..."

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on issue #15433:
URL: https://github.com/apache/pulsar/issues/15433#issuecomment-1170673346

   The issue had no activity for 30 days, mark with Stale label.


-- 
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: commits-unsubscribe@pulsar.apache.org

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