This is an automated email from the ASF dual-hosted git repository. guangning pushed a commit to branch branch-2.5 in repository https://gitbox.apache.org/repos/asf/pulsar.git
commit 5df30c0a3f0d10a72d78200655206b1b52433f59 Author: lipenghui <[email protected]> AuthorDate: Sat Feb 8 10:05:07 2020 +0800 Create managed ledger path on local zookeeper when create partitions (#6189) ### Motivation Create managed ledger path on local zookeeper when creating partitions for a partitioned topic. ### Modifications Change globalZk() to localZk() when creating partitions. ### Verifying this change PartitionCreationTest can cover this change, since we use the same zookeeper for the unit test in ProducerConsumerBase, so the test passed before. --- .../src/main/java/org/apache/pulsar/broker/admin/AdminResource.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java index e9d559e..b6330f6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/AdminResource.java @@ -113,8 +113,8 @@ public abstract class AdminResource extends PulsarWebResource { ZkUtils.createFullPathOptimistic(globalZk(), path, content, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); } - protected void zkCreateOptimisticAsync(String path, byte[] content, AsyncCallback.StringCallback callback) { - ZkUtils.asyncCreateFullPathOptimistic(globalZk(), path, content, ZooDefs.Ids.OPEN_ACL_UNSAFE, + protected void zkCreateOptimisticAsync(ZooKeeper zk, String path, byte[] content, AsyncCallback.StringCallback callback) { + ZkUtils.asyncCreateFullPathOptimistic(zk, path, content, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, callback, null); } @@ -265,7 +265,7 @@ public abstract class AdminResource extends PulsarWebResource { } private void tryCreatePartitionAsync(final int partition) { - zkCreateOptimisticAsync(ZkAdminPaths.managedLedgerPath(topicName.getPartition(partition)), new byte[0], + zkCreateOptimisticAsync(localZk(), ZkAdminPaths.managedLedgerPath(topicName.getPartition(partition)), new byte[0], (rc, s, o, s1) -> { if (KeeperException.Code.OK.intValue() == rc) { if (log.isDebugEnabled()) {
