-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
Fix create partitioned topic with a substring of an existing topic name. #6478
Merged
Merged
Changes from all commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
830ac9b
Fix create partitioned topic with a substring of an existing topic name.
codelipenghui d49e680
Fix tests
codelipenghui 2495e5c
handling resume async response exceptionally
codelipenghui 12825ba
Add more logs
codelipenghui 99f5486
Fix log
codelipenghui dd0077c
Fix comments
codelipenghui a3f4470
sync global zk first
codelipenghui File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -27,6 +27,7 @@ | |
|
||
import java.net.MalformedURLException; | ||
import java.net.URI; | ||
import java.util.ArrayList; | ||
import java.util.List; | ||
import java.util.Set; | ||
import java.util.concurrent.CompletableFuture; | ||
|
@@ -36,6 +37,7 @@ | |
|
||
import javax.servlet.ServletContext; | ||
import javax.ws.rs.WebApplicationException; | ||
import javax.ws.rs.container.AsyncResponse; | ||
import javax.ws.rs.core.Response; | ||
import javax.ws.rs.core.Response.Status; | ||
import javax.ws.rs.core.UriBuilder; | ||
|
@@ -46,6 +48,7 @@ | |
import org.apache.pulsar.broker.cache.LocalZooKeeperCacheService; | ||
import org.apache.pulsar.broker.web.PulsarWebResource; | ||
import org.apache.pulsar.broker.web.RestException; | ||
import org.apache.pulsar.common.api.proto.PulsarApi; | ||
import org.apache.pulsar.common.naming.Constants; | ||
import org.apache.pulsar.common.naming.NamespaceBundle; | ||
import org.apache.pulsar.common.naming.NamespaceBundleFactory; | ||
|
@@ -255,35 +258,42 @@ protected List<String> getListOfNamespaces(String property) throws Exception { | |
return namespaces; | ||
} | ||
|
||
protected void tryCreatePartitionsAsync(int numPartitions) { | ||
protected CompletableFuture<Void> tryCreatePartitionsAsync(int numPartitions) { | ||
if (!topicName.isPersistent()) { | ||
return; | ||
return CompletableFuture.completedFuture(null); | ||
} | ||
List<CompletableFuture<Void>> futures = new ArrayList<>(numPartitions); | ||
for (int i = 0; i < numPartitions; i++) { | ||
tryCreatePartitionAsync(i); | ||
futures.add(tryCreatePartitionAsync(i, null)); | ||
} | ||
return FutureUtil.waitForAll(futures); | ||
} | ||
|
||
private void tryCreatePartitionAsync(final int partition) { | ||
private CompletableFuture<Void> tryCreatePartitionAsync(final int partition, CompletableFuture<Void> reuseFuture) { | ||
CompletableFuture<Void> result = reuseFuture == null ? new CompletableFuture<>() : reuseFuture; | ||
zkCreateOptimisticAsync(localZk(), ZkAdminPaths.managedLedgerPath(topicName.getPartition(partition)), new byte[0], | ||
(rc, s, o, s1) -> { | ||
if (KeeperException.Code.OK.intValue() == rc) { | ||
if (log.isDebugEnabled()) { | ||
log.debug("[{}] Topic partition {} created.", clientAppId(), | ||
topicName.getPartition(partition)); | ||
} | ||
result.complete(null); | ||
} else if (KeeperException.Code.NODEEXISTS.intValue() == rc) { | ||
log.info("[{}] Topic partition {} is exists, doing nothing.", clientAppId(), | ||
topicName.getPartition(partition)); | ||
result.completeExceptionally(KeeperException.create(KeeperException.Code.NODEEXISTS)); | ||
} else if (KeeperException.Code.BADVERSION.intValue() == rc) { | ||
log.warn("[{}] Fail to create topic partition {} with concurrent modification, retry now.", | ||
clientAppId(), topicName.getPartition(partition)); | ||
tryCreatePartitionAsync(partition); | ||
tryCreatePartitionAsync(partition, result); | ||
} else { | ||
log.error("[{}] Fail to create topic partition {}", clientAppId(), | ||
topicName.getPartition(partition), KeeperException.create(KeeperException.Code.get(rc))); | ||
result.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc))); | ||
} | ||
}); | ||
return result; | ||
} | ||
|
||
protected NamespaceName namespaceName; | ||
|
@@ -707,4 +717,98 @@ protected List<String> getPartitionedTopicList(TopicDomain topicDomain) { | |
partitionedTopics.sort(null); | ||
return partitionedTopics; | ||
} | ||
|
||
protected void internalCreatePartitionedTopic(AsyncResponse asyncResponse, int numPartitions) { | ||
try { | ||
validateAdminAccessForTenant(topicName.getTenant()); | ||
} catch (Exception e) { | ||
log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e); | ||
resumeAsyncResponseExceptionally(asyncResponse, e); | ||
return; | ||
} | ||
if (numPartitions <= 0) { | ||
asyncResponse.resume(new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 0")); | ||
return; | ||
} | ||
checkTopicExistsAsync(topicName).thenAccept(exists -> { | ||
if (exists) { | ||
log.warn("[{}] Failed to create already existing topic {}", clientAppId(), topicName); | ||
asyncResponse.resume(new RestException(Status.CONFLICT, "This topic already exists")); | ||
} else { | ||
try { | ||
String path = ZkAdminPaths.partitionedTopicPath(topicName); | ||
byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions)); | ||
zkCreateOptimisticAsync(globalZk(), path, data, (rc, s, o, s1) -> { | ||
if (KeeperException.Code.OK.intValue() == rc) { | ||
globalZk().sync(path, (rc2, s2, ctx) -> { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. same comment as above There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This should happen in the callback of |
||
if (KeeperException.Code.OK.intValue() == rc2) { | ||
log.info("[{}] Successfully created partitioned topic {}", clientAppId(), topicName); | ||
tryCreatePartitionsAsync(numPartitions).thenAccept(v -> { | ||
log.info("[{}] Successfully created partitions for topic {}", clientAppId(), topicName); | ||
asyncResponse.resume(Response.noContent().build()); | ||
}).exceptionally(e -> { | ||
log.error("[{}] Failed to create partitions for topic {}", clientAppId(), topicName); | ||
// The partitioned topic is created but there are some partitions create failed | ||
asyncResponse.resume(new RestException(e)); | ||
return null; | ||
}); | ||
} else { | ||
log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, KeeperException.create(KeeperException.Code.get(rc2))); | ||
asyncResponse.resume(new RestException(KeeperException.create(KeeperException.Code.get(rc2)))); | ||
} | ||
}, null); | ||
} else if (KeeperException.Code.NODEEXISTS.intValue() == rc) { | ||
log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), topicName); | ||
asyncResponse.resume(new RestException(Status.CONFLICT, "Partitioned topic already exists")); | ||
} else if (KeeperException.Code.BADVERSION.intValue() == rc) { | ||
log.warn("[{}] Failed to create partitioned topic {}: concurrent modification", clientAppId(), | ||
topicName); | ||
asyncResponse.resume(new RestException(Status.CONFLICT, "Concurrent modification")); | ||
} else { | ||
log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, KeeperException.create(KeeperException.Code.get(rc))); | ||
asyncResponse.resume(new RestException(KeeperException.create(KeeperException.Code.get(rc)))); | ||
} | ||
}); | ||
} catch (Exception e) { | ||
log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e); | ||
resumeAsyncResponseExceptionally(asyncResponse, e); | ||
} | ||
} | ||
}).exceptionally(ex -> { | ||
log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, ex); | ||
resumeAsyncResponseExceptionally(asyncResponse, ex); | ||
return null; | ||
}); | ||
} | ||
|
||
/** | ||
* Check the exists topics contains the given topic. | ||
* Since there are topic partitions and non-partitioned topics in Pulsar, must ensure both partitions | ||
* and non-partitioned topics are not duplicated. So, if compare with a partition name, we should compare | ||
* to the partitioned name of this partition. | ||
* | ||
* @param topicName given topic name | ||
*/ | ||
protected CompletableFuture<Boolean> checkTopicExistsAsync(TopicName topicName) { | ||
return pulsar().getNamespaceService().getListOfTopics(topicName.getNamespaceObject(), | ||
PulsarApi.CommandGetTopicsOfNamespace.Mode.ALL) | ||
.thenCompose(topics -> { | ||
boolean exists = false; | ||
for (String topic : topics) { | ||
if (topicName.getPartitionedTopicName().equals(TopicName.get(topic).getPartitionedTopicName())) { | ||
exists = true; | ||
break; | ||
} | ||
} | ||
return CompletableFuture.completedFuture(exists); | ||
}); | ||
} | ||
|
||
protected void resumeAsyncResponseExceptionally(AsyncResponse asyncResponse, Throwable throwable) { | ||
if (throwable instanceof WebApplicationException) { | ||
asyncResponse.resume(throwable); | ||
} else { | ||
asyncResponse.resume(new RestException(throwable)); | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@codelipenghui why it is
globalZk()
here?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The partitioned topic metadata is stored in the global zookeeper.