You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by "MMirelli (via GitHub)" <gi...@apache.org> on 2023/01/20 21:58:15 UTC

[GitHub] [bookkeeper] MMirelli opened a new issue, #3751: Pulsar broker cannot start due to bookies timing out on putting schema

MMirelli opened a new issue, #3751:
URL: https://github.com/apache/bookkeeper/issues/3751

   **BUG REPORT**
   
   ***Describe the bug***
   
   Pulsar broker cannot start due to bookies timing out, when using [BookKeeper DS fork v4.14.6.1.0.0](https://github.com/datastax/bookkeeper/releases/tag/ds-4.14.6.1.0.0). The following error manifests on the broker pods. No other error / exception is logged by the bookies. After git-bisecting the issue against the DS fork, the error appears to have been introduced by https://github.com/apache/bookkeeper/pull/3528.
   
   ```
   2023-01-19T13:32:45,290+0000 [BookKeeperClientWorker-OrderedExecutor-3-0] ERROR org.apache.bookkeeper.client.PendingReadOp - Read of ledger entry failed: L3 E0-E0, Sent to [my-cluster-bookkeeper-7.my-cluster-bookkeeper.pulsar.svc.cluster.local:3181, my-cluster-bookkeeper-1.my-cluster-bookkeeper.pulsar.svc.cluster.local:3181], Heard from [] : bitset = {}, Error = 'Bookie operation timeout'. First unread entry is (-1, rc = null)
   2023-01-19T13:32:45,290+0000 [BookKeeperClientWorker-OrderedExecutor-3-0] ERROR org.apache.pulsar.broker.service.schema.SchemaRegistryServiceImpl - [pulsar/my-cluster-broker-0.my-cluster-broker.pulsar.svc.cluster.local:8080/healthcheck] Put schema failed
   java.util.concurrent.CompletionException: org.apache.pulsar.broker.service.schema.exceptions.SchemaException: Bookie operation timeout -  ledger=3 - operation=Failed to read entry - entry=0
   	at java.util.concurrent.CompletableFuture.encodeRelay(CompletableFuture.java:367) ~[?:?]
       at java.util.concurrent.CompletableFuture.completeRelay(CompletableFuture.java:376) ~[?:?]
   	at java.util.concurrent.CompletableFuture$UniRelay.tryFire(CompletableFuture.java:1019) ~[?:?]
   	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506) ~[?:?]
   	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2088) ~[?:?]
   	at org.apache.pulsar.broker.service.schema.SchemaRegistryServiceImpl.lambda$trimDeletedSchemaAndGetList$24(SchemaRegistryServiceImpl.java:463) ~[com.datastax.oss-pulsar-broker-2.10.3.0.jar:2.10.3.0]
   	at java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:930) ~[?:?]
   	at java.util.concurrent.CompletableFuture$UniHandle.tryFire(CompletableFuture.java:907) ~[?:?]
   	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506) ~[?:?]
   	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2088) ~[?:?]
   	at org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage$Functions.lambda$getLedgerEntry$0(BookkeeperSchemaStorage.java:651) ~[com.datastax.oss-pulsar-broker-2.10.3.0.jar:2.10.3.0]
   	at org.apache.bookkeeper.client.LedgerHandle$6.onFailure(LedgerHandle.java:825) ~[com.datastax.oss-bookkeeper-server-4.14.6.1.0.0.jar:4.14.6.1.0.0]
   	at org.apache.bookkeeper.common.concurrent.FutureEventListener.accept(FutureEventListener.java:38) ~[com.datastax.oss-bookkeeper-common-4.14.6.1.0.0.jar:4.14.6.1.0.0]
   	at org.apache.bookkeeper.common.concurrent.FutureEventListener.accept(FutureEventListener.java:26) ~[com.datastax.oss-bookkeeper-common-4.14.6.1.0.0.jar:4.14.6.1.0.0]
   	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:859) ~[?:?]
   	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:837) ~[?:?]
   	at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:478) ~[?:?]
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) ~[?:?]
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) ~[?:?]
   	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[io.netty-netty-common-4.1.86.Final.jar:4.1.86.Final]
   	at java.lang.Thread.run(Thread.java:829) ~[?:?]
   Caused by: org.apache.pulsar.broker.service.schema.exceptions.SchemaException: Bookie operation timeout -  ledger=3 - operation=Failed to read entry - entry=0
   	at org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage.bkException(BookkeeperSchemaStorage.java:707) ~[com.datastax.oss-pulsar-broker-2.10.3.0.jar:2.10.3.0]
   	... 11 more
   ```
   
   ***To Reproduce***
   
   Steps to reproduce the behavior:
   1. Deploy pulsar via the [DataStax pulsar helmchart](https://github.com/datastax/pulsar-helm-chart), using the values:
   ```
   image:
     brokerSts:
       repository: dsmmirelli/lunastreaming-all
       pullPolicy: IfNotPresent
       imagePullSecrets:
         - name: aws-ecr-bellburnell-cred
       tag: bk-cef988f1
     bookkeeper:
       repository: dsmmirelli/lunastreaming-all
       pullPolicy: IfNotPresent
       imagePullSecrets:
         - name: aws-ecr-bellburnell-cred
       tag: bk-cef988f1
   bookkeeper:
     component: bookkeeper
     replicaCount: 3
     annotations:
       prometheus.io/scrape: "true"
       prometheus.io/port: "8000"
     tolerations:
       - key: tier
         effect: NoSchedule
         operator: Equal
         value: pulsar
     probe:
       port: 3181
       initial: 10
       period: 30
     resources:
       requests:
         memory: 9Gi
         cpu: 1
     configData:
       BOOKIE_MEM: "\"-Xms4500m -Xmx4500m -XX:MaxDirectMemorySize=4500m -Dio.netty.recycler.linkCapacity=1024 -XX:+ParallelRefProcEnabled -XX:+DisableExplicitGC -XX:+ExitOnOutOfMemoryError -XX:+PerfDisableSharedMem\""
       BOOKIE_GC: "\"-XX:+UseG1GC -XX:MaxGCPauseMillis=10\""
       BOOKIE_EXTRA_OPTS: "\"-Dpulsar.allocator.leak_detection=Simple -Dio.netty.leakDetectionLevel=simple\""
       statsProviderClass: org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider
       useHostNameAsBookieID: "true"
       majorCompactionInterval: "1200"
       minorCompactionInterval: "600"
       compactionRateByEntries: "5000"
       gcWaitTime: "300000"
       PULSAR_PREFIX_diskUsageLwmThreshold: "0.90"
       PULSAR_PREFIX_isForceGCAllowWhenNoSpace: "true"
     service:
       annotations:
         publishNotReadyAddresses: "true"
       ports:
         - name: server
           port: 3181
   brokerSts:
     component: broker
     replicaCount: 1
     annotations:
       prometheus.io/scrape: "true"
       prometheus.io/port: "8080"
     tolerations:
       - key: tier
         effect: NoSchedule
         operator: Equal
         value: pulsar
     functionsWorkerEnabled: no
     webSocketServiceEnabled: no
     ledger:
       defaultEnsembleSize: 2
       defaultAckQuorum: 2
       defaultWriteQuorum: 2
     probe:
       port: 8443
       initial: 10
       period: 30
     resources:
       requests:
         memory: 10Gi
         cpu: 5
         #  authenticationProviders: "com.datastax.oss.pulsar.auth.AuthenticationProviderOpenID"
     configData:
       PULSAR_MEM: "\"-Xms5g -Xmx5g -XX:MaxDirectMemorySize=5g -Dio.netty.recycler.linkCapacity=1024 -XX:+ParallelRefProcEnabled -XX:+DisableExplicitGC -XX:+ExitOnOutOfMemoryError -XX:+PerfDisableSharedMem -Dpulsar.allocator.leak_detection=Simple -Dio.netty.leakDetectionLevel=simple\""
       PULSAR_GC: "\"-XX:+UseG1GC -XX:MaxGCPauseMillis=10\""
       exposeTopicLevelMetricsInPrometheus: "true"
       brokerDeleteInactiveTopicsEnabled: "false"
       brokerDeleteInactivePartitionedTopicMetadataEnabled: "false"
       forceDeleteNamespaceAllowed: "true"
       topicPublisherThrottlingTickTimeMillis: "10"
       brokerPublisherThrottlingTickTimeMillis: "50"
       managedLedgerMinLedgerRolloverTimeMinutes: "15"
       managedLedgerMaxLedgerRolloverTimeMinutes: "240"
       managedLedgerMaxSizePerLedgerMbytes: "2048"
       backlogQuotaDefaultRetentionPolicy: "producer_exception"
       dispatcherMaxReadBatchSize: "1000"
       allowAutoTopicCreation: "true"
       exposeConsumerLevelMetricsInPrometheus: "false"
       subscriptionExpirationTimeMinutes: "20160"
       topicFencingTimeoutSeconds: "5"
       replicationConnectionsPerBroker: "1"
       forceDeleteTenantAllowed: "true"
       managedLedgerNumSchedulerThreads: "8"
       webSocketNumIoThreads: "8"
       managedLedgerNumWorkerThreads: "8"
       numWorkerThreadsForNonPersistentTopic: "8"
       webSocketConnectionsPerBroker: "8"
       zooKeeperCacheExpirySeconds: "300"
       zooKeeperOperationTimeoutSeconds: "30"
       zooKeeperSessionTimeoutMillis: "30000"
       zookeeperSessionExpiredPolicy: shutdown
       loadBalancerOverrideBrokerNicSpeedGbps: "128"
       # cache parameters tuning
       loadBalancerLoadSheddingStrategy: "org.apache.pulsar.broker.loadbalance.impl.OverloadShedder"
       managedLedgerCacheEvictionFrequency: '0.2'
       managedLedgerCacheEvictionTimeThresholdMillis: "10000"
       managedLedgerCacheSizeMB: '512'
       managedLedgerCursorBackloggedThreshold: '1000'
       managedLedgerMaxBacklogBetweenCursorsForCaching: '1000000'
       managedLedgerMaxReadsInFlightSizeInMB: '100'
       managedLedgerMinimumBacklogCursorsForCaching: '4'
       managedLedgerMinimumBacklogEntriesForCaching: '1'
   ```
   Where `dsmmirelli/lunastreaming-all:bk-cef988f1` is the image with bk version containing https://github.com/apache/bookkeeper/pull/3528. 
   2. Observe the aforementioned error in the broker logs.
   
   ***Expected behavior***
   
   The broker statefulset should start smoothly with BookKeeper ds-4.14.6.1.0.0. 
   
   ***Screenshots***
   
   If applicable, add screenshots to help explain your problem.
   
   ***Additional context***
   
   1. Changing the brokers / bookies' image to `dsmmirelli/lunastreaming-all:bk-dc855d22`, the broker starts smoothly. That is why we expect the regression to be in https://github.com/apache/bookkeeper/pull/3528.
   2. Changing the docker image on the bookies to (not) contain the breaking change doesn't make a difference, so the issue seems to be on the BookKeeper client running on the pulsar broker.
   


-- 
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@bookkeeper.apache.org.apache.org

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


[GitHub] [bookkeeper] hangc0276 commented on issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema

Posted by "hangc0276 (via GitHub)" <gi...@apache.org>.
hangc0276 commented on issue #3751:
URL: https://github.com/apache/bookkeeper/issues/3751#issuecomment-1427202950

   > @merlimat has been fixed this issue through #3560
   
   @zymap I will cherry-pick this PR into branch-4.14, and trigger a new RC for BookKeeper 4.14.7.


-- 
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@bookkeeper.apache.org

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


[GitHub] [bookkeeper] mattisonchao commented on issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema

Posted by "mattisonchao (via GitHub)" <gi...@apache.org>.
mattisonchao commented on issue #3751:
URL: https://github.com/apache/bookkeeper/issues/3751#issuecomment-1414953713

   Would you mind providing the memory dump (broker and bookie) to show up which thread blocking causes this request is a timeout? 
   


-- 
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@bookkeeper.apache.org

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


[GitHub] [bookkeeper] eolivelli commented on issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema

Posted by "eolivelli (via GitHub)" <gi...@apache.org>.
eolivelli commented on issue #3751:
URL: https://github.com/apache/bookkeeper/issues/3751#issuecomment-1413735697

   @MMirelli here we are missing the Bookie side logs that explain why reverting https://github.com/apache/bookkeeper/pull/3528 is the cure


-- 
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@bookkeeper.apache.org

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


[GitHub] [bookkeeper] horizonzy commented on issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema

Posted by "horizonzy (via GitHub)" <gi...@apache.org>.
horizonzy commented on issue #3751:
URL: https://github.com/apache/bookkeeper/issues/3751#issuecomment-1423958186

   Could you check the timeoutOpLogger metrics, I want to know the value of the timeout.
   ```
           void timeout() {
               errorOut(BKException.Code.TimeoutException);
               timeoutOpLogger.registerSuccessfulEvent(latency(),
                                                       TimeUnit.NANOSECONDS);
           }
   
   ```


-- 
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@bookkeeper.apache.org

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


[GitHub] [bookkeeper] MMirelli commented on issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema

Posted by "MMirelli (via GitHub)" <gi...@apache.org>.
MMirelli commented on issue #3751:
URL: https://github.com/apache/bookkeeper/issues/3751#issuecomment-1423882617

   Unfortunately, I cannot share full memory dumps / logs generated by our internal testing. 
   
   Looking at the locking section of the broker JFR file I generated, the main bottleneck seems to be in the `BookKeeperClientScheduler-OrderedScheduler` thread having the maximum block time of 209microseconds. That is executing `BookieClientImpl.monitorPendingOperations()`. 
   
   The method which is called more often is instead `.__lock_text_start(k)` (53 calls in 1 minute), which is likely to be the way a process acquires locks in Linux. That is called by `PulsarRegistrationClient.updatedBookies(Notification)` in a `pulsar-io` thread.
   
   The bookie which, from the broker's logs appears to have problems, is instead pretty much idling. The highest blocking time other than the one to create Object is 36 microseconds and it's caused by `org.apache.zookeeper.ClientCnxnSocketNIO`. Also there there are few calls of `.__lock_text_start(k)`, but only 15 calls in 1 minute. These are all related to communication with ZK (8 using directly `org.apache.zookeeper.ClientCnxnSocketNIO`).
   


-- 
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@bookkeeper.apache.org

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


[GitHub] [bookkeeper] zymap commented on issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema

Posted by "zymap (via GitHub)" <gi...@apache.org>.
zymap commented on issue #3751:
URL: https://github.com/apache/bookkeeper/issues/3751#issuecomment-1427182286

   I and @hangc0276 take a deep look into this issue. We found the root cause is that Netty casts the message to ReferenceCounted. When the bookkeeper gets the ReadResponse, we can not parse it correctly [at here](https://github.com/apache/bookkeeper/blob/591ceb971a7eda7ce5252f88c5cf694f3f4c30b2/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java#L1344) and then lose the response for read. 
   
   In Netty, when we fireChannelRead, it passes the message as an Object to the next channel.
   
   ```
       @Override
       public ChannelHandlerContext fireChannelRead(final Object msg) {
           invokeChannelRead(findContextInbound(MASK_CHANNEL_READ), msg);
           return this;
       }
   
       static void invokeChannelRead(final AbstractChannelHandlerContext next, Object msg) {
           final Object m = next.pipeline.touch(ObjectUtil.checkNotNull(msg, "msg"), next);
           EventExecutor executor = next.executor();
           if (executor.inEventLoop()) {
               next.invokeChannelRead(m);
           } else {
               executor.execute(new Runnable() {
                   @Override
                   public void run() {
                       next.invokeChannelRead(m);
                   }
               });
           }
       }
   ```
   
   When the message comes out from the ResponseEnDeCoderPreV3 [decoding](https://github.com/apache/bookkeeper/blob/591ceb971a7eda7ce5252f88c5cf694f3f4c30b2/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtoEncoding.java#L577), it should be a ReadResponse. As we show upon, Netty passes it as an Object, then it will go to `final Object m = next.pipeline.touch(ObjectUtil.checkNotNull(msg, "msg"), next);`.
   
   DefaultChannelPipeline.touch:
   ```
       final Object touch(Object msg, AbstractChannelHandlerContext next) {
           return touch ? ReferenceCountUtil.touch(msg, next) : msg;
       }
   ```
   ReferenceCountUtil.touch(msg, next)
   ```
       public static <T> T touch(T msg, Object hint) {
           if (msg instanceof ReferenceCounted) {
               return (T) ((ReferenceCounted) msg).touch(hint);
           }
           return msg;
       }
   ```
   
   When it goes to `ReferenceCountUtil.touch(msg, next)`, the change https://github.com/apache/bookkeeper/pull/3528 will make it differently. 
   Before the change, the ReadResponse is not a ReferenceCounted, it will return directly. The message type still is a ReadResponse.
   But since we change the ReadResponse to a ReferenceCounted, it will use the [touch method](https://github.com/apache/bookkeeper/pull/3528/files#diff-5bbab7efe840cf7924788317f180e2e6bc99c0e9dabfccc1dea5d5b26ed413ebR483) of ReadResponse. Then the type changed to ByteBuf.
   
   I will push a fix for that. And If you have time, please help to verify it! @MMirelli @eolivelli 


-- 
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@bookkeeper.apache.org

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


[GitHub] [bookkeeper] horizonzy commented on issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema

Posted by "horizonzy (via GitHub)" <gi...@apache.org>.
horizonzy commented on issue #3751:
URL: https://github.com/apache/bookkeeper/issues/3751#issuecomment-1427204691

   Add more context: 
   The method io.netty.channel.DefaultChannelPipeline#touch is for memory leak detection.
   ```
       private final boolean touch = ResourceLeakDetector.isEnabled();
   
   
      final Object touch(Object msg, AbstractChannelHandlerContext next) {
           return touch ? ReferenceCountUtil.touch(msg, next) : msg;
       }
   ```
   
   io.netty.util.ResourceLeakDetector#isEnabled:
   ```
       public static boolean isEnabled() {
           return getLevel().ordinal() > ResourceLeakDetector.Level.DISABLED.ordinal();
       }
   ```
   
   io.netty.util.ResourceLeakDetector.Level:
   ```
       public static enum Level {
           DISABLED,
           SIMPLE,
           ADVANCED,
           PARANOID;
       }
   ```
   If the user didn't config ResourceLeakDetector LEVEL, it may work well. but If the user config the ResourceLeakDetector.LEVEL greater than DISABLED, the problem will happen.


-- 
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@bookkeeper.apache.org

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


[GitHub] [bookkeeper] MMirelli closed issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema

Posted by "MMirelli (via GitHub)" <gi...@apache.org>.
MMirelli closed issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema
URL: https://github.com/apache/bookkeeper/issues/3751


-- 
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@bookkeeper.apache.org

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


[GitHub] [bookkeeper] zymap commented on issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema

Posted by "zymap (via GitHub)" <gi...@apache.org>.
zymap commented on issue #3751:
URL: https://github.com/apache/bookkeeper/issues/3751#issuecomment-1427183142

   @merlimat has been fixed this issue through https://github.com/apache/bookkeeper/pull/3560


-- 
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@bookkeeper.apache.org

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


[GitHub] [bookkeeper] MMirelli commented on issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema

Posted by "MMirelli (via GitHub)" <gi...@apache.org>.
MMirelli commented on issue #3751:
URL: https://github.com/apache/bookkeeper/issues/3751#issuecomment-1427637729

   I have run my sanity check on the HEAD of branch-2.14 (`917fa6abca93d7d20f18eddcda1994b4576130d7`). It fixes the issue I reported. Thank you for the fix. 
   
   I will test the rc too, once out. 
   
   @hangc0276 @zymap @eolivelli 


-- 
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@bookkeeper.apache.org

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


[GitHub] [bookkeeper] MMirelli commented on issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema

Posted by "MMirelli (via GitHub)" <gi...@apache.org>.
MMirelli commented on issue #3751:
URL: https://github.com/apache/bookkeeper/issues/3751#issuecomment-1411654240

   Yes reverting #3528 it works ok.


-- 
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@bookkeeper.apache.org

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


[GitHub] [bookkeeper] MMirelli commented on issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema

Posted by "MMirelli (via GitHub)" <gi...@apache.org>.
MMirelli commented on issue #3751:
URL: https://github.com/apache/bookkeeper/issues/3751#issuecomment-1432806723

   For reference I tested this, by copying 
   ```
   org.apache.bookkeeper-bookkeeper-common-4.14.7.jar
   org.apache.bookkeeper-bookkeeper-proto-4.14.7.jar
   org.apache.bookkeeper-bookkeeper-server-4.14.7.jar
   org.apache.bookkeeper-bookkeeper-tools-framework-4.14.7.jar
   ```
   into the `/pulsar/lib` of the pulsar image having problems and removed the corresponding old libs. 
   
   Using the resulting image on a broker pod, I couldn't spot the error reported in this issue.


-- 
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@bookkeeper.apache.org

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


[GitHub] [bookkeeper] horizonzy commented on issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema

Posted by "horizonzy (via GitHub)" <gi...@apache.org>.
horizonzy commented on issue #3751:
URL: https://github.com/apache/bookkeeper/issues/3751#issuecomment-1408694269

   Only revert #3528, it works well?


-- 
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@bookkeeper.apache.org

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


[GitHub] [bookkeeper] horizonzy commented on issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema

Posted by "horizonzy (via GitHub)" <gi...@apache.org>.
horizonzy commented on issue #3751:
URL: https://github.com/apache/bookkeeper/issues/3751#issuecomment-1413438292

   Maybe the problem is not introduced by #3528. #3528 didn't change the bk client behavior.
   So we use the newest bk server, only make pulsar rollback to `dsmmirelli/lunastreaming-all:bk-dc855d22`, the pulsar still works smoothly. 
   So the problem is not related to bk, there are some unknown problems in pulsar.
   


-- 
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@bookkeeper.apache.org

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


[GitHub] [bookkeeper] MMirelli commented on issue #3751: Pulsar broker cannot start due to bookies timing out on putting schema

Posted by "MMirelli (via GitHub)" <gi...@apache.org>.
MMirelli commented on issue #3751:
URL: https://github.com/apache/bookkeeper/issues/3751#issuecomment-1432798590

   I tested bookkeeper-4.14.7-rc1. I confirm #3560 fixes the issue.


-- 
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@bookkeeper.apache.org

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