Skip to content

Conversation

@nicoloboschi
Copy link
Contributor

@nicoloboschi nicoloboschi commented Jan 21, 2022

Motivation

The PR #13296 introduces a bug that breaks the functions worker in certain types of configuration.

To reproduce:

  • Use zookkeeper entry in initialize-cluster-metadata command (which is the common configuration before PIP-45) (see the unit test for reference)
  • With this change the DL configuration does not use the correct ledgersPath (/ledgers by default) anymore and instead, it uses the root path (ledgersPath="").
  • Start functions-worker and install a connector. This trigger the DLog client to upload the Nar file to the bookies. Since the ledgers path does not match with the actual BK ledgers path, the client is not able to find an available bookie.
 
java.io.IOException: org.apache.bookkeeper.client.BKException$ZKException: Error while using ZooKeeper
	at org.apache.distributedlog.BookKeeperClient.commonInitialization(BookKeeperClient.java:123) ~[org.apache.distributedlog-distributedlog-core-4.14.4.jar:4.14.4]
	at org.apache.distributedlog.BookKeeperClient.initialize(BookKeeperClient.java:172) ~[org.apache.distributedlog-distributedlog-core-4.14.4.jar:4.14.4]
	at org.apache.distributedlog.BookKeeperClient.get(BookKeeperClient.java:199) ~[org.apache.distributedlog-distributedlog-core-4.14.4.jar:4.14.4]
	at org.apache.distributedlog.BookKeeperClient.createLedger(BookKeeperClient.java:211) ~[org.apache.distributedlog-distributedlog-core-4.14.4.jar:4.14.4]
	at org.apache.distributedlog.bk.SimpleLedgerAllocator.allocateLedger(SimpleLedgerAllocator.java:370) ~[org.apache.distributedlog-distributedlog-core-4.14.4.jar:4.14.4]
	at org.apache.distributedlog.bk.SimpleLedgerAllocator.allocate(SimpleLedgerAllocator.java:271) ~[org.apache.distributedlog-distributedlog-core-4.14.4.jar:4.14.4]
	at org.apache.distributedlog.bk.LedgerAllocatorDelegator.allocate(LedgerAllocatorDelegator.java:67) ~[org.apache.distributedlog-distributedlog-core-4.14.4.jar:4.14.4]
	at org.apache.distributedlog.impl.logsegment.BKLogSegmentAllocator.allocate(BKLogSegmentAllocator.java:55) ~[org.apache.distributedlog-distributedlog-core-4.14.4.jar:4.14.4]
	at org.apache.distributedlog.BKLogWriteHandler.doStartLogSegment(BKLogWriteHandler.java:571) ~[org.apache.distributedlog-distributedlog-core-4.14.4.jar:4.14.4]
	at org.apache.distributedlog.BKLogWriteHandler$10.onSuccess(BKLogWriteHandler.java:538) ~[org.apache.distributedlog-distributedlog-core-4.14.4.jar:4.14.4]
	at org.apache.distributedlog.BKLogWriteHandler$10.onSuccess(BKLogWriteHandler.java:530) ~[org.apache.distributedlog-distributedlog-core-4.14.4.jar:4.14.4]
	at org.apache.bookkeeper.common.concurrent.FutureEventListener.accept(FutureEventListener.java:42) ~[org.apache.bookkeeper-bookkeeper-common-4.14.4.jar:4.14.4]
	at org.apache.bookkeeper.common.concurrent.FutureEventListener.accept(FutureEventListener.java:26) ~[org.apache.bookkeeper-bookkeeper-common-4.14.4.jar:4.14.4]
	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:859) ~[?:?]
	at java.util.concurrent.CompletableFuture.uniWhenCompleteStage(CompletableFuture.java:883) ~[?:?]
	at java.util.concurrent.CompletableFuture.whenComplete(CompletableFuture.java:2251) ~[?:?]
	at org.apache.distributedlog.BKLogWriteHandler.asyncStartLogSegment(BKLogWriteHandler.java:530) ~[org.apache.distributedlog-distributedlog-core-4.14.4.jar:4.14.4]
	at org.apache.distributedlog.BKAbstractLogWriter.lambda$asyncStartNewLogSegment$1(BKAbstractLogWriter.java:379) ~[org.apache.distributedlog-distributedlog-core-4.14.4.jar:4.14.4]
	at java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:1072) [?:?]
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506) [?:?]
	at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2073) [?:?]
	at org.apache.bookkeeper.common.concurrent.FutureUtils$ListFutureProcessor.run(FutureUtils.java:229) [org.apache.bookkeeper-bookkeeper-common-4.14.4.jar:4.14.4]
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) [?:?]
	at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125) [com.google.guava-guava-30.1-jre.jar:?]
	at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69) [com.google.guava-guava-30.1-jre.jar:?]
	at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78) [com.google.guava-guava-30.1-jre.jar:?]
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) [?:?]
	at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?]
	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304) [?:?]
	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.72.Final.jar:4.1.72.Final]
	at java.lang.Thread.run(Thread.java:829) [?:?]
Caused by: org.apache.bookkeeper.client.BKException$ZKException: Error while using ZooKeeper
	at org.apache.bookkeeper.discover.ZKRegistrationClient.lambda$getChildren$4(ZKRegistrationClient.java:351) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$25$1.processResult(ZooKeeperClient.java:1177) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
	at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:678) ~[org.apache.zookeeper-zookeeper-3.6.3.jar:3.6.3]
	at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:563) ~[org.apache.zookeeper-zookeeper-3.6.3.jar:3.6.3]
Caused by: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /available
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:118) ~[org.apache.zookeeper-zookeeper-3.6.3.jar:3.6.3]
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:54) ~[org.apache.zookeeper-zookeeper-3.6.3.jar:3.6.3]
	at org.apache.bookkeeper.discover.ZKRegistrationClient.lambda$getChildren$4(ZKRegistrationClient.java:350) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$25$1.processResult(ZooKeeperClient.java:1177) ~[org.apache.bookkeeper-bookkeeper-server-4.14.4.jar:4.14.4]
	at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:678) ~[org.apache.zookeeper-zookeeper-3.6.3.jar:3.6.3]
	at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:563) ~[org.apache.zookeeper-zookeeper-3.6.3.jar:3.6.3]

Modifications

  • Pass the correct ledgersPath during the cluster metadata initialization
  • BK client in broker now always use the correct ledgersPath

Documentation

  • no-need-doc

@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Jan 21, 2022
@nicoloboschi nicoloboschi marked this pull request as draft January 21, 2022 16:59
@nicoloboschi
Copy link
Contributor Author

/pulsarbot run-failure-checks

@eolivelli
Copy link
Contributor

@MMirelli is this patch fixing your issue ?

@MMirelli
Copy link
Contributor

@MMirelli is this patch fixing your issue ?

Hopefully, I haven't completed the testing yet. But yes it is an attempt to fix #13967.

@nicoloboschi
Copy link
Contributor Author

/pulsarbot run-failure-checks

@nicoloboschi nicoloboschi marked this pull request as ready for review January 26, 2022 20:03
@eolivelli eolivelli requested a review from merlimat January 27, 2022 20:08
@nicoloboschi
Copy link
Contributor Author

/pulsarbot rerun-failure-checks

@nicoloboschi
Copy link
Contributor Author

@MMirelli I've run the test you mentioned and now it works with these changes. Also the integration tests are passing

@eolivelli @lhotari @merlimat @michaeljmarshall PTAL

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@codelipenghui codelipenghui added this to the 2.10.0 milestone Jan 30, 2022
Copy link
Member

@lhotari lhotari left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It might not be related to this PR, but I don't understand the format of the metadata-store: URL. It's really odd that zk: in metadata-store:zk: seems to be optional. @merlimat What's the reasoning for this? Can you also review this PR?

@lhotari lhotari requested a review from Jason918 February 2, 2022 08:21
@lhotari
Copy link
Member

lhotari commented Feb 2, 2022

@Jason918 would you be able to review these changes? thank you

@merlimat
Copy link
Contributor

merlimat commented Feb 2, 2022

It might not be related to this PR, but I don't understand the format of the metadata-store: URL. It's really odd that zk: in metadata-store:zk: seems to be optional. @merlimat What's the reasoning for this? Can you also review this PR?

@lhotari Main reason was to accept my-zk:2181 and interpret it as a ZK address for backward compatibility. Eg: if you're passing just upgrading and you have the zookeeperServers.

When configuring BK access, it has already its own prefix for the BK pluggable metadata. There we use the prefix metadata-store: to load the Pulsar plugin. Everything after that should be the same that the user has configured for Pulsar broker.

@eolivelli
Copy link
Contributor

I have restarted CI. Merging as soon as it passes

@nicoloboschi
Copy link
Contributor Author

/pulsarbot rerun-failure-checks

Copy link
Member

@lhotari lhotari left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good work @nicoloboschi !

@eolivelli eolivelli merged commit 25db606 into apache:master Feb 3, 2022
Nicklee007 pushed a commit to Nicklee007/pulsar that referenced this pull request Apr 20, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

doc-not-needed Your PR changes do not impact docs

Projects

None yet

Development

Successfully merging this pull request may close these issues.

9 participants