From d35e6c1a711c686b0af8072bfff36b11214825ea Mon Sep 17 00:00:00 2001 From: ltamber Date: Sun, 16 Feb 2020 13:27:50 +0800 Subject: [PATCH 01/74] [Issue 5904]Support `unload` all partitions of a partitioned topic (#6187) Fixes #5904 ### Motivation Pulsar supports unload a non-partitioned-topic or a partition of a partitioned topic. If there has a partitioned topic with too many partitions, users need to get all partition and unload them one by one. We need to support unload all partition of a partitioned topic. --- .../pulsar/broker/admin/AdminResource.java | 26 +++++++ .../admin/impl/PersistentTopicsBase.java | 71 ++++++++++++++----- .../broker/admin/v1/NonPersistentTopics.java | 18 ++--- .../broker/admin/v1/PersistentTopics.java | 7 +- .../broker/admin/v2/NonPersistentTopics.java | 13 ++-- .../broker/admin/v2/PersistentTopics.java | 11 ++- .../broker/admin/PersistentTopicsTest.java | 44 +++++++++--- 7 files changed, 146 insertions(+), 44 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 38c57be887f76..722da4f49214c 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 @@ -559,6 +559,32 @@ protected ZooKeeperChildrenCache failureDomainListCache() { return pulsar().getConfigurationCache().failureDomainListCache(); } + protected CompletableFuture getPartitionedTopicMetadataAsync( + TopicName topicName, boolean authoritative, boolean checkAllowAutoCreation) { + validateClusterOwnership(topicName.getCluster()); + // validates global-namespace contains local/peer cluster: if peer/local cluster present then lookup can + // serve/redirect request else fail partitioned-metadata-request so, client fails while creating + // producer/consumer + validateGlobalNamespaceOwnership(topicName.getNamespaceObject()); + + try { + checkConnect(topicName); + } catch (WebApplicationException e) { + validateAdminAccessForTenant(topicName.getTenant()); + } catch (Exception e) { + // unknown error marked as internal server error + log.warn("Unexpected error while authorizing lookup. topic={}, role={}. Error: {}", topicName, + clientAppId(), e.getMessage(), e); + return FutureUtil.failedFuture(e); + } + + if (checkAllowAutoCreation) { + return pulsar().getBrokerService().fetchPartitionedTopicMetadataCheckAllowAutoCreationAsync(topicName); + } else { + return pulsar().getBrokerService().fetchPartitionedTopicMetadataAsync(topicName); + } + } + protected PartitionedTopicMetadata getPartitionedTopicMetadata(TopicName topicName, boolean authoritative, boolean checkAllowAutoCreation) { validateClusterOwnership(topicName.getCluster()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 0add56e311cc4..4a1021f9aa366 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -655,12 +655,63 @@ protected void internalDeletePartitionedTopic(AsyncResponse asyncResponse, boole }); } - protected void internalUnloadTopic(boolean authoritative) { + protected void internalUnloadTopic(AsyncResponse asyncResponse, boolean authoritative) { log.info("[{}] Unloading topic {}", clientAppId(), topicName); if (topicName.isGlobal()) { validateGlobalNamespaceOwnership(namespaceName); } - unloadTopic(topicName, authoritative); + + getPartitionedTopicMetadataAsync(topicName, authoritative, false).whenComplete((meta, t) -> { + if (meta.partitions > 0) { + final List> futures = Lists.newArrayList(); + + for (int i = 0; i < meta.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + futures.add(pulsar().getAdminClient().topics().unloadAsync(topicNamePartition.toString())); + } catch (Exception e) { + log.error("[{}] Failed to unload topic {}", clientAppId(), topicNamePartition, e); + asyncResponse.resume(new RestException(e)); + return; + } + } + + FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable th = exception.getCause(); + if (th instanceof NotFoundException) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, th.getMessage())); + } else { + log.error("[{}] Failed to unload topic {}", clientAppId(), topicName, exception); + asyncResponse.resume(new RestException(exception)); + } + return null; + } + + asyncResponse.resume(Response.noContent().build()); + return null; + }); + } else { + validateAdminAccessForTenant(topicName.getTenant()); + validateTopicOwnership(topicName, authoritative); + + Topic topic = getTopicReference(topicName); + topic.close(false).whenComplete((r, ex) -> { + if (ex != null) { + log.error("[{}] Failed to unload topic {}, {}", clientAppId(), topicName, ex.getMessage(), ex); + asyncResponse.resume(new RestException(ex)); + + } else { + log.info("[{}] Successfully unloaded topic {}", clientAppId(), topicName); + asyncResponse.resume(Response.noContent().build()); + } + }); + } + }).exceptionally(t -> { + Throwable th = t.getCause(); + asyncResponse.resume(new RestException(th)); + return null; + }); } protected void internalDeleteTopic(boolean authoritative, boolean force) { @@ -1893,22 +1944,6 @@ private CompletableFuture createSubscriptions(TopicName topicName, int num return result; } - protected void unloadTopic(TopicName topicName, boolean authoritative) { - validateSuperUserAccess(); - validateTopicOwnership(topicName, authoritative); - try { - Topic topic = getTopicReference(topicName); - topic.close(false).get(); - log.info("[{}] Successfully unloaded topic {}", clientAppId(), topicName); - } catch (NullPointerException e) { - log.error("[{}] topic {} not found", clientAppId(), topicName); - throw new RestException(Status.NOT_FOUND, "Topic does not exist"); - } catch (Exception e) { - log.error("[{}] Failed to unload topic {}, {}", clientAppId(), topicName, e.getMessage(), e); - throw new RestException(e); - } - } - // as described at : (PR: #836) CPP-client old client lib should not be allowed to connect on partitioned-topic. // So, all requests from old-cpp-client (< v1.21) must be rejected. // Pulsar client-java lib always passes user-agent as X-Java-$version. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java index f65d18a4b2394..81759cc840675 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java @@ -169,16 +169,18 @@ public void createPartitionedTopic(@PathParam("property") String property, @Path @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Topic does not exist") }) - public void unloadTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic, + public void unloadTopic(@Suspended final AsyncResponse asyncResponse, @PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("topic") @Encoded String encodedTopic, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - validateTopicName(property, cluster, namespace, encodedTopic); - log.info("[{}] Unloading topic {}", clientAppId(), topicName); - - if (topicName.isGlobal()) { - validateGlobalNamespaceOwnership(namespaceName); + try { + validateTopicName(property, cluster, namespace, encodedTopic); + internalUnloadTopic(asyncResponse, authoritative); + } catch (WebApplicationException wae) { + asyncResponse.resume(wae); + } catch (Exception e) { + asyncResponse.resume(new RestException(e)); } - unloadTopic(topicName, authoritative); } @GET diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java index 203555c66a6a6..ae139a1f0894a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java @@ -225,11 +225,12 @@ public void deletePartitionedTopic(@Suspended final AsyncResponse asyncResponse, @ApiOperation(hidden = true, value = "Unload a topic") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Topic does not exist") }) - public void unloadTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, - @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic, + public void unloadTopic(@Suspended final AsyncResponse asyncResponse, @PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, + @PathParam("topic") @Encoded String encodedTopic, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(property, cluster, namespace, encodedTopic); - internalUnloadTopic(authoritative); + internalUnloadTopic(asyncResponse, authoritative); } @DELETE diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index 219ca0d9b38fb..480e63ba0da8a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -219,6 +219,7 @@ public void createPartitionedTopic( @ApiResponse(code = 503, message = "Failed to validate global cluster configuration"), }) public void unloadTopic( + @Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Specify the tenant", required = true) @PathParam("tenant") String tenant, @ApiParam(value = "Specify the namespace", required = true) @@ -227,12 +228,14 @@ public void unloadTopic( @PathParam("topic") @Encoded String encodedTopic, @ApiParam(value = "Is authentication required to perform this operation") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - validateTopicName(tenant, namespace, encodedTopic); - log.info("[{}] Unloading topic {}", clientAppId(), topicName); - if (topicName.isGlobal()) { - validateGlobalNamespaceOwnership(namespaceName); + try { + validateTopicName(tenant, namespace, encodedTopic); + internalUnloadTopic(asyncResponse, authoritative); + } catch (WebApplicationException wae) { + asyncResponse.resume(wae); + } catch (Exception e) { + asyncResponse.resume(new RestException(e)); } - unloadTopic(topicName, authoritative); } @GET diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index d47d8b587b00c..fbf74c971cdf1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -373,6 +373,7 @@ public void deletePartitionedTopic( @ApiResponse(code = 500, message = "Internal server error"), @ApiResponse(code = 503, message = "Failed to validate global cluster configuration") }) public void unloadTopic( + @Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Specify the tenant", required = true) @PathParam("tenant") String tenant, @ApiParam(value = "Specify the namespace", required = true) @@ -381,8 +382,14 @@ public void unloadTopic( @PathParam("topic") @Encoded String encodedTopic, @ApiParam(value = "Is authentication required to perform this operation") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { - validateTopicName(tenant, namespace, encodedTopic); - internalUnloadTopic(authoritative); + try { + validateTopicName(tenant, namespace, encodedTopic); + internalUnloadTopic(asyncResponse, authoritative); + } catch (WebApplicationException wae) { + asyncResponse.resume(wae); + } catch (Exception e) { + asyncResponse.resume(new RestException(e)); + } } @DELETE diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index a4cd3258198ac..1825d313124e7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -276,18 +276,46 @@ public void testUpdatePartitionedTopicHavingNonPartitionTopicWithPartitionSuffix @Test public void testUnloadTopic() { final String topicName = "standard-topic-to-be-unload"; + final String partitionTopicName = "partition-topic-to-be-unload"; + + // 1) not exist topic + AsyncResponse response = mock(AsyncResponse.class); + persistentTopics.unloadTopic(response, testTenant, testNamespace, "topic-not-exist", true); + ArgumentCaptor errCaptor = ArgumentCaptor.forClass(RestException.class); + verify(response, timeout(5000).times(1)).resume(errCaptor.capture()); + Assert.assertEquals(errCaptor.getValue().getResponse().getStatus(), Response.Status.NOT_FOUND.getStatusCode()); + + // 2) create non partitioned topic and unload + response = mock(AsyncResponse.class); persistentTopics.createNonPartitionedTopic(testTenant, testNamespace, topicName, true); - persistentTopics.unloadTopic(testTenant, testNamespace, topicName, true); + persistentTopics.unloadTopic(response, testTenant, testNamespace, topicName, true); + ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); + + // 3) create partitioned topic and unload + response = mock(AsyncResponse.class); + persistentTopics.createPartitionedTopic(testTenant, testNamespace, partitionTopicName, 6); + persistentTopics.unloadTopic(response, testTenant, testNamespace, partitionTopicName, true); + responseCaptor = ArgumentCaptor.forClass(Response.class); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); + + // 4) delete partitioned topic + response = mock(AsyncResponse.class); + persistentTopics.deletePartitionedTopic(response, testTenant, testNamespace, partitionTopicName, true, true); + responseCaptor = ArgumentCaptor.forClass(Response.class); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); } - @Test(expectedExceptions = RestException.class) + @Test public void testUnloadTopicShallThrowNotFoundWhenTopicNotExist() { - try { - persistentTopics.unloadTopic(testTenant, testNamespace,"non-existent-topic", true); - } catch (RestException e) { - Assert.assertEquals(e.getResponse().getStatus(), Response.Status.NOT_FOUND.getStatusCode()); - throw e; - } + AsyncResponse response = mock(AsyncResponse.class); + persistentTopics.unloadTopic(response, testTenant, testNamespace,"non-existent-topic", true); + ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(RestException.class); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getResponse().getStatus(), Response.Status.NOT_FOUND.getStatusCode()); } @Test From bf7302a743ed847b7c8906d448aefb3164bb205e Mon Sep 17 00:00:00 2001 From: Devin Bost Date: Sat, 15 Feb 2020 22:29:53 -0700 Subject: [PATCH 02/74] [Issue 4175] [pulsar-function-go] Create integration tests for Go Functions for production-readiness (#6104) This PR is to provide integration tests that test execution of Go functions that are managed by the Java FunctionManager. This will allow us to test things like behavior during function timeouts, heartbeat failures, and other situations that can only be effectively tested in an integration test. Master issue: #4175 Fixes issue: #6204 ### Modifications We must add Go to the integration testing logic. We must also build the Go dependencies into the test Dockerfile to ensure the Go binaries are available at runtime for the integration tests. --- .github/workflows/ci-go-functions-style.yaml | 2 +- .../examples/{test => consumer}/consumer.go | 0 .../examples/{ => contextFunc}/contextFunc.go | 0 .../exclamationFunc/exclamationFunc.go | 34 ++++++ pulsar-function-go/examples/go.mod | 18 +++ pulsar-function-go/examples/go.sum | 85 ++++++++++++++ .../examples/{ => hello}/hello.go | 0 .../examples/{ => inputFunc}/inputFunc.go | 0 .../examples/{ => outputFunc}/outputFunc.go | 0 .../examples/{test => producer}/producer.go | 0 pulsar-function-go/go.mod | 15 ++- pulsar-function-go/go.sum | 32 +++-- pulsar-function-go/pf/instance.go | 4 +- .../instance/go/GoInstanceConfig.java | 1 + .../latest-version-image/Dockerfile | 27 ++++- .../latest-version-image/pom.xml | 22 ++++ .../functions/PulsarFunctionsTest.java | 110 ++++++++++++------ .../functions/PulsarFunctionsTestBase.java | 6 +- .../functions/utils/CommandGenerator.java | 89 +++++++++----- 19 files changed, 354 insertions(+), 91 deletions(-) rename pulsar-function-go/examples/{test => consumer}/consumer.go (100%) rename pulsar-function-go/examples/{ => contextFunc}/contextFunc.go (100%) create mode 100644 pulsar-function-go/examples/exclamationFunc/exclamationFunc.go create mode 100644 pulsar-function-go/examples/go.mod create mode 100644 pulsar-function-go/examples/go.sum rename pulsar-function-go/examples/{ => hello}/hello.go (100%) rename pulsar-function-go/examples/{ => inputFunc}/inputFunc.go (100%) rename pulsar-function-go/examples/{ => outputFunc}/outputFunc.go (100%) rename pulsar-function-go/examples/{test => producer}/producer.go (100%) diff --git a/.github/workflows/ci-go-functions-style.yaml b/.github/workflows/ci-go-functions-style.yaml index 93269c47d3125..7acb091fc0aa1 100644 --- a/.github/workflows/ci-go-functions-style.yaml +++ b/.github/workflows/ci-go-functions-style.yaml @@ -58,7 +58,7 @@ jobs: if: steps.docs.outputs.changed_only == 'no' run: | cd pulsar-function-go - go build ./pf + go build ./... - name: CheckStyle if: steps.docs.outputs.changed_only == 'no' diff --git a/pulsar-function-go/examples/test/consumer.go b/pulsar-function-go/examples/consumer/consumer.go similarity index 100% rename from pulsar-function-go/examples/test/consumer.go rename to pulsar-function-go/examples/consumer/consumer.go diff --git a/pulsar-function-go/examples/contextFunc.go b/pulsar-function-go/examples/contextFunc/contextFunc.go similarity index 100% rename from pulsar-function-go/examples/contextFunc.go rename to pulsar-function-go/examples/contextFunc/contextFunc.go diff --git a/pulsar-function-go/examples/exclamationFunc/exclamationFunc.go b/pulsar-function-go/examples/exclamationFunc/exclamationFunc.go new file mode 100644 index 0000000000000..d938f7dbb8c18 --- /dev/null +++ b/pulsar-function-go/examples/exclamationFunc/exclamationFunc.go @@ -0,0 +1,34 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +// + +package main + +import ( + "context" + + "github.com/apache/pulsar/pulsar-function-go/pf" +) + +func HandleExclamation(ctx context.Context, in []byte) ([]byte, error) { + return []byte(string(in) + "!"), nil +} + +func main() { + pf.Start(HandleExclamation) +} diff --git a/pulsar-function-go/examples/go.mod b/pulsar-function-go/examples/go.mod new file mode 100644 index 0000000000000..2b47e06f775f0 --- /dev/null +++ b/pulsar-function-go/examples/go.mod @@ -0,0 +1,18 @@ +module github.com/apache/pulsar/pulsar-function-go/tests + +go 1.13 + +require ( + github.com/apache/pulsar-client-go v0.0.0-20200116214305-4d788d9935ed + github.com/apache/pulsar/pulsar-function-go v0.0.0 +) + +replace github.com/apache/pulsar/pulsar-function-go => ../ + +replace github.com/apache/pulsar/pulsar-function-go/pf => ../pf + +replace github.com/apache/pulsar/pulsar-function-go/logutil => ../logutil + +replace github.com/apache/pulsar/pulsar-function-go/pb => ../pb + +replace github.com/apache/pulsar/pulsar-function-go/conf => ../conf diff --git a/pulsar-function-go/examples/go.sum b/pulsar-function-go/examples/go.sum new file mode 100644 index 0000000000000..1023ffc88e0bc --- /dev/null +++ b/pulsar-function-go/examples/go.sum @@ -0,0 +1,85 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/apache/pulsar-client-go v0.0.0-20200113085434-9b739cf9d098/go.mod h1:G+CQVHnh2EPfNEQXOuisIDAyPMiKnzz4Vim/kjtj4U4= +github.com/apache/pulsar-client-go v0.0.0-20200116214305-4d788d9935ed h1:Lp7eU5ym84jPmIXoonoaJWVN6psyB90Olookp61LCeA= +github.com/apache/pulsar-client-go v0.0.0-20200116214305-4d788d9935ed/go.mod h1:G+CQVHnh2EPfNEQXOuisIDAyPMiKnzz4Vim/kjtj4U4= +github.com/apache/pulsar/pulsar-function-go v0.0.0-20200124033432-ec122ed9562c/go.mod h1:2a3PacwSg4KPcGxO3bjH29xsoKSuSkq2mG0sjKtxsP4= +github.com/beefsack/go-rate v0.0.0-20180408011153-efa7637bb9b6/go.mod h1:6YNgTHLutezwnBvyneBbwvB8C82y3dcoOj5EQJIdGXA= +github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b/go.mod h1:ac9efd0D1fsDb3EJvhqgXRbFx7bs2wqZ10HQPeU8U/Q= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2 h1:6nsPYzhq5kReh6QImI3k5qWzO4PEbvbIW2cwSfR/6xs= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= +github.com/klauspost/compress v1.9.2 h1:LfVyl+ZlLlLDeQ/d2AqfGIIH4qEDu0Ed2S5GyhCWIWY= +github.com/klauspost/compress v1.9.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/pierrec/lz4 v2.0.5+incompatible h1:2xWsjqPFWcplujydGg4WmhC/6fZqK42wMM8aXeqhl0I= +github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= +github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= +github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/sirupsen/logrus v1.4.1/go.mod h1:ni0Sbl8bgC9z8RoU9G6nDWqqs/fq4eDPysMBDgk/93Q= +github.com/sirupsen/logrus v1.4.2 h1:SPIRibHv4MatM3XXNO2BJeFLZwZ2LvZgfQ5+UNI2im4= +github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= +github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= +github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= +github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= +github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a h1:oWX7TPOiFAMXLq8o0ikBYfCJVlRHBcsciT5bXOrH628= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190422165155-953cdadca894 h1:Cz4ceDQGXuKRnVBDTS23GTn/pU5OE2C0WrNTOYK1Uuc= +golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55 h1:gSJIx1SDwno+2ElGhA4+qG2zF97qiUzTM+rQ0klBOcE= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.26.0 h1:2dTRdpdFEEhJYQD8EMLB61nnrzSCTbG38PhqdhvOltg= +google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.7/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.8 h1:obN1ZagJSUGI0Ek/LBmuj4SNLPfIny3KsKFopxRdj10= +gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/pulsar-function-go/examples/hello.go b/pulsar-function-go/examples/hello/hello.go similarity index 100% rename from pulsar-function-go/examples/hello.go rename to pulsar-function-go/examples/hello/hello.go diff --git a/pulsar-function-go/examples/inputFunc.go b/pulsar-function-go/examples/inputFunc/inputFunc.go similarity index 100% rename from pulsar-function-go/examples/inputFunc.go rename to pulsar-function-go/examples/inputFunc/inputFunc.go diff --git a/pulsar-function-go/examples/outputFunc.go b/pulsar-function-go/examples/outputFunc/outputFunc.go similarity index 100% rename from pulsar-function-go/examples/outputFunc.go rename to pulsar-function-go/examples/outputFunc/outputFunc.go diff --git a/pulsar-function-go/examples/test/producer.go b/pulsar-function-go/examples/producer/producer.go similarity index 100% rename from pulsar-function-go/examples/test/producer.go rename to pulsar-function-go/examples/producer/producer.go diff --git a/pulsar-function-go/go.mod b/pulsar-function-go/go.mod index e5af139f974d8..f1c8cb91526ec 100644 --- a/pulsar-function-go/go.mod +++ b/pulsar-function-go/go.mod @@ -5,9 +5,16 @@ go 1.13 require ( github.com/apache/pulsar-client-go v0.0.0-20200116214305-4d788d9935ed github.com/golang/protobuf v1.3.2 - github.com/sirupsen/logrus v1.4.1 - github.com/stretchr/testify v1.3.0 - golang.org/x/tools v0.0.0-20200119215504-eb0d8dd85bcc // indirect + github.com/sirupsen/logrus v1.4.2 + github.com/stretchr/testify v1.4.0 google.golang.org/grpc v1.26.0 - gopkg.in/yaml.v2 v2.2.2 + gopkg.in/yaml.v2 v2.2.8 ) + +replace github.com/apache/pulsar/pulsar-function-go/pf => ./pf + +replace github.com/apache/pulsar/pulsar-function-go/logutil => ./logutil + +replace github.com/apache/pulsar/pulsar-function-go/pb => ./pb + +replace github.com/apache/pulsar/pulsar-function-go/conf => ./conf diff --git a/pulsar-function-go/go.sum b/pulsar-function-go/go.sum index a52acbedc8c53..aac8d3ab9d945 100644 --- a/pulsar-function-go/go.sum +++ b/pulsar-function-go/go.sum @@ -1,8 +1,11 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/apache/pulsar v2.5.0+incompatible h1:g7BI4EcfHFq+OwyXXPeEnTo2hL2Koq8Y/ckBkccDgmA= +github.com/apache/pulsar-client-go v0.0.0-20200113085434-9b739cf9d098/go.mod h1:G+CQVHnh2EPfNEQXOuisIDAyPMiKnzz4Vim/kjtj4U4= github.com/apache/pulsar-client-go v0.0.0-20200116214305-4d788d9935ed h1:Lp7eU5ym84jPmIXoonoaJWVN6psyB90Olookp61LCeA= github.com/apache/pulsar-client-go v0.0.0-20200116214305-4d788d9935ed/go.mod h1:G+CQVHnh2EPfNEQXOuisIDAyPMiKnzz4Vim/kjtj4U4= +github.com/apache/pulsar/pulsar-function-go v0.0.0-20200124033432-ec122ed9562c h1:uqA9RBsmQz3gN045GQ6we1RfRsk+5dco60yJ695Yb0E= +github.com/apache/pulsar/pulsar-function-go v0.0.0-20200124033432-ec122ed9562c/go.mod h1:2a3PacwSg4KPcGxO3bjH29xsoKSuSkq2mG0sjKtxsP4= github.com/beefsack/go-rate v0.0.0-20180408011153-efa7637bb9b6/go.mod h1:6YNgTHLutezwnBvyneBbwvB8C82y3dcoOj5EQJIdGXA= github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b/go.mod h1:ac9efd0D1fsDb3EJvhqgXRbFx7bs2wqZ10HQPeU8U/Q= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= @@ -14,13 +17,11 @@ github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.m github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/mock v1.1.1 h1:G5FRp8JnTd7RQH5kemVNlMeyXQAztQ3mOWV95KxsXH8= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2 h1:6nsPYzhq5kReh6QImI3k5qWzO4PEbvbIW2cwSfR/6xs= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/google/go-cmp v0.2.0 h1:+dTQ8DZQJz0Mb/HjFlkptS1FeQ4cWSnN941F8aEG4SQ= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0 h1:crn/baboCvb5fXaQ0IJ1SGTsTVrWpDsCWC8EGETZijY= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= @@ -36,8 +37,9 @@ github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/sirupsen/logrus v1.4.1 h1:GL2rEmy6nsikmW0r8opw9JIRScdMF5hA8cOYLH7In1k= github.com/sirupsen/logrus v1.4.1/go.mod h1:ni0Sbl8bgC9z8RoU9G6nDWqqs/fq4eDPysMBDgk/93Q= +github.com/sirupsen/logrus v1.4.2 h1:SPIRibHv4MatM3XXNO2BJeFLZwZ2LvZgfQ5+UNI2im4= +github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= @@ -45,44 +47,34 @@ github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnIn github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= -github.com/stretchr/testify v1.3.0 h1:TivCn/peBQ7UY8ooIcPgZFpTNSz0Q2U6UrFlUfqbe0Q= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2 h1:VklqNMn3ovrHsnt90PveolxSbWFaJdECFbxSq0Mqo2M= +github.com/stretchr/testify v1.4.0 h1:2E4SXV/wtOkTonXsotYi4li6zVWxYlZuYNCXe9XRJyk= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= -golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a h1:oWX7TPOiFAMXLq8o0ikBYfCJVlRHBcsciT5bXOrH628= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f h1:wMNYb4v58l5UBM7MYRLPG6ZhfOqbKu7X5eyFl8ZhKvA= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33 h1:I6FyU15t786LL7oL/hn43zqTuEGr4PN7F4XJ1p4E3Y8= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a h1:1BGLXjeY4akVXGgbC9HugT3Jv3hCI0z56oJR5vAMgBU= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190422165155-953cdadca894 h1:Cz4ceDQGXuKRnVBDTS23GTn/pU5OE2C0WrNTOYK1Uuc= +golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135 h1:5Beo0mZN8dRzgrMMkDp0jc8YXQKx9DiJ2k1dkvGsn5A= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20200119215504-eb0d8dd85bcc h1:ZA7KFRdqWZkBr0/YbHm1h08vDJ5gQdjVG/8L153z5c4= -golang.org/x/tools v0.0.0-20200119215504-eb0d8dd85bcc/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= @@ -94,7 +86,9 @@ google.golang.org/grpc v1.26.0 h1:2dTRdpdFEEhJYQD8EMLB61nnrzSCTbG38PhqdhvOltg= google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.7/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.8 h1:obN1ZagJSUGI0Ek/LBmuj4SNLPfIny3KsKFopxRdj10= +gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/pulsar-function-go/pf/instance.go b/pulsar-function-go/pf/instance.go index 689b176e2c777..e039490cb0e6a 100644 --- a/pulsar-function-go/pf/instance.go +++ b/pulsar-function-go/pf/instance.go @@ -38,6 +38,7 @@ type goInstance struct { consumers map[string]pulsar.Consumer client pulsar.Client lastHealthCheckTs int64 + properties map[string]string } // newGoInstance init goInstance and init function context @@ -48,6 +49,7 @@ func newGoInstance() *goInstance { } now := time.Now() goInstance.lastHealthCheckTs = now.UnixNano() + goInstance.properties = make(map[string]string) return goInstance } @@ -176,7 +178,7 @@ func (gi *goInstance) setupProducer() (err error) { Properties: properties, CompressionType: pulsar.LZ4, BatchingMaxPublishDelay: time.Millisecond * 10, - // set send timeout to be infinity to prevent potential deadlock with consumer + // Set send timeout to be infinity to prevent potential deadlock with consumer // that might happen when consumer is blocked due to unacked messages }) if err != nil { diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/go/GoInstanceConfig.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/go/GoInstanceConfig.java index 5c95208225f95..2d2d7ad66b73e 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/go/GoInstanceConfig.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/go/GoInstanceConfig.java @@ -32,6 +32,7 @@ public class GoInstanceConfig { private int port; private String clusterName = ""; private int killAfterIdleMs; + private int expectedHealthCheckInterval; private String tenant = ""; private String nameSpace = ""; diff --git a/tests/docker-images/latest-version-image/Dockerfile b/tests/docker-images/latest-version-image/Dockerfile index 35b757d7789a1..eaa279a42cd11 100644 --- a/tests/docker-images/latest-version-image/Dockerfile +++ b/tests/docker-images/latest-version-image/Dockerfile @@ -19,7 +19,9 @@ FROM apachepulsar/pulsar-all:latest -RUN apt-get update && apt-get install -y supervisor vim procps +RUN rm -rf /var/lib/apt/lists/* && apt update + +RUN apt-get clean && apt-get update && apt-get install -y supervisor vim procps curl git RUN mkdir -p /var/log/pulsar && mkdir -p /var/run/supervisor/ && mkdir -p /pulsar/ssl @@ -43,10 +45,33 @@ COPY scripts/init-cluster.sh scripts/run-global-zk.sh scripts/run-local-zk.sh \ RUN mkdir -p /pulsar/instances/deps +ENV GOLANG_VERSION 1.13.3 + +RUN curl -sSL https://storage.googleapis.com/golang/go$GOLANG_VERSION.linux-amd64.tar.gz \ + | tar -v -C /usr/local -xz + +# RUN wget https://dl.google.com/go/go1.13.3.linux-amd64.tar.gz && tar -xvf go1.13.3.linux-amd64.tar.gz && mv go /usr/local +# RUN export GOROOT=/usr/local/go && export GOPATH=$HOME/go && export PATH=$GOPATH/bin:$GOROOT/bin:$PATH +# RUN echo "export GOROOT=/usr/local/go" >> ~/.profile && echo "export GOPATH=$HOME/go" >> ~/.profile && echo "export PATH=$GOPATH/bin:$GOROOT/bin:$PATH" >> ~/.profile + +ENV PATH /usr/local/go/bin:$PATH + +RUN mkdir -p /go/src /go/bin && chmod -R 777 /go +ENV GOROOT /usr/local/go +ENV GOPATH /go +ENV PATH /go/bin:$PATH + COPY python-examples/exclamation_lib.py /pulsar/instances/deps/ COPY python-examples/exclamation_with_extra_deps.py /pulsar/examples/python-examples/ COPY python-examples/exclamation.zip /pulsar/examples/python-examples/ COPY python-examples/producer_schema.py /pulsar/examples/python-examples/ COPY python-examples/consumer_schema.py /pulsar/examples/python-examples/ COPY python-examples/exception_function.py /pulsar/examples/python-examples/ + +COPY target/pulsar-function-go/ /go/src/github.com/apache/pulsar/pulsar-function-go +RUN cd /go/src/github.com/apache/pulsar/pulsar-function-go && go install ./... +RUN cd /go/src/github.com/apache/pulsar/pulsar-function-go/pf && go install +RUN cd /go/src/github.com/apache/pulsar/pulsar-function-go/examples && go install ./... + COPY target/java-test-functions.jar /pulsar/examples/ +RUN cp -a /go/bin/. /pulsar/examples/go-examples/ diff --git a/tests/docker-images/latest-version-image/pom.xml b/tests/docker-images/latest-version-image/pom.xml index b0c3f76a94306..5a961b016f2f7 100644 --- a/tests/docker-images/latest-version-image/pom.xml +++ b/tests/docker-images/latest-version-image/pom.xml @@ -79,6 +79,28 @@ + + maven-resources-plugin + + + copy-pulsar-function-go + generate-resources + + copy-resources + + + ${project.build.directory}/pulsar-function-go/ + true + + + ../../../pulsar-function-go/ + false + + + + + + com.spotify dockerfile-maven-plugin diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java index e112474f73593..e18a2ee320efe 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java @@ -61,6 +61,8 @@ import org.testng.annotations.Test; import org.testng.collections.Maps; +import java.io.BufferedReader; +import java.io.InputStreamReader; import java.time.Duration; import java.util.Collections; import java.util.HashSet; @@ -842,6 +844,10 @@ protected void getSourceInfoNotFound(String tenant, String namespace, String sou public void testPythonFunctionLocalRun() throws Exception { testFunctionLocalRun(Runtime.PYTHON); } + @Test(groups = "function") + public void testGoFunctionLocalRun() throws Exception { + testFunctionLocalRun(Runtime.GO); + } @Test(groups = "function") public void testJavaFunctionLocalRun() throws Exception { @@ -862,19 +868,26 @@ public void testFunctionLocalRun(Runtime runtime) throws Exception { String outputTopicName = "test-function-local-run-" + runtime + "-output-" + randomName(8); final int numMessages = 10; - String cmd; + String cmd = ""; CommandGenerator commandGenerator = new CommandGenerator(); commandGenerator.setAdminUrl("pulsar://pulsar-broker-0:6650"); commandGenerator.setSourceTopic(inputTopicName); commandGenerator.setSinkTopic(outputTopicName); commandGenerator.setFunctionName("localRunTest"); commandGenerator.setRuntime(runtime); - if (runtime == Runtime.JAVA) { - commandGenerator.setFunctionClassName(EXCLAMATION_JAVA_CLASS); - cmd = commandGenerator.generateLocalRunCommand(null); - } else { - commandGenerator.setFunctionClassName(EXCLAMATION_PYTHON_CLASS); - cmd = commandGenerator.generateLocalRunCommand(EXCLAMATION_PYTHON_FILE); + switch (runtime) { + case JAVA: + commandGenerator.setFunctionClassName(EXCLAMATION_JAVA_CLASS); + cmd = commandGenerator.generateLocalRunCommand(null); + break; + case PYTHON: + commandGenerator.setFunctionClassName(EXCLAMATION_PYTHON_CLASS); + cmd = commandGenerator.generateLocalRunCommand(EXCLAMATION_PYTHON_FILE); + break; + case GO: + commandGenerator.setFunctionClassName(null); + cmd = commandGenerator.generateLocalRunCommand(EXCLAMATION_GO_FILE); + break; } log.info("cmd: {}", cmd); @@ -1316,6 +1329,11 @@ public void testJavaPublishFunction() throws Exception { testPublishFunction(Runtime.JAVA); } + @Test + public void testGoPublishFunction() throws Exception { + testPublishFunction(Runtime.GO); + } + private void testPublishFunction(Runtime runtime) throws Exception { if (functionRuntimeType == FunctionRuntimeType.THREAD) { return; @@ -1344,27 +1362,39 @@ private void testPublishFunction(Runtime runtime) throws Exception { final int numMessages = 10; // submit the exclamation function - - if (runtime == Runtime.PYTHON) { - submitFunction( - runtime, - inputTopicName, - outputTopicName, - functionName, - PUBLISH_FUNCTION_PYTHON_FILE, - PUBLISH_PYTHON_CLASS, - schema, - Collections.singletonMap("publish-topic", outputTopicName)); - } else { - submitFunction( - runtime, - inputTopicName, - outputTopicName, - functionName, - null, - PUBLISH_JAVA_CLASS, - schema, - Collections.singletonMap("publish-topic", outputTopicName)); + switch (runtime){ + case JAVA: + submitFunction( + runtime, + inputTopicName, + outputTopicName, + functionName, + null, + PUBLISH_JAVA_CLASS, + schema, + Collections.singletonMap("publish-topic", outputTopicName)); + break; + case PYTHON: + submitFunction( + runtime, + inputTopicName, + outputTopicName, + functionName, + PUBLISH_FUNCTION_PYTHON_FILE, + PUBLISH_PYTHON_CLASS, + schema, + Collections.singletonMap("publish-topic", outputTopicName)); + break; + case GO: + submitFunction( + runtime, + inputTopicName, + outputTopicName, + functionName, + PUBLISH_FUNCTION_GO_FILE, + null, + schema, + Collections.singletonMap("publish-topic", outputTopicName)); } // get function info @@ -1374,11 +1404,12 @@ private void testPublishFunction(Runtime runtime) throws Exception { getFunctionStatsEmpty(functionName); // publish and consume result + if (Runtime.JAVA == runtime) { // java supports schema publishAndConsumeMessages(inputTopicName, outputTopicName, numMessages); } else { - // python doesn't support schema + // python doesn't support schema. Does Go? Maybe we need a switch instead for the Go case. @Cleanup PulsarClient client = PulsarClient.builder() .serviceUrl(pulsarCluster.getPlainTextServiceUrl()) @@ -1673,14 +1704,19 @@ private static void submitFunction(Runtime runtime, if (userConfigs != null) { generator.setUserConfig(userConfigs); } - String command; - if (Runtime.JAVA == runtime) { - command = generator.generateCreateFunctionCommand(); - } else if (Runtime.PYTHON == runtime) { - generator.setRuntime(runtime); - command = generator.generateCreateFunctionCommand(functionFile); - } else { - throw new IllegalArgumentException("Unsupported runtime : " + runtime); + String command = ""; + + switch (runtime){ + case JAVA: + command = generator.generateCreateFunctionCommand(); + break; + case PYTHON: + case GO: + generator.setRuntime(runtime); + command = generator.generateCreateFunctionCommand(functionFile); + break; + default: + throw new IllegalArgumentException("Unsupported runtime : " + runtime); } log.info("---------- Function command: {}", command); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTestBase.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTestBase.java index cb5291bed7bc3..94bffb3316c64 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTestBase.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTestBase.java @@ -103,6 +103,9 @@ public void teardownFunctionWorkers() { public static final String PUBLISH_FUNCTION_PYTHON_FILE = "typed_message_builder_publish.py"; public static final String EXCEPTION_FUNCTION_PYTHON_FILE = "exception_function.py"; + public static final String EXCLAMATION_GO_FILE = "exclamationFunc"; + public static final String PUBLISH_FUNCTION_GO_FILE = "exclamationFunc"; + protected static String getExclamationClass(Runtime runtime, boolean pyZip, boolean extraDeps) { @@ -125,7 +128,8 @@ protected static String getExclamationClass(Runtime runtime, public static Object[][] functionRuntimes() { return new Object[][] { new Object[] { Runtime.JAVA }, - new Object[] { Runtime.PYTHON } + new Object[] { Runtime.PYTHON }, + new Object[] { Runtime.GO } }; } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/utils/CommandGenerator.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/utils/CommandGenerator.java index f18e0c8720c98..40458d59aecf2 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/utils/CommandGenerator.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/utils/CommandGenerator.java @@ -33,6 +33,7 @@ public class CommandGenerator { public enum Runtime { JAVA, PYTHON, + GO, }; private String functionName; private String tenant = "public"; @@ -56,6 +57,7 @@ public enum Runtime { private Map userConfig = new HashMap<>(); public static final String JAVAJAR = "/pulsar/examples/java-test-functions.jar"; public static final String PYTHONBASE = "/pulsar/examples/python-examples/"; + public static final String GOBASE = "/pulsar/examples/go-examples/"; public static CommandGenerator createDefaultGenerator(String sourceTopic, String functionClassName) { CommandGenerator generator = new CommandGenerator(); @@ -88,22 +90,33 @@ public String generateLocalRunCommand(String codeFile) { if (functionName != null) { commandBuilder.append(" --name " + functionName); } - commandBuilder.append(" --className " + functionClassName); + if(runtime != Runtime.GO){ + commandBuilder.append(" --className " + functionClassName); + } if (sourceTopic != null) { commandBuilder.append(" --inputs " + sourceTopic); } if (sinkTopic != null) { commandBuilder.append(" --output " + sinkTopic); } - - if (runtime == Runtime.JAVA) { - commandBuilder.append(" --jar " + JAVAJAR); - } else { - if (codeFile != null) { - commandBuilder.append(" --py " + PYTHONBASE + codeFile); - } else { - commandBuilder.append(" --py " + PYTHONBASE); - } + switch (runtime){ + case JAVA: + commandBuilder.append(" --jar " + JAVAJAR); + break; + case PYTHON: + if (codeFile != null) { + commandBuilder.append(" --py " + PYTHONBASE + codeFile); + } else { + commandBuilder.append(" --py " + PYTHONBASE); + } + break; + case GO: + if (codeFile != null) { + commandBuilder.append(" --go " + GOBASE + codeFile); + } else { + commandBuilder.append(" --go " + GOBASE); + } + break; } return commandBuilder.toString(); @@ -129,7 +142,9 @@ public String generateCreateFunctionCommand(String codeFile) { if (functionName != null) { commandBuilder.append(" --name " + functionName); } - commandBuilder.append(" --className " + functionClassName); + if (runtime != Runtime.GO){ + commandBuilder.append(" --className " + functionClassName); + } if (sourceTopic != null) { commandBuilder.append(" --inputs " + sourceTopic); } @@ -170,14 +185,24 @@ public String generateCreateFunctionCommand(String codeFile) { commandBuilder.append(" --slidingIntervalDurationMs " + slidingIntervalDurationMs); } - if (runtime == Runtime.JAVA) { - commandBuilder.append(" --jar " + JAVAJAR); - } else { - if (codeFile != null) { - commandBuilder.append(" --py " + PYTHONBASE + codeFile); - } else { - commandBuilder.append(" --py " + PYTHONBASE); - } + switch (runtime){ + case JAVA: + commandBuilder.append(" --jar " + JAVAJAR); + break; + case PYTHON: + if (codeFile != null) { + commandBuilder.append(" --py " + PYTHONBASE + codeFile); + } else { + commandBuilder.append(" --py " + PYTHONBASE); + } + break; + case GO: + if (codeFile != null) { + commandBuilder.append(" --go " + GOBASE + codeFile); + } else { + commandBuilder.append(" --go " + GOBASE); + } + break; } return commandBuilder.toString(); } @@ -243,14 +268,24 @@ public String generateUpdateFunctionCommand(String codeFile) { commandBuilder.append(" --slidingIntervalDurationMs " + slidingIntervalDurationMs); } - if (runtime == Runtime.JAVA) { - commandBuilder.append(" --jar " + JAVAJAR); - } else { - if (codeFile != null) { - commandBuilder.append(" --py " + PYTHONBASE + codeFile); - } else { - commandBuilder.append(" --py " + PYTHONBASE); - } + switch (runtime){ + case JAVA: + commandBuilder.append(" --jar " + JAVAJAR); + break; + case PYTHON: + if (codeFile != null) { + commandBuilder.append(" --py " + PYTHONBASE + codeFile); + } else { + commandBuilder.append(" --py " + PYTHONBASE); + } + break; + case GO: + if (codeFile != null) { + commandBuilder.append(" --go " + GOBASE + codeFile); + } else { + commandBuilder.append(" --go " + GOBASE); + } + break; } return commandBuilder.toString(); } From e9083f5592035b4812e086040d4a758379fdd349 Mon Sep 17 00:00:00 2001 From: Fangbin Sun Date: Sun, 16 Feb 2020 13:37:07 +0800 Subject: [PATCH 03/74] [Issue 5999] Support create/update tenant with empty cluster (#6027) ### Motivation Fixes #5999 ### Modifications Add the logic to handle the blank cluster name. --- .../pulsar/broker/admin/impl/TenantsBase.java | 19 ++++++---- .../apache/pulsar/broker/admin/AdminTest.java | 37 ++++++++++++++++--- .../auth/MockedPulsarServiceBaseTest.java | 16 ++++++++ .../broker/service/ResendRequestTest.java | 9 +++-- .../pulsar/broker/web/WebServiceTest.java | 8 ++++ .../impl/PatternTopicsConsumerImplTest.java | 21 +++++++---- ...erMessageUnAcknowledgedRedeliveryTest.java | 3 +- .../client/impl/TopicsConsumerImplTest.java | 30 ++++++++++----- .../client/cli/PulsarClientToolTest.java | 4 +- .../pulsar/proxy/server/ProxyParserTest.java | 3 +- .../apache/pulsar/proxy/server/ProxyTest.java | 3 +- .../pulsar/proxy/server/ProxyTlsTest.java | 3 +- 12 files changed, 118 insertions(+), 38 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java index a6e1d35ee85bc..b7b64ddd91a08 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TenantsBase.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.admin.impl; -import io.swagger.annotations.ApiParam; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -31,6 +30,7 @@ import javax.ws.rs.PathParam; import javax.ws.rs.core.Response.Status; +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.web.RestException; import org.apache.pulsar.common.naming.Constants; @@ -42,7 +42,9 @@ import org.slf4j.LoggerFactory; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiParam; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; @@ -91,6 +93,7 @@ public TenantInfo getTenantAdmin( @ApiResponses(value = { @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 409, message = "Tenant already exists"), @ApiResponse(code = 412, message = "Tenant name is not valid"), + @ApiResponse(code = 412, message = "Clusters can not be empty"), @ApiResponse(code = 412, message = "Clusters do not exist") }) public void createTenant( @ApiParam(value = "The tenant name") @@ -102,9 +105,6 @@ public void createTenant( try { NamedEntity.checkName(tenant); - if (config == null) { - config = new TenantInfo(); - } zkCreate(path(POLICIES, tenant), jsonMapper().writeValueAsBytes(config)); log.info("[{}] Created tenant {}", clientAppId(), tenant); } catch (KeeperException.NodeExistsException e) { @@ -125,6 +125,7 @@ public void createTenant( @ApiResponses(value = { @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 404, message = "Tenant does not exist"), @ApiResponse(code = 409, message = "Tenant already exists"), + @ApiResponse(code = 412, message = "Clusters can not be empty"), @ApiResponse(code = 412, message = "Clusters do not exist") }) public void updateTenant( @ApiParam(value = "The tenant name") @@ -225,11 +226,15 @@ public void deleteTenant( } private void validateClusters(TenantInfo info) { + // empty cluster shouldn't be allowed + if (info == null || info.getAllowedClusters().stream().filter(c -> !StringUtils.isBlank(c)).collect(Collectors.toSet()).isEmpty() + || info.getAllowedClusters().stream().anyMatch(ac -> StringUtils.isBlank(ac))) { + log.warn("[{}] Failed to validate due to clusters are empty", clientAppId()); + throw new RestException(Status.PRECONDITION_FAILED, "Clusters can not be empty"); + } + List nonexistentClusters; try { - if (info == null) { - info = new TenantInfo(); - } Set availableClusters = clustersListCache().get(); Set allowedClusters = info.getAllowedClusters(); nonexistentClusters = allowedClusters.stream() diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java index 06ec02ae000b7..00d1a31b1ef53 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java @@ -381,7 +381,11 @@ void properties() throws Exception { assertEquals(properties.getTenants(), Lists.newArrayList()); verify(properties, times(1)).validateSuperUserAccess(); + // create local cluster + clusters.createCluster(configClusterName, new ClusterData()); + Set allowedClusters = Sets.newHashSet(); + allowedClusters.add(configClusterName); TenantInfo tenantInfo = new TenantInfo(Sets.newHashSet("role1", "role2"), allowedClusters); properties.createTenant("test-property", tenantInfo); verify(properties, times(2)).validateSuperUserAccess(); @@ -489,7 +493,6 @@ void properties() throws Exception { assertEquals(properties.getTenants(), Lists.newArrayList()); // Create a namespace to test deleting a non-empty property - clusters.createCluster("use", new ClusterData()); newPropertyAdmin = new TenantInfo(Sets.newHashSet("role1", "other-role"), Sets.newHashSet("use")); properties.createTenant("my-tenant", newPropertyAdmin); @@ -511,9 +514,34 @@ void properties() throws Exception { } // Check tenantInfo is null - TenantInfo nullTenantInfo = new TenantInfo(); - properties.createTenant("tenant-config-is-null", null); - assertEquals(properties.getTenantAdmin("tenant-config-is-null"), nullTenantInfo); + try { + properties.createTenant("tenant-config-is-null", null); + fail("should have failed"); + } catch (RestException e) { + assertEquals(e.getResponse().getStatus(), Status.PRECONDITION_FAILED.getStatusCode()); + } + + // Check tenantInfo with empty cluster + String blankCluster = ""; + Set blankClusters = Sets.newHashSet(blankCluster); + TenantInfo tenantWithEmptyCluster = new TenantInfo(Sets.newHashSet("role1", "role2"), blankClusters); + try { + properties.createTenant("tenant-config-is-empty", tenantWithEmptyCluster); + fail("should have failed"); + } catch (RestException e) { + assertEquals(e.getResponse().getStatus(), Status.PRECONDITION_FAILED.getStatusCode()); + } + + // Check tenantInfo contains empty cluster + Set containBlankClusters = Sets.newHashSet(blankCluster); + containBlankClusters.add(configClusterName); + TenantInfo tenantContainEmptyCluster = new TenantInfo(Sets.newHashSet(), containBlankClusters); + try { + properties.createTenant("tenant-config-contain-empty", tenantContainEmptyCluster); + fail("should have failed"); + } catch (RestException e) { + assertEquals(e.getResponse().getStatus(), Status.PRECONDITION_FAILED.getStatusCode()); + } AsyncResponse response = mock(AsyncResponse.class); namespaces.deleteNamespace(response, "my-tenant", "use", "my-namespace", false); @@ -521,7 +549,6 @@ void properties() throws Exception { verify(response, timeout(5000).times(1)).resume(captor.capture()); assertEquals(captor.getValue().getStatus(), Status.NO_CONTENT.getStatusCode()); properties.deleteTenant("my-tenant"); - properties.deleteTenant("tenant-config-is-null"); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index a47e3ebc3dea4..7b3b04061e55d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java @@ -21,6 +21,7 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; +import com.google.common.collect.Sets; import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -31,6 +32,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -48,8 +50,11 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.compaction.Compactor; import org.apache.pulsar.zookeeper.ZooKeeperClientFactory; import org.apache.pulsar.zookeeper.ZookeeperClientFactoryImpl; @@ -231,6 +236,17 @@ protected void setupBrokerMocks(PulsarService pulsar) throws Exception { doReturn(sameThreadOrderedSafeExecutor).when(pulsar).getOrderedExecutor(); } + public TenantInfo createDefaultTenantInfo() throws PulsarAdminException { + // create local cluster if not exist + if (!admin.clusters().getClusters().contains(configClusterName)) { + admin.clusters().createCluster(configClusterName, new ClusterData()); + } + Set allowedClusters = Sets.newHashSet(); + allowedClusters.add(configClusterName); + TenantInfo tenantInfo = new TenantInfo(Sets.newHashSet(), allowedClusters); + return tenantInfo; + } + public static MockZooKeeper createMockZooKeeper() throws Exception { MockZooKeeper zk = MockZooKeeper.newInstance(MoreExecutors.newDirectExecutorService()); List dummyAclList = new ArrayList<>(0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ResendRequestTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ResendRequestTest.java index e6bdd71ad4de5..f9147250346b8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ResendRequestTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ResendRequestTest.java @@ -421,7 +421,8 @@ public void testExclusiveSingleAckedPartitionedTopic() throws Exception { final String messagePredicate = "my-message-" + key + "-"; final int totalMessages = 10; final int numberOfPartitions = 4; - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName, numberOfPartitions); // Special step to create partitioned topic @@ -476,7 +477,8 @@ public void testSharedSingleAckedPartitionedTopic() throws Exception { final String messagePredicate = "my-message-" + key + "-"; final int totalMessages = 10; final int numberOfPartitions = 3; - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName, numberOfPartitions); Random rn = new Random(); // Special step to create partitioned topic @@ -576,7 +578,8 @@ public void testFailoverSingleAckedPartitionedTopic() throws Exception { final String messagePredicate = "my-message-" + key + "-"; final int totalMessages = 10; final int numberOfPartitions = 3; - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName, numberOfPartitions); Random rn = new Random(); // Special step to create partitioned topic diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java index 591176dd544fa..fb427473ea3e2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceTest.java @@ -22,6 +22,7 @@ import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; +import com.google.common.collect.Sets; import com.google.common.io.CharStreams; import com.google.common.io.Closeables; @@ -70,6 +71,7 @@ import org.apache.pulsar.common.util.SecurityUtility; import org.apache.pulsar.zookeeper.MockedZooKeeperClientFactoryImpl; import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.ZooDefs; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -228,8 +230,14 @@ public void testMaxRequestSize() throws Exception { // This should have failed assertEquals(response.getStatusLine().getStatusCode(), 400); + // Create local cluster + String localCluster = "test"; + String clusterPath = PulsarWebResource.path("clusters", localCluster); + byte[] content = ObjectMapperFactory.getThreadLocal().writeValueAsBytes(new ClusterData()); + pulsar.getGlobalZkCache().getZooKeeper().create(clusterPath, content, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); TenantInfo info2 = new TenantInfo(); info2.setAdminRoles(Collections.singleton(StringUtils.repeat("*", 1 * 1024))); + info2.setAllowedClusters(Sets.newHashSet(localCluster)); httpPut.setEntity(new ByteArrayEntity(ObjectMapperFactory.getThreadLocal().writeValueAsBytes(info2))); response = client.execute(httpPut); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java index ff51d74cef7b3..b14c1cd466090 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java @@ -84,7 +84,8 @@ public void testPatternTopicsSubscribeWithBuilderFail() throws Exception { final String patternString = "persistent://my-property/my-ns/pattern-topic.*"; Pattern pattern = Pattern.compile(patternString); - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName2, 2); admin.topics().createPartitionedTopic(topicName3, 3); @@ -143,7 +144,8 @@ public void testBinaryProtoToGetTopicsOfNamespacePersistent() throws Exception { Pattern pattern = Pattern.compile("my-property/my-ns/pattern-topic.*"); // 1. create partition - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName2, 2); admin.topics().createPartitionedTopic(topicName3, 3); @@ -234,7 +236,8 @@ public void testBinaryProtoToGetTopicsOfNamespaceNonPersistent() throws Exceptio Pattern pattern = Pattern.compile("my-property/my-ns/np-pattern-topic.*"); // 1. create partition - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName2, 2); admin.topics().createPartitionedTopic(topicName3, 3); @@ -324,7 +327,8 @@ public void testBinaryProtoToGetTopicsOfNamespaceAll() throws Exception { Pattern pattern = Pattern.compile("my-property/my-ns/pattern-topic.*"); // 1. create partition - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName2, 2); admin.topics().createPartitionedTopic(topicName3, 3); @@ -473,7 +477,8 @@ public void testStartEmptyPatternConsumer() throws Exception { Pattern pattern = Pattern.compile("persistent://my-property/my-ns/pattern-topic.*"); // 1. create partition - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName2, 2); admin.topics().createPartitionedTopic(topicName3, 3); @@ -560,7 +565,8 @@ public void testAutoSubscribePatternConsumer() throws Exception { Pattern pattern = Pattern.compile("persistent://my-property/my-ns/pattern-topic.*"); // 1. create partition - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName2, 2); admin.topics().createPartitionedTopic(topicName3, 3); @@ -668,7 +674,8 @@ public void testAutoUnbubscribePatternConsumer() throws Exception { Pattern pattern = Pattern.compile("persistent://my-property/my-ns/pattern-topic.*"); // 1. create partition - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName2, 2); admin.topics().createPartitionedTopic(topicName3, 3); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PerMessageUnAcknowledgedRedeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PerMessageUnAcknowledgedRedeliveryTest.java index 508de46482db0..8b91af6f87fd7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PerMessageUnAcknowledgedRedeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PerMessageUnAcknowledgedRedeliveryTest.java @@ -403,7 +403,8 @@ public void testSharedAckedPartitionedTopic() throws Exception { final String messagePredicate = "my-message-" + key + "-"; final int totalMessages = 15; final int numberOfPartitions = 3; - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName, numberOfPartitions); // 1. producer connect diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java index 0d693663b76fc..c823e608dbe7d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java @@ -88,7 +88,8 @@ public void testDifferentTopicsNameSubscribe() throws Exception { final String topicName3 = "persistent://prop/use/ns-abc3/topic-3-" + key; List topicNames = Lists.newArrayList(topicName1, topicName2, topicName3); - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName2, 2); admin.topics().createPartitionedTopic(topicName3, 3); @@ -116,7 +117,8 @@ public void testGetConsumersAndGetTopics() throws Exception { final String topicName3 = "persistent://prop/use/ns-abc/topic-3-" + key; List topicNames = Lists.newArrayList(topicName1, topicName2); - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName2, 2); admin.topics().createPartitionedTopic(topicName3, 3); @@ -159,7 +161,8 @@ public void testSyncProducerAndConsumer() throws Exception { final String topicName3 = "persistent://prop/use/ns-abc/topic-3-" + key; List topicNames = Lists.newArrayList(topicName1, topicName2, topicName3); - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName2, 2); admin.topics().createPartitionedTopic(topicName3, 3); @@ -224,7 +227,8 @@ public void testAsyncConsumer() throws Exception { final String topicName3 = "persistent://prop/use/ns-abc/topic-3-" + key; List topicNames = Lists.newArrayList(topicName1, topicName2, topicName3); - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName2, 2); admin.topics().createPartitionedTopic(topicName3, 3); @@ -308,7 +312,8 @@ public void testConsumerUnackedRedelivery() throws Exception { final String topicName3 = "persistent://prop/use/ns-abc/topic-3-" + key; List topicNames = Lists.newArrayList(topicName1, topicName2, topicName3); - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName2, 2); admin.topics().createPartitionedTopic(topicName3, 3); @@ -451,7 +456,8 @@ public void testSubscribeUnsubscribeSingleTopic() throws Exception { final String topicName3 = "persistent://prop/use/ns-abc/topic-3-" + key; List topicNames = Lists.newArrayList(topicName1, topicName2, topicName3); - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName2, 2); admin.topics().createPartitionedTopic(topicName3, 3); @@ -575,7 +581,8 @@ public void testTopicsNameSubscribeWithBuilderFail() throws Exception { final String topicName2 = "persistent://prop/use/ns-abc/topic-2-" + key; final String topicName3 = "persistent://prop/use/ns-abc/topic-3-" + key; - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName2, 2); admin.topics().createPartitionedTopic(topicName3, 3); @@ -644,7 +651,8 @@ public void testMultiTopicsMessageListener() throws Exception { final String topicName1 = "persistent://prop/use/ns-abc/topic-1-" + key; List topicNames = Lists.newArrayList(topicName1); - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName1, 2); // 1. producer connect @@ -707,7 +715,8 @@ public void testTopicAutoUpdatePartitions() throws Exception { final String topicName2 = "persistent://prop/use/ns-abc/topic-2-" + key; List topicNames = Lists.newArrayList(topicName1, topicName2); - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName1, 2); admin.topics().createPartitionedTopic(topicName2, 2); @@ -811,7 +820,8 @@ public void testGetLastMessageId() throws Exception { final String topicName3 = "persistent://prop/use/ns-abc/topic-3-" + key; List topicNames = Lists.newArrayList(topicName1, topicName2, topicName3); - admin.tenants().createTenant("prop", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("prop", tenantInfo); admin.topics().createPartitionedTopic(topicName2, 2); admin.topics().createPartitionedTopic(topicName3, 3); diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java index bac300aebcf74..21b71eb042755 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java @@ -29,7 +29,6 @@ import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.cli.PulsarClientTool; import org.apache.pulsar.common.policies.data.TenantInfo; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -60,7 +59,8 @@ public void testInitialzation() throws MalformedURLException, InterruptedExcepti String tenantName = UUID.randomUUID().toString(); - admin.tenants().createTenant(tenantName, new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant(tenantName, tenantInfo); String topicName = String.format("persistent://%s/ns/topic-scale-ns-0/topic", tenantName); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java index 7ae6cc03a00c5..3557441d80612 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java @@ -141,7 +141,8 @@ public void testProducerConsumer() throws Exception { @Test public void testPartitions() throws Exception { - admin.tenants().createTenant("sample", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("sample", tenantInfo); PulsarClient client = PulsarClient.builder().serviceUrl(proxyService.getServiceUrl()) .build(); admin.topics().createPartitionedTopic("persistent://sample/test/local/partitioned-topic", 2); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java index eb5a32e32720e..7b308d198e401 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java @@ -161,7 +161,8 @@ public void testProducerConsumer() throws Exception { @Test public void testPartitions() throws Exception { - admin.tenants().createTenant("sample", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("sample", tenantInfo); @Cleanup PulsarClient client = PulsarClient.builder().serviceUrl(proxyService.getServiceUrl()) .build(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java index 7fdd9fd95bf87..847e6a569ca6e 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java @@ -98,7 +98,8 @@ public void testPartitions() throws Exception { PulsarClient client = PulsarClient.builder() .serviceUrl(proxyService.getServiceUrlTls()) .allowTlsInsecureConnection(false).tlsTrustCertsFilePath(TLS_TRUST_CERT_FILE_PATH).build(); - admin.tenants().createTenant("sample", new TenantInfo()); + TenantInfo tenantInfo = createDefaultTenantInfo(); + admin.tenants().createTenant("sample", tenantInfo); admin.topics().createPartitionedTopic("persistent://sample/test/local/partitioned-topic", 2); Producer producer = client.newProducer(Schema.BYTES).topic("persistent://sample/test/local/partitioned-topic") From 91dfa1a9648e8f24a649d06ce96f38ef933a3c15 Mon Sep 17 00:00:00 2001 From: lipenghui Date: Sun, 16 Feb 2020 22:55:06 +0800 Subject: [PATCH 04/74] Introduce maxMessagePublishBufferSizeInMB configuration to avoid broker OOM (#6178) Motivation Introduce maxMessagePublishBufferSizeInMB configuration to avoid broker OOM. Modifications If the processing message size exceeds this value, the broker will stop read data from the connection. When available size > half of the maxMessagePublishBufferSizeInMB, start auto-read data from the connection. --- conf/broker.conf | 12 ++ .../pulsar/broker/ServiceConfiguration.java | 17 ++ .../pulsar/broker/service/AbstractTopic.java | 22 ++- .../pulsar/broker/service/BrokerService.java | 53 ++++++ .../pulsar/broker/service/Producer.java | 12 +- .../pulsar/broker/service/ServerCnx.java | 50 +++++- .../MessagePublishBufferThrottleTest.java | 168 ++++++++++++++++++ .../naming/ServiceConfigurationTest.java | 1 + .../configurations/pulsar_broker_test.conf | 1 + 9 files changed, 319 insertions(+), 17 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessagePublishBufferThrottleTest.java diff --git a/conf/broker.conf b/conf/broker.conf index a98aa16ad84dc..55838f43f849d 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -324,6 +324,18 @@ replicatedSubscriptionsSnapshotTimeoutSeconds=30 # Max number of snapshot to be cached per subscription. replicatedSubscriptionsSnapshotMaxCachedPerSubscription=10 +# Max memory size for broker handling messages sending from producers. +# If the processing message size exceed this value, broker will stop read data +# from the connection. The processing messages means messages are sends to broker +# but broker have not send response to client, usually waiting to write to bookies. +# It's shared across all the topics running in the same broker. +# Use -1 to disable the memory limitation. Default is 1/2 of direct memory. +maxMessagePublishBufferSizeInMB= + +# Interval between checks to see if message publish buffer size is exceed the max message publish buffer size +# Use 0 or negative number to disable the max publish buffer limiting. +messagePublishBufferCheckIntervalInMillis=100 + ### --- Authentication --- ### # Role names that are treated as "proxy roles". If the broker sees a request with #role as proxyRoles - it will demand to see a valid original principal. diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index d44f84644e920..f015a92b66952 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -604,6 +604,23 @@ public class ServiceConfiguration implements PulsarConfiguration { doc = "Max number of snapshot to be cached per subscription.") private int replicatedSubscriptionsSnapshotMaxCachedPerSubscription = 10; + @FieldContext( + category = CATEGORY_SERVER, + doc = "Max memory size for broker handling messages sending from producers.\n\n" + + " If the processing message size exceed this value, broker will stop read data" + + " from the connection. The processing messages means messages are sends to broker" + + " but broker have not send response to client, usually waiting to write to bookies.\n\n" + + " It's shared across all the topics running in the same broker.\n\n" + + " Use -1 to disable the memory limitation. Default is 1/2 of direct memory.\n\n") + private int maxMessagePublishBufferSizeInMB = Math.max(64, + (int) (PlatformDependent.maxDirectMemory() / 2 / (1024 * 1024))); + + @FieldContext( + category = CATEGORY_SERVER, + doc = "Interval between checks to see if message publish buffer size is exceed the max message publish buffer size" + ) + private int messagePublishBufferCheckIntervalInMillis = 100; + /**** --- Messaging Protocols --- ****/ @FieldContext( diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java index 173e3d074e900..b1a166ea26f14 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java @@ -277,7 +277,7 @@ public void incrementPublishCount(int numOfMessages, long msgSizeInBytes) { public void resetTopicPublishCountAndEnableReadIfRequired() { // broker rate not exceeded. and completed topic limiter reset. if (!getBrokerPublishRateLimiter().isPublishRateExceeded() && topicPublishRateLimiter.resetPublishCount()) { - enableProducerRead(); + enableProducerReadForPublishRateLimiting(); } } @@ -285,16 +285,28 @@ public void resetTopicPublishCountAndEnableReadIfRequired() { public void resetBrokerPublishCountAndEnableReadIfRequired(boolean doneBrokerReset) { // topic rate not exceeded, and completed broker limiter reset. if (!topicPublishRateLimiter.isPublishRateExceeded() && doneBrokerReset) { - enableProducerRead(); + enableProducerReadForPublishRateLimiting(); } } /** * it sets cnx auto-readable if producer's cnx is disabled due to publish-throttling */ - protected void enableProducerRead() { + protected void enableProducerReadForPublishRateLimiting() { if (producers != null) { - producers.values().forEach(producer -> producer.getCnx().enableCnxAutoRead()); + producers.values().forEach(producer -> { + producer.getCnx().cancelPublishRateLimiting(); + producer.getCnx().enableCnxAutoRead(); + }); + } + } + + protected void enableProducerReadForPublishBufferLimiting() { + if (producers != null) { + producers.values().forEach(producer -> { + producer.getCnx().cancelPublishBufferLimiting(); + producer.getCnx().enableCnxAutoRead(); + }); } } @@ -389,7 +401,7 @@ private void updatePublishDispatcher(Policies policies) { } else { log.info("Disabling publish throttling for {}", this.topic); this.topicPublishRateLimiter = PublishRateLimiter.DISABLED_RATE_LIMITER; - enableProducerRead(); + enableProducerReadForPublishRateLimiting(); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 46f322ff3729b..db055b69b9ddc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -27,6 +27,7 @@ import static org.apache.pulsar.broker.cache.LocalZooKeeperCacheService.LOCAL_POLICIES_ROOT; import static org.apache.pulsar.broker.web.PulsarWebResource.joinPath; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Queues; @@ -58,6 +59,7 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.LongAdder; import java.util.concurrent.locks.ReadWriteLock; @@ -185,6 +187,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener 0 ? + pulsar.getConfiguration().getMaxMessagePublishBufferSizeInMB() * 1024 * 1024 : -1; + this.resumeProducerReadMessagePublishBufferBytes = this.maxMessagePublishBufferBytes / 2; this.managedLedgerFactory = pulsar.getManagedLedgerFactory(); this.topics = new ConcurrentOpenHashMap<>(); this.replicationClients = new ConcurrentOpenHashMap<>(); @@ -257,6 +267,8 @@ public BrokerService(PulsarService pulsar) throws Exception { .newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-msg-expiry-monitor")); this.compactionMonitor = Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-compaction-monitor")); + this.messagePublishBufferMonitor = + Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-publish-buffer-monitor")); this.backlogQuotaManager = new BacklogQuotaManager(pulsar); this.backlogQuotaChecker = Executors @@ -386,6 +398,7 @@ public void start() throws Exception { this.startInactivityMonitor(); this.startMessageExpiryMonitor(); this.startCompactionMonitor(); + this.startMessagePublishBufferMonitor(); this.startBacklogQuotaChecker(); this.updateBrokerPublisherThrottlingMaxRate(); // register listener to capture zk-latency @@ -438,6 +451,14 @@ protected void startCompactionMonitor() { } } + protected void startMessagePublishBufferMonitor() { + int interval = pulsar().getConfiguration().getMessagePublishBufferCheckIntervalInMillis(); + if (interval > 0 && maxMessagePublishBufferBytes > 0) { + messagePublishBufferMonitor.scheduleAtFixedRate(safeRun(this::checkMessagePublishBuffer), + interval, interval, TimeUnit.MILLISECONDS); + } + } + protected void startBacklogQuotaChecker() { if (pulsar().getConfiguration().isBacklogQuotaCheckEnabled()) { final int interval = pulsar().getConfiguration().getBacklogQuotaCheckIntervalInSeconds(); @@ -2011,4 +2032,36 @@ public Optional getListenPortTls() { return Optional.empty(); } } + + private void checkMessagePublishBuffer() { + AtomicLong currentMessagePublishBufferBytes = new AtomicLong(); + foreachProducer(producer -> currentMessagePublishBufferBytes.addAndGet(producer.getCnx().getMessagePublishBufferSize())); + if (currentMessagePublishBufferBytes.get() >= maxMessagePublishBufferBytes + && !reachMessagePublishBufferThreshold) { + reachMessagePublishBufferThreshold = true; + } + if (currentMessagePublishBufferBytes.get() < resumeProducerReadMessagePublishBufferBytes + && reachMessagePublishBufferThreshold) { + reachMessagePublishBufferThreshold = false; + forEachTopic(topic -> ((AbstractTopic) topic).enableProducerReadForPublishBufferLimiting()); + } + } + + private void foreachProducer(Consumer consumer) { + topics.forEach((n, t) -> { + Optional topic = extractTopic(t); + topic.ifPresent(value -> value.getProducers().values().forEach(consumer)); + }); + } + + public boolean isReachMessagePublishBufferThreshold() { + return reachMessagePublishBufferThreshold; + } + + @VisibleForTesting + long getCurrentMessagePublishBufferSize() { + AtomicLong currentMessagePublishBufferBytes = new AtomicLong(); + foreachProducer(producer -> currentMessagePublishBufferBytes.addAndGet(producer.getCnx().getMessagePublishBufferSize())); + return currentMessagePublishBufferBytes.get(); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java index b6c2b496c522c..a7c97da3414b7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java @@ -147,7 +147,7 @@ public void publishMessage(long producerId, long lowestSequenceId, long highestS cnx.ctx().channel().eventLoop().execute(() -> { cnx.ctx().writeAndFlush(Commands.newSendError(producerId, highestSequenceId, ServerError.MetadataError, "Invalid lowest or highest sequence id")); - cnx.completedSendOperation(isNonPersistentTopic); + cnx.completedSendOperation(isNonPersistentTopic, headersAndPayload.readableBytes()); }); return; } @@ -160,7 +160,7 @@ public void beforePublish(long producerId, long sequenceId, ByteBuf headersAndPa cnx.ctx().channel().eventLoop().execute(() -> { cnx.ctx().writeAndFlush(Commands.newSendError(producerId, sequenceId, ServerError.PersistenceError, "Producer is closed")); - cnx.completedSendOperation(isNonPersistentTopic); + cnx.completedSendOperation(isNonPersistentTopic, headersAndPayload.readableBytes()); }); return; @@ -170,7 +170,7 @@ public void beforePublish(long producerId, long sequenceId, ByteBuf headersAndPa cnx.ctx().channel().eventLoop().execute(() -> { cnx.ctx().writeAndFlush( Commands.newSendError(producerId, sequenceId, ServerError.ChecksumError, "Checksum failed on the broker")); - cnx.completedSendOperation(isNonPersistentTopic); + cnx.completedSendOperation(isNonPersistentTopic, headersAndPayload.readableBytes()); }); return; } @@ -187,7 +187,7 @@ public void beforePublish(long producerId, long sequenceId, ByteBuf headersAndPa cnx.ctx().channel().eventLoop().execute(() -> { cnx.ctx().writeAndFlush(Commands.newSendError(producerId, sequenceId, ServerError.MetadataError, "Messages must be encrypted")); - cnx.completedSendOperation(isNonPersistentTopic); + cnx.completedSendOperation(isNonPersistentTopic, headersAndPayload.readableBytes()); }); return; } @@ -353,7 +353,7 @@ public void completed(Exception exception, long ledgerId, long entryId) { producer.cnx.ctx().writeAndFlush(Commands.newSendError(producer.producerId, callBackSequenceId, serverError, exception.getMessage())); } - producer.cnx.completedSendOperation(producer.isNonPersistentTopic); + producer.cnx.completedSendOperation(producer.isNonPersistentTopic, msgSize); producer.publishOperationCompleted(); recycle(); }); @@ -385,7 +385,7 @@ public void run() { producer.cnx.ctx().writeAndFlush( Commands.newSendReceipt(producer.producerId, sequenceId, highestSequenceId, ledgerId, entryId), producer.cnx.ctx().voidPromise()); - producer.cnx.completedSendOperation(producer.isNonPersistentTopic); + producer.cnx.completedSendOperation(producer.isNonPersistentTopic, msgSize); producer.publishOperationCompleted(); recycle(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index d79f41bacbaa3..50a5107c05fd9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -25,6 +25,7 @@ import static org.apache.pulsar.common.protocol.Commands.newLookupErrorResponse; import static org.apache.pulsar.common.api.proto.PulsarApi.ProtocolVersion.v5; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import io.netty.buffer.ByteBuf; @@ -151,6 +152,9 @@ public class ServerCnx extends PulsarHandler { // Flag to manage throttling-rate by atomically enable/disable read-channel. private volatile boolean autoReadDisabledRateLimiting = false; private FeatureFlags features; + // Flag to manage throttling-publish-buffer by atomically enable/disable read-channel. + private volatile boolean autoReadDisabledPublishBufferLimiting = false; + private volatile long messagePublishBufferSize = 0; enum State { Start, Connected, Failed, Connecting @@ -1154,7 +1158,7 @@ protected void handleSend(CommandSend send, ByteBuf headersAndPayload) { } } - startSendOperation(producer); + startSendOperation(producer, headersAndPayload.readableBytes()); // Persist the message if (send.hasHighestSequenceId() && send.getSequenceId() <= send.getHighestSequenceId()) { @@ -1677,17 +1681,24 @@ public boolean isWritable() { return ctx.channel().isWritable(); } - public void startSendOperation(Producer producer) { + private void startSendOperation(Producer producer, int msgSize) { + messagePublishBufferSize += msgSize; boolean isPublishRateExceeded = producer.getTopic().isPublishRateExceeded(); if (++pendingSendRequest == MaxPendingSendRequests || isPublishRateExceeded) { // When the quota of pending send requests is reached, stop reading from socket to cause backpressure on // client connection, possibly shared between multiple producers ctx.channel().config().setAutoRead(false); autoReadDisabledRateLimiting = isPublishRateExceeded; + + } + if (getBrokerService().isReachMessagePublishBufferThreshold()) { + ctx.channel().config().setAutoRead(false); + autoReadDisabledPublishBufferLimiting = true; } } - public void completedSendOperation(boolean isNonPersistentTopic) { + void completedSendOperation(boolean isNonPersistentTopic, int msgSize) { + messagePublishBufferSize -= msgSize; if (--pendingSendRequest == ResumeReadsThreshold) { // Resume reading from socket ctx.channel().config().setAutoRead(true); @@ -1699,19 +1710,32 @@ public void completedSendOperation(boolean isNonPersistentTopic) { } } - public void enableCnxAutoRead() { + void enableCnxAutoRead() { // we can add check (&& pendingSendRequest < MaxPendingSendRequests) here but then it requires // pendingSendRequest to be volatile and it can be expensive while writing. also this will be called on if // throttling is enable on the topic. so, avoid pendingSendRequest check will be fine. - if (!ctx.channel().config().isAutoRead() && autoReadDisabledRateLimiting) { + if (!ctx.channel().config().isAutoRead() && !autoReadDisabledRateLimiting && !autoReadDisabledPublishBufferLimiting) { // Resume reading from socket if pending-request is not reached to threshold ctx.channel().config().setAutoRead(true); // triggers channel read ctx.read(); + } + } + + @VisibleForTesting + void cancelPublishRateLimiting() { + if (autoReadDisabledRateLimiting) { autoReadDisabledRateLimiting = false; } } + @VisibleForTesting + void cancelPublishBufferLimiting() { + if (autoReadDisabledPublishBufferLimiting) { + autoReadDisabledPublishBufferLimiting = false; + } + } + private ServerError getErrorCode(CompletableFuture future) { ServerError error = ServerError.UnknownError; try { @@ -1724,7 +1748,7 @@ private ServerError getErrorCode(CompletableFuture future) { return error; } - private final void disableTcpNoDelayIfNeeded(String topic, String producerName) { + private void disableTcpNoDelayIfNeeded(String topic, String producerName) { if (producerName != null && producerName.startsWith(replicatorPrefix)) { // Re-enable nagle algorithm on connections used for replication purposes try { @@ -1799,4 +1823,18 @@ boolean supportsAuthenticationRefresh() { public String getClientVersion() { return clientVersion; } + + public long getMessagePublishBufferSize() { + return this.messagePublishBufferSize; + } + + @VisibleForTesting + void setMessagePublishBufferSize(long bufferSize) { + this.messagePublishBufferSize = bufferSize; + } + + @VisibleForTesting + void setAutoReadDisabledRateLimiting(boolean isLimiting) { + this.autoReadDisabledRateLimiting = isLimiting; + } } \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessagePublishBufferThrottleTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessagePublishBufferThrottleTest.java new file mode 100644 index 0000000000000..5397725a99c67 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessagePublishBufferThrottleTest.java @@ -0,0 +1,168 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.service; + +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.common.util.FutureUtil; +import org.testng.Assert; +import org.testng.annotations.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + */ +public class MessagePublishBufferThrottleTest extends BrokerTestBase { + + @Override + protected void setup() throws Exception { + //No-op + } + + @Override + protected void cleanup() throws Exception { + //No-op + } + + @Test + public void testMessagePublishBufferThrottleDisabled() throws Exception { + conf.setMaxMessagePublishBufferSizeInMB(-1); + conf.setMessagePublishBufferCheckIntervalInMillis(10); + super.baseSetup(); + final String topic = "persistent://prop/ns-abc/testMessagePublishBufferThrottleDisabled"; + Producer producer = pulsarClient.newProducer() + .topic(topic) + .producerName("producer-name") + .create(); + Topic topicRef = pulsar.getBrokerService().getTopicReference(topic).get(); + Assert.assertNotNull(topicRef); + ((AbstractTopic)topicRef).producers.get("producer-name").getCnx().setMessagePublishBufferSize(Long.MAX_VALUE / 2); + Thread.sleep(20); + Assert.assertFalse(pulsar.getBrokerService().isReachMessagePublishBufferThreshold()); + List> futures = new ArrayList<>(); + // Make sure the producer can publish succeed. + for (int i = 0; i < 10; i++) { + futures.add(producer.sendAsync(new byte[1024 * 1024])); + } + FutureUtil.waitForAll(futures).get(); + for (CompletableFuture future : futures) { + Assert.assertNotNull(future.get()); + } + Thread.sleep(20); + Assert.assertFalse(pulsar.getBrokerService().isReachMessagePublishBufferThreshold()); + super.internalCleanup(); + } + + @Test + public void testMessagePublishBufferThrottleEnable() throws Exception { + conf.setMaxMessagePublishBufferSizeInMB(1); + conf.setMessagePublishBufferCheckIntervalInMillis(2); + super.baseSetup(); + Thread.sleep(4); + Assert.assertFalse(pulsar.getBrokerService().isReachMessagePublishBufferThreshold()); + final String topic = "persistent://prop/ns-abc/testMessagePublishBufferThrottleEnable"; + Producer producer = pulsarClient.newProducer() + .topic(topic) + .producerName("producer-name") + .create(); + Topic topicRef = pulsar.getBrokerService().getTopicReference(topic).get(); + Assert.assertNotNull(topicRef); + ((AbstractTopic)topicRef).producers.get("producer-name").getCnx().setMessagePublishBufferSize(Long.MAX_VALUE / 2); + Thread.sleep(4); + Assert.assertTrue(pulsar.getBrokerService().isReachMessagePublishBufferThreshold()); + // The first message can publish success, but the second message should be blocked + producer.sendAsync(new byte[1024]).get(1, TimeUnit.SECONDS); + MessageId messageId = null; + try { + messageId = producer.sendAsync(new byte[1024]).get(1, TimeUnit.SECONDS); + Assert.fail("should failed, because producer blocked by publish buffer limiting"); + } catch (TimeoutException e) { + // No-op + } + Assert.assertNull(messageId); + + ((AbstractTopic)topicRef).producers.get("producer-name").getCnx().setMessagePublishBufferSize(0L); + Thread.sleep(4); + + List> futures = new ArrayList<>(); + // Make sure the producer can publish succeed. + for (int i = 0; i < 10; i++) { + futures.add(producer.sendAsync(new byte[1024 * 1024])); + } + FutureUtil.waitForAll(futures).get(); + for (CompletableFuture future : futures) { + Assert.assertNotNull(future.get()); + } + Thread.sleep(4); + Assert.assertEquals(pulsar.getBrokerService().getCurrentMessagePublishBufferSize(), 0L); + super.internalCleanup(); + } + + @Test + public void testBlockByPublishRateLimiting() throws Exception { + conf.setMaxMessagePublishBufferSizeInMB(1); + conf.setMessagePublishBufferCheckIntervalInMillis(2); + super.baseSetup(); + Thread.sleep(4); + Assert.assertFalse(pulsar.getBrokerService().isReachMessagePublishBufferThreshold()); + final String topic = "persistent://prop/ns-abc/testMessagePublishBufferThrottleEnable"; + Producer producer = pulsarClient.newProducer() + .topic(topic) + .producerName("producer-name") + .create(); + Topic topicRef = pulsar.getBrokerService().getTopicReference(topic).get(); + Assert.assertNotNull(topicRef); + ((AbstractTopic)topicRef).producers.get("producer-name").getCnx().setMessagePublishBufferSize(Long.MAX_VALUE / 2); + producer.sendAsync(new byte[1024]).get(1, TimeUnit.SECONDS); + + Thread.sleep(4); + ((AbstractTopic)topicRef).producers.get("producer-name").getCnx().setAutoReadDisabledRateLimiting(true); + ((AbstractTopic)topicRef).producers.get("producer-name").getCnx().setMessagePublishBufferSize(0); + Thread.sleep(4); + Assert.assertFalse(pulsar.getBrokerService().isReachMessagePublishBufferThreshold()); + MessageId messageId = null; + try { + messageId = producer.sendAsync(new byte[1024]).get(1, TimeUnit.SECONDS); + Assert.fail("should failed, because producer blocked by publish buffer limiting"); + } catch (TimeoutException e) { + // No-op + } + Assert.assertNull(messageId); + + ((AbstractTopic)topicRef).producers.get("producer-name").getCnx().setAutoReadDisabledRateLimiting(false); + ((AbstractTopic)topicRef).producers.get("producer-name").getCnx().enableCnxAutoRead(); + + List> futures = new ArrayList<>(); + // Make sure the producer can publish succeed. + for (int i = 0; i < 10; i++) { + futures.add(producer.sendAsync(new byte[1024 * 1024])); + } + FutureUtil.waitForAll(futures).get(); + for (CompletableFuture future : futures) { + Assert.assertNotNull(future.get()); + } + Thread.sleep(4); + Assert.assertEquals(pulsar.getBrokerService().getCurrentMessagePublishBufferSize(), 0L); + super.internalCleanup(); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java index 24e7152856718..258c1234411c1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java @@ -63,6 +63,7 @@ public void testInit() throws Exception { assertEquals(config.getBrokerDeleteInactiveTopicsMode(), InactiveTopicDeleteMode.delete_when_subscriptions_caught_up); assertEquals(config.getDefaultNamespaceBundleSplitAlgorithm(), "topic_count_equally_divide"); assertEquals(config.getSupportedNamespaceBundleSplitAlgorithms().size(), 1); + assertEquals(config.getMaxMessagePublishBufferSizeInMB(), -1); } @Test diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf index bd966ad76f5b7..cffb006d5b350 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf @@ -88,3 +88,4 @@ replicatorPrefix=pulsar.repl brokerDeleteInactiveTopicsMode=delete_when_subscriptions_caught_up supportedNamespaceBundleSplitAlgorithms=[range_equally_divide] defaultNamespaceBundleSplitAlgorithm=topic_count_equally_divide +maxMessagePublishBufferSizeInMB=-1 From df152109415f2b10dd83e8afe50d9db7ab7cbad5 Mon Sep 17 00:00:00 2001 From: lipenghui Date: Mon, 17 Feb 2020 05:46:18 +0800 Subject: [PATCH 05/74] Enable get precise backlog and backlog without delayed messages. (#6310) Fixes #6045 #6281 ### Motivation Enable get precise backlog and backlog without delayed messages. ### Verifying this change Added new unit tests for the change. --- conf/broker.conf | 5 + conf/standalone.conf | 5 + .../bookkeeper/mledger/ManagedCursor.java | 3 +- .../mledger/impl/ManagedCursorImpl.java | 6 +- .../mledger/impl/ManagedLedgerImpl.java | 7 + .../mledger/impl/ManagedLedgerMBeanImpl.java | 2 +- .../impl/ManagedCursorContainerTest.java | 2 +- .../impl/ManagedCursorListAckTest.java | 8 +- .../mledger/impl/ManagedCursorTest.java | 132 ++++++------- .../mledger/impl/ManagedLedgerBkTest.java | 14 +- .../mledger/impl/ManagedLedgerErrorsTest.java | 8 +- .../mledger/impl/ManagedLedgerTest.java | 12 +- .../mledger/impl/NonDurableCursorTest.java | 36 ++-- .../pulsar/broker/ServiceConfiguration.java | 8 + .../admin/impl/PersistentTopicsBase.java | 8 +- .../broker/admin/v1/NonPersistentTopics.java | 2 +- .../broker/admin/v1/PersistentTopics.java | 4 +- .../broker/admin/v2/NonPersistentTopics.java | 6 +- .../broker/admin/v2/PersistentTopics.java | 14 +- .../broker/service/BacklogQuotaManager.java | 2 +- .../pulsar/broker/service/BrokerService.java | 2 +- .../pulsar/broker/service/ServerCnx.java | 2 +- .../pulsar/broker/service/Subscription.java | 2 +- .../apache/pulsar/broker/service/Topic.java | 2 +- .../NonPersistentSubscription.java | 2 +- .../nonpersistent/NonPersistentTopic.java | 6 +- .../persistent/CompactorSubscription.java | 2 +- ...PersistentDispatcherMultipleConsumers.java | 2 +- ...sistentDispatcherSingleActiveConsumer.java | 2 +- .../PersistentMessageExpiryMonitor.java | 4 +- .../persistent/PersistentReplicator.java | 16 +- .../persistent/PersistentSubscription.java | 31 +-- .../service/persistent/PersistentTopic.java | 10 +- .../prometheus/AggregatedNamespaceStats.java | 1 + .../AggregatedSubscriptionStats.java | 2 + .../prometheus/NamespaceStatsAggregator.java | 13 +- .../broker/stats/prometheus/TopicStats.java | 1 + .../pulsar/broker/admin/AdminApiTest2.java | 183 ++++++++++++++++++ .../broker/service/BatchMessageTest.java | 42 ++-- .../broker/service/BrokerServiceTest.java | 18 +- .../service/PersistentFailoverE2ETest.java | 10 +- .../service/PersistentQueueE2ETest.java | 4 +- .../service/PersistentTopicE2ETest.java | 18 +- .../broker/service/SubscriptionSeekTest.java | 20 +- .../AggregatedNamespaceStatsTest.java | 3 + .../api/DispatcherBlockConsumerTest.java | 4 +- .../client/api/NonPersistentTopicTest.java | 10 +- .../pulsar/client/impl/MessageParserTest.java | 2 +- .../client/impl/TopicsConsumerImplTest.java | 4 +- .../apache/pulsar/client/admin/Topics.java | 28 ++- .../client/admin/internal/TopicsImpl.java | 16 +- .../pulsar/admin/cli/PulsarAdminToolTest.java | 4 +- .../apache/pulsar/admin/cli/CmdTopics.java | 12 +- .../policies/data/SubscriptionStats.java | 5 + .../docs/admin-api-non-partitioned-topics.md | 56 ++++++ site2/docs/admin-api-partitioned-topics.md | 5 +- site2/docs/reference-configuration.md | 2 + 57 files changed, 577 insertions(+), 253 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 55838f43f849d..cf01b2d2fddc4 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -790,6 +790,11 @@ exposePublisherStats=true statsUpdateFrequencyInSecs=60 statsUpdateInitialDelayInSecs=60 +# Enable expose the precise backlog stats. +# Set false to use published counter and consumed counter to calculate, this would be more efficient but may be inaccurate. +# Default is false. +exposePreciseBacklogInPrometheus=false + ### --- Schema storage --- ### # The schema storage implementation used by this broker schemaRegistryStorageClassName=org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorageFactory diff --git a/conf/standalone.conf b/conf/standalone.conf index 3f31b422c5ebd..7dff0c3c913ab 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -531,6 +531,11 @@ exposeTopicLevelMetricsInPrometheus=true # Enable topic level metrics exposePublisherStats=true +# Enable expose the precise backlog stats. +# Set false to use published counter and consumed counter to calculate, this would be more efficient but may be inaccurate. +# Default is false. +exposePreciseBacklogInPrometheus=false + ### --- Deprecated config variables --- ### # Deprecated. Use configurationStoreServers diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java index ae0426935637c..6f12fb2cb20dc 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java @@ -193,9 +193,10 @@ void asyncGetNthEntry(int n, IndividualDeletedEntries deletedEntries, ReadEntryC * *

This method has linear time complexity on the number of ledgers included in the managed ledger. * + * @param isPrecise set to true to get precise backlog count * @return the number of entries */ - long getNumberOfEntriesInBacklog(); + long getNumberOfEntriesInBacklog(boolean isPrecise); /** * This signals that the reader is done with all the entries up to "position" (included). This can potentially diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index c75c45a64a579..9498145de2823 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -734,12 +734,16 @@ public long getEstimatedSizeSinceMarkDeletePosition() { } @Override - public long getNumberOfEntriesInBacklog() { + public long getNumberOfEntriesInBacklog(boolean isPrecise) { if (log.isDebugEnabled()) { log.debug("[{}] Consumer {} cursor ml-entries: {} -- deleted-counter: {} other counters: mdPos {} rdPos {}", ledger.getName(), name, ManagedLedgerImpl.ENTRIES_ADDED_COUNTER_UPDATER.get(ledger), messagesConsumedCounter, markDeletePosition, readPosition); } + if (isPrecise) { + return getNumberOfEntries(Range.closed(markDeletePosition, ledger.getLastPosition())) - 1; + } + long backlog = ManagedLedgerImpl.ENTRIES_ADDED_COUNTER_UPDATER.get(ledger) - messagesConsumedCounter; if (backlog < 0) { // In some case the counters get incorrect values, fall back to the precise backlog count diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 982e588286b53..ecf99a426dbd4 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -24,6 +24,7 @@ import static org.apache.bookkeeper.mledger.util.Errors.isNoSuchLedgerExistsException; import static org.apache.bookkeeper.mledger.util.SafeRun.safeRun; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.BoundType; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -142,6 +143,7 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback { private final ManagedCursorContainer activeCursors = new ManagedCursorContainer(); // Ever increasing counter of entries added + @VisibleForTesting static final AtomicLongFieldUpdater ENTRIES_ADDED_COUNTER_UPDATER = AtomicLongFieldUpdater .newUpdater(ManagedLedgerImpl.class, "entriesAddedCounter"); @SuppressWarnings("unused") @@ -3176,6 +3178,11 @@ public long getOffloadedSize() { return offloadedSize; } + @VisibleForTesting + public void setEntriesAddedCounter(long count) { + ENTRIES_ADDED_COUNTER_UPDATER.set(this, count); + } + private static final Logger log = LoggerFactory.getLogger(ManagedLedgerImpl.class); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java index 38c67f49f8085..ec1dc74a290bd 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java @@ -263,7 +263,7 @@ public long getNumberOfMessagesInBacklog() { long count = 0; for (ManagedCursor cursor : managedLedger.getCursors()) { - count += cursor.getNumberOfEntriesInBacklog(); + count += cursor.getNumberOfEntriesInBacklog(false); } return count; diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java index 21f5747919b2f..99e7aec740bed 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java @@ -92,7 +92,7 @@ public long getNumberOfEntries() { } @Override - public long getNumberOfEntriesInBacklog() { + public long getNumberOfEntriesInBacklog(boolean isPrecise) { return 0; } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorListAckTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorListAckTest.java index aaf7cd1c6f282..dc876e19eb1ab 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorListAckTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorListAckTest.java @@ -51,24 +51,24 @@ void testMultiPositionDelete() throws Exception { Position p7 = ledger.addEntry("dummy-entry-7".getBytes(Encoding)); assertEquals(c1.getNumberOfEntries(), 7); - assertEquals(c1.getNumberOfEntriesInBacklog(), 7); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 7); c1.delete(Lists.newArrayList(p2, p3, p5, p7)); assertEquals(c1.getNumberOfEntries(), 3); - assertEquals(c1.getNumberOfEntriesInBacklog(), 3); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 3); assertEquals(c1.getMarkDeletedPosition(), p0); c1.delete(Lists.newArrayList(p1)); assertEquals(c1.getNumberOfEntries(), 2); - assertEquals(c1.getNumberOfEntriesInBacklog(), 2); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 2); assertEquals(c1.getMarkDeletedPosition(), p3); c1.delete(Lists.newArrayList(p4, p6, p7)); assertEquals(c1.getNumberOfEntries(), 0); - assertEquals(c1.getNumberOfEntriesInBacklog(), 0); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 0); assertEquals(c1.getMarkDeletedPosition(), p7); } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 94faa9931d391..75d0280ca52b7 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -265,31 +265,31 @@ void testNumberOfEntriesInBacklog() throws Exception { Position p4 = ledger.addEntry("dummy-entry-4".getBytes(Encoding)); ManagedCursor c5 = ledger.openCursor("c5"); - assertEquals(c1.getNumberOfEntriesInBacklog(), 4); - assertEquals(c2.getNumberOfEntriesInBacklog(), 3); - assertEquals(c3.getNumberOfEntriesInBacklog(), 2); - assertEquals(c4.getNumberOfEntriesInBacklog(), 1); - assertEquals(c5.getNumberOfEntriesInBacklog(), 0); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 4); + assertEquals(c2.getNumberOfEntriesInBacklog(false), 3); + assertEquals(c3.getNumberOfEntriesInBacklog(false), 2); + assertEquals(c4.getNumberOfEntriesInBacklog(false), 1); + assertEquals(c5.getNumberOfEntriesInBacklog(false), 0); List entries = c1.readEntries(2); assertEquals(entries.size(), 2); entries.forEach(e -> e.release()); assertEquals(c1.getNumberOfEntries(), 2); - assertEquals(c1.getNumberOfEntriesInBacklog(), 4); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 4); c1.markDelete(p1); assertEquals(c1.getNumberOfEntries(), 2); - assertEquals(c1.getNumberOfEntriesInBacklog(), 3); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 3); c1.delete(p3); assertEquals(c1.getNumberOfEntries(), 1); - assertEquals(c1.getNumberOfEntriesInBacklog(), 2); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 2); c1.markDelete(p4); assertEquals(c1.getNumberOfEntries(), 0); - assertEquals(c1.getNumberOfEntriesInBacklog(), 0); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 0); } @Test(timeOut = 20000) @@ -315,11 +315,11 @@ void testNumberOfEntriesInBacklogWithFallback() throws Exception { field.setLong(c4, counter); field.setLong(c5, counter); - assertEquals(c1.getNumberOfEntriesInBacklog(), 4); - assertEquals(c2.getNumberOfEntriesInBacklog(), 3); - assertEquals(c3.getNumberOfEntriesInBacklog(), 2); - assertEquals(c4.getNumberOfEntriesInBacklog(), 1); - assertEquals(c5.getNumberOfEntriesInBacklog(), 0); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 4); + assertEquals(c2.getNumberOfEntriesInBacklog(false), 3); + assertEquals(c3.getNumberOfEntriesInBacklog(false), 2); + assertEquals(c4.getNumberOfEntriesInBacklog(false), 1); + assertEquals(c5.getNumberOfEntriesInBacklog(false), 0); } @Test(timeOut = 20000) @@ -811,34 +811,34 @@ void rewind() throws Exception { log.debug("p4: {}", p4); assertEquals(c1.getNumberOfEntries(), 4); - assertEquals(c1.getNumberOfEntriesInBacklog(), 4); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 4); c1.markDelete(p1); assertEquals(c1.getNumberOfEntries(), 3); - assertEquals(c1.getNumberOfEntriesInBacklog(), 3); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 3); List entries = c1.readEntries(10); assertEquals(entries.size(), 3); entries.forEach(e -> e.release()); assertEquals(c1.getNumberOfEntries(), 0); - assertEquals(c1.getNumberOfEntriesInBacklog(), 3); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 3); c1.rewind(); assertEquals(c1.getNumberOfEntries(), 3); - assertEquals(c1.getNumberOfEntriesInBacklog(), 3); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 3); c1.markDelete(p2); assertEquals(c1.getNumberOfEntries(), 2); - assertEquals(c1.getNumberOfEntriesInBacklog(), 2); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 2); entries = c1.readEntries(10); assertEquals(entries.size(), 2); entries.forEach(e -> e.release()); assertEquals(c1.getNumberOfEntries(), 0); - assertEquals(c1.getNumberOfEntriesInBacklog(), 2); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 2); c1.rewind(); assertEquals(c1.getNumberOfEntries(), 2); c1.markDelete(p4); assertEquals(c1.getNumberOfEntries(), 0); - assertEquals(c1.getNumberOfEntriesInBacklog(), 0); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 0); c1.rewind(); assertEquals(c1.getNumberOfEntries(), 0); ledger.addEntry("dummy-entry-5".getBytes(Encoding)); @@ -1324,26 +1324,26 @@ void testFilteringReadEntries() throws Exception { /* Position p6 = */ledger.addEntry("entry6".getBytes()); assertEquals(cursor.getNumberOfEntries(), 6); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 6); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 6); List entries = cursor.readEntries(3); assertEquals(entries.size(), 3); entries.forEach(e -> e.release()); assertEquals(cursor.getNumberOfEntries(), 3); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 6); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 6); log.info("Deleting {}", p5); cursor.delete(p5); assertEquals(cursor.getNumberOfEntries(), 2); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 5); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 5); entries = cursor.readEntries(3); assertEquals(entries.size(), 2); entries.forEach(e -> e.release()); assertEquals(cursor.getNumberOfEntries(), 0); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 5); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 5); } @Test(timeOut = 20000) @@ -1384,21 +1384,21 @@ void testCountingWithDeletedEntries() throws Exception { Position p8 = ledger.addEntry("entry8".getBytes()); assertEquals(cursor.getNumberOfEntries(), 8); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 8); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 8); cursor.delete(p8); assertEquals(cursor.getNumberOfEntries(), 7); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 7); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 7); cursor.delete(p1); assertEquals(cursor.getNumberOfEntries(), 6); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 6); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 6); cursor.delete(p7); cursor.delete(p6); cursor.delete(p5); assertEquals(cursor.getNumberOfEntries(), 3); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 3); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 3); } @Test(timeOut = 20000, dataProvider = "useOpenRangeSet") @@ -1504,22 +1504,22 @@ void testClearBacklog(boolean useOpenRangeSet) throws Exception { ManagedCursor c3 = ledger.openCursor("c3"); ledger.addEntry("dummy-entry-3".getBytes(Encoding)); - assertEquals(c1.getNumberOfEntriesInBacklog(), 3); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 3); assertEquals(c1.getNumberOfEntries(), 3); assertTrue(c1.hasMoreEntries()); c1.clearBacklog(); c3.clearBacklog(); - assertEquals(c1.getNumberOfEntriesInBacklog(), 0); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 0); assertEquals(c1.getNumberOfEntries(), 0); assertFalse(c1.hasMoreEntries()); - assertEquals(c2.getNumberOfEntriesInBacklog(), 2); + assertEquals(c2.getNumberOfEntriesInBacklog(false), 2); assertEquals(c2.getNumberOfEntries(), 2); assertTrue(c2.hasMoreEntries()); - assertEquals(c3.getNumberOfEntriesInBacklog(), 0); + assertEquals(c3.getNumberOfEntriesInBacklog(false), 0); assertEquals(c3.getNumberOfEntries(), 0); assertFalse(c3.hasMoreEntries()); @@ -1530,15 +1530,15 @@ void testClearBacklog(boolean useOpenRangeSet) throws Exception { c2 = ledger.openCursor("c2"); c3 = ledger.openCursor("c3"); - assertEquals(c1.getNumberOfEntriesInBacklog(), 0); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 0); assertEquals(c1.getNumberOfEntries(), 0); assertFalse(c1.hasMoreEntries()); - assertEquals(c2.getNumberOfEntriesInBacklog(), 2); + assertEquals(c2.getNumberOfEntriesInBacklog(false), 2); assertEquals(c2.getNumberOfEntries(), 2); assertTrue(c2.hasMoreEntries()); - assertEquals(c3.getNumberOfEntriesInBacklog(), 0); + assertEquals(c3.getNumberOfEntriesInBacklog(false), 0); assertEquals(c3.getNumberOfEntries(), 0); assertFalse(c3.hasMoreEntries()); factory2.shutdown(); @@ -1555,12 +1555,12 @@ void testRateLimitMarkDelete(boolean useOpenRangeSet) throws Exception { Position p2 = ledger.addEntry("dummy-entry-2".getBytes(Encoding)); Position p3 = ledger.addEntry("dummy-entry-3".getBytes(Encoding)); - assertEquals(c1.getNumberOfEntriesInBacklog(), 3); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 3); c1.markDelete(p1); c1.markDelete(p2); c1.markDelete(p3); - assertEquals(c1.getNumberOfEntriesInBacklog(), 0); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 0); // Re-open to recover from storage ManagedLedgerFactory factory2 = new ManagedLedgerFactoryImpl(bkc, bkc.getZkHandle()); @@ -1569,7 +1569,7 @@ void testRateLimitMarkDelete(boolean useOpenRangeSet) throws Exception { c1 = ledger.openCursor("c1"); // Only the 1st mark-delete was persisted - assertEquals(c1.getNumberOfEntriesInBacklog(), 2); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 2); factory2.shutdown(); } @@ -1585,57 +1585,57 @@ void deleteSingleMessageTwice(boolean useOpenRangeSet) throws Exception { Position p3 = ledger.addEntry("entry-3".getBytes(Encoding)); Position p4 = ledger.addEntry("entry-4".getBytes(Encoding)); - assertEquals(c1.getNumberOfEntriesInBacklog(), 4); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 4); assertEquals(c1.getNumberOfEntries(), 4); c1.delete(p1); - assertEquals(c1.getNumberOfEntriesInBacklog(), 3); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 3); assertEquals(c1.getNumberOfEntries(), 3); assertEquals(c1.getMarkDeletedPosition(), p1); assertEquals(c1.getReadPosition(), p2); // Should have not effect since p1 is already deleted c1.delete(p1); - assertEquals(c1.getNumberOfEntriesInBacklog(), 3); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 3); assertEquals(c1.getNumberOfEntries(), 3); assertEquals(c1.getMarkDeletedPosition(), p1); assertEquals(c1.getReadPosition(), p2); c1.delete(p2); - assertEquals(c1.getNumberOfEntriesInBacklog(), 2); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 2); assertEquals(c1.getNumberOfEntries(), 2); assertEquals(c1.getMarkDeletedPosition(), p2); assertEquals(c1.getReadPosition(), p3); // Should have not effect since p2 is already deleted c1.delete(p2); - assertEquals(c1.getNumberOfEntriesInBacklog(), 2); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 2); assertEquals(c1.getNumberOfEntries(), 2); assertEquals(c1.getMarkDeletedPosition(), p2); assertEquals(c1.getReadPosition(), p3); c1.delete(p3); - assertEquals(c1.getNumberOfEntriesInBacklog(), 1); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 1); assertEquals(c1.getNumberOfEntries(), 1); assertEquals(c1.getMarkDeletedPosition(), p3); assertEquals(c1.getReadPosition(), p4); // Should have not effect since p3 is already deleted c1.delete(p3); - assertEquals(c1.getNumberOfEntriesInBacklog(), 1); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 1); assertEquals(c1.getNumberOfEntries(), 1); assertEquals(c1.getMarkDeletedPosition(), p3); assertEquals(c1.getReadPosition(), p4); c1.delete(p4); - assertEquals(c1.getNumberOfEntriesInBacklog(), 0); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 0); assertEquals(c1.getNumberOfEntries(), 0); assertEquals(c1.getMarkDeletedPosition(), p4); assertEquals(c1.getReadPosition(), p4.getNext()); // Should have not effect since p4 is already deleted c1.delete(p4); - assertEquals(c1.getNumberOfEntriesInBacklog(), 0); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 0); assertEquals(c1.getNumberOfEntries(), 0); assertEquals(c1.getMarkDeletedPosition(), p4); assertEquals(c1.getReadPosition(), p4.getNext()); @@ -2367,19 +2367,19 @@ void outOfOrderAcks() throws Exception { positions.add(ledger.addEntry("entry".getBytes())); } - assertEquals(c1.getNumberOfEntriesInBacklog(), N); + assertEquals(c1.getNumberOfEntriesInBacklog(false), N); c1.delete(positions.get(3)); - assertEquals(c1.getNumberOfEntriesInBacklog(), N - 1); + assertEquals(c1.getNumberOfEntriesInBacklog(false), N - 1); c1.delete(positions.get(2)); - assertEquals(c1.getNumberOfEntriesInBacklog(), N - 2); + assertEquals(c1.getNumberOfEntriesInBacklog(false), N - 2); c1.delete(positions.get(1)); - assertEquals(c1.getNumberOfEntriesInBacklog(), N - 3); + assertEquals(c1.getNumberOfEntriesInBacklog(false), N - 3); c1.delete(positions.get(0)); - assertEquals(c1.getNumberOfEntriesInBacklog(), N - 4); + assertEquals(c1.getNumberOfEntriesInBacklog(false), N - 4); } @Test(timeOut = 20000) @@ -2394,17 +2394,17 @@ void randomOrderAcks() throws Exception { positions.add(ledger.addEntry("entry".getBytes())); } - assertEquals(c1.getNumberOfEntriesInBacklog(), N); + assertEquals(c1.getNumberOfEntriesInBacklog(false), N); // Randomize the ack sequence Collections.shuffle(positions); int toDelete = N; for (Position p : positions) { - assertEquals(c1.getNumberOfEntriesInBacklog(), toDelete); + assertEquals(c1.getNumberOfEntriesInBacklog(false), toDelete); c1.delete(p); --toDelete; - assertEquals(c1.getNumberOfEntriesInBacklog(), toDelete); + assertEquals(c1.getNumberOfEntriesInBacklog(false), toDelete); } } @@ -2572,7 +2572,7 @@ public void testOutOfOrderDeletePersistenceWithClose() throws Exception { c1.delete(addedPositions.get(8)); c1.delete(addedPositions.get(9)); - assertEquals(c1.getNumberOfEntriesInBacklog(), 20 - 5); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 20 - 5); ledger.close(); factory.shutdown(); @@ -2581,7 +2581,7 @@ public void testOutOfOrderDeletePersistenceWithClose() throws Exception { factory = new ManagedLedgerFactoryImpl(bkc, bkc.getZkHandle()); ledger = factory.open("my_test_ledger", new ManagedLedgerConfig()); c1 = ledger.openCursor("c1"); - assertEquals(c1.getNumberOfEntriesInBacklog(), 20 - 5); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 20 - 5); List entries = c1.readEntries(20); assertEquals(entries.size(), 20 - 5); @@ -2617,13 +2617,13 @@ public void testOutOfOrderDeletePersistenceAfterCrash() throws Exception { c1.delete(addedPositions.get(8)); c1.delete(addedPositions.get(9)); - assertEquals(c1.getNumberOfEntriesInBacklog(), 20 - 5); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 20 - 5); // Re-Open ManagedLedgerFactory factory2 = new ManagedLedgerFactoryImpl(bkc, bkc.getZkHandle()); ledger = factory2.open("my_test_ledger", new ManagedLedgerConfig()); c1 = ledger.openCursor("c1"); - assertEquals(c1.getNumberOfEntriesInBacklog(), 20 - 5); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 20 - 5); List entries = c1.readEntries(20); assertEquals(entries.size(), 20 - 5); @@ -2736,7 +2736,7 @@ public void testOutOfOrderDeletePersistenceIntoLedgerWithClose() throws Exceptio } } - assertEquals(c1.getNumberOfEntriesInBacklog(), totalAddEntries / 2); + assertEquals(c1.getNumberOfEntriesInBacklog(false), totalAddEntries / 2); // Close ledger to persist individual-deleted positions into cursor-ledger ledger.close(); @@ -2788,7 +2788,7 @@ public void operationFailed(MetaStoreException e) { ledger = (ManagedLedgerImpl) factory.open(ledgerName, managedLedgerConfig); c1 = (ManagedCursorImpl) ledger.openCursor("c1"); // verify cursor has been recovered - assertEquals(c1.getNumberOfEntriesInBacklog(), totalAddEntries / 2); + assertEquals(c1.getNumberOfEntriesInBacklog(false), totalAddEntries / 2); // try to read entries which should only read non-deleted positions List entries = c1.readEntries(totalAddEntries); @@ -2820,7 +2820,7 @@ public void testOutOfOrderDeletePersistenceIntoZkWithClose() throws Exception { } } - assertEquals(c1.getNumberOfEntriesInBacklog(), totalAddEntries / 2); + assertEquals(c1.getNumberOfEntriesInBacklog(false), totalAddEntries / 2); // Close ledger to persist individual-deleted positions into cursor-ledger ledger.close(); @@ -2848,7 +2848,7 @@ public void operationFailed(MetaStoreException e) { ledger = (ManagedLedgerImpl) factory.open(ledgerName, managedLedgerConfig); c1 = (ManagedCursorImpl) ledger.openCursor(cursorName); // verify cursor has been recovered - assertEquals(c1.getNumberOfEntriesInBacklog(), totalAddEntries / 2); + assertEquals(c1.getNumberOfEntriesInBacklog(false), totalAddEntries / 2); // try to read entries which should only read non-deleted positions List entries = c1.readEntries(totalAddEntries); @@ -3019,7 +3019,7 @@ public void deleteMessagesCheckhMarkDelete() throws Exception { totalDeletedMessages += 1; } } - assertEquals(c1.getNumberOfEntriesInBacklog(), totalEntries - totalDeletedMessages); + assertEquals(c1.getNumberOfEntriesInBacklog(false), totalEntries - totalDeletedMessages); assertEquals(c1.getNumberOfEntries(), totalEntries - totalDeletedMessages); assertEquals(c1.getMarkDeletedPosition(), positions[0]); assertEquals(c1.getReadPosition(), positions[1]); @@ -3033,7 +3033,7 @@ public void deleteMessagesCheckhMarkDelete() throws Exception { } } int markDelete = totalEntries / 2 - 1; - assertEquals(c1.getNumberOfEntriesInBacklog(), totalEntries - totalDeletedMessages); + assertEquals(c1.getNumberOfEntriesInBacklog(false), totalEntries - totalDeletedMessages); assertEquals(c1.getNumberOfEntries(), totalEntries - totalDeletedMessages); assertEquals(c1.getMarkDeletedPosition(), positions[markDelete]); assertEquals(c1.getReadPosition(), positions[markDelete + 1]); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java index ea69c63f2c21c..5a9e8d4bafc7b 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java @@ -124,7 +124,7 @@ public void testBookieFailure() throws Exception { // Next add should succeed ledger.addEntry("entry-2".getBytes()); - assertEquals(3, cursor.getNumberOfEntriesInBacklog()); + assertEquals(3, cursor.getNumberOfEntriesInBacklog(false)); List entries = cursor.readEntries(1); assertEquals(1, entries.size()); @@ -357,13 +357,13 @@ public void ledgerFencedByAutoReplication() throws Exception { ledger.addEntry("entry-2".getBytes()); assertEquals(2, c1.getNumberOfEntries()); - assertEquals(2, c1.getNumberOfEntriesInBacklog()); + assertEquals(2, c1.getNumberOfEntriesInBacklog(false)); PositionImpl p3 = (PositionImpl) ledger.addEntry("entry-3".getBytes()); // Now entry-2 should have been written before entry-3 assertEquals(3, c1.getNumberOfEntries()); - assertEquals(3, c1.getNumberOfEntriesInBacklog()); + assertEquals(3, c1.getNumberOfEntriesInBacklog(false)); assertTrue(p1.getLedgerId() != p3.getLedgerId()); factory.shutdown(); } @@ -402,7 +402,7 @@ public void ledgerFencedByFailover() throws Exception { // Ok } - assertEquals(2, c2.getNumberOfEntriesInBacklog()); + assertEquals(2, c2.getNumberOfEntriesInBacklog(false)); factory1.shutdown(); factory2.shutdown(); } @@ -459,12 +459,12 @@ void testResetCursorAfterRecovery() throws Exception { assertEquals(cursor.getMarkDeletedPosition(), p3); assertEquals(cursor.getReadPosition(), p4); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 1); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 1); cursor.resetCursor(p2); assertEquals(cursor.getMarkDeletedPosition(), p1); assertEquals(cursor.getReadPosition(), p2); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 3); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 3); factory2.shutdown(); factory.shutdown(); @@ -531,7 +531,7 @@ public void testChangeCrcType() throws Exception { ledger.addEntry("entry-3".getBytes()); assertEquals(c1.getNumberOfEntries(), 4); - assertEquals(c1.getNumberOfEntriesInBacklog(), 4); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 4); List entries = c1.readEntries(4); assertEquals(entries.size(), 4); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java index 416a58cc256ca..99b6bd59e91f8 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java @@ -366,7 +366,7 @@ public void recoverAfterWriteError() throws Exception { // With one single error, the write should succeed ledger.addEntry("entry-1".getBytes()); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 1); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 1); bkc.failNow(BKException.Code.BookieHandleNotAvailableException); zkc.failNow(Code.CONNECTIONLOSS); @@ -385,7 +385,7 @@ public void recoverAfterWriteError() throws Exception { // ok } - assertEquals(cursor.getNumberOfEntriesInBacklog(), 1); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 1); // Signal that ManagedLedger has recovered from write error and will be availbe for writes again ledger.readyToCreateNewLedger(); @@ -393,7 +393,7 @@ public void recoverAfterWriteError() throws Exception { // Next add should succeed, and the previous write should not appear ledger.addEntry("entry-4".getBytes()); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 2); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 2); List entries = cursor.readEntries(10); assertEquals(entries.size(), 2); @@ -435,7 +435,7 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { counter.await(); assertNull(ex.get()); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 2); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 2); // Ensure that we are only creating one new ledger // even when there are multiple (here, 2) add entry failed ops diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index 8ffd7383f563a..a31416f8dbd03 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -176,14 +176,14 @@ public void simple() throws Exception { assertFalse(cursor.hasMoreEntries()); assertEquals(cursor.getNumberOfEntries(), 0); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 0); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 0); assertEquals(cursor.readEntries(100), new ArrayList()); ledger.addEntry("dummy-entry-2".getBytes(Encoding)); assertTrue(cursor.hasMoreEntries()); assertEquals(cursor.getNumberOfEntries(), 1); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 1); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 1); assertEquals(ledger.getNumberOfActiveEntries(), 1); List entries = cursor.readEntries(100); @@ -243,7 +243,7 @@ public void acknowledge1() throws Exception { assertEquals(entries.size(), 2); assertEquals(cursor.getNumberOfEntries(), 0); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 2); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 2); assertFalse(cursor.hasMoreEntries()); assertEquals(ledger.getNumberOfEntries(), 2); @@ -252,7 +252,7 @@ public void acknowledge1() throws Exception { entries.forEach(e -> e.release()); assertEquals(cursor.getNumberOfEntries(), 0); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 1); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 1); assertFalse(cursor.hasMoreEntries()); assertEquals(ledger.getNumberOfActiveEntries(), 1); @@ -267,7 +267,7 @@ public void acknowledge1() throws Exception { assertEquals(ledger.getTotalSize(), "dummy-entry-1".getBytes(Encoding).length * 2); assertEquals(cursor.getNumberOfEntries(), 1); - assertEquals(cursor.getNumberOfEntriesInBacklog(), 1); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 1); assertTrue(cursor.hasMoreEntries()); entries = cursor.readEntries(100); @@ -2314,7 +2314,7 @@ public void testConsumerSubscriptionInitializePosition() throws Exception{ assertEquals(earliestPositionAndCounter.getLeft().getNext(), p2); assertEquals(latestPositionAndCounter.getRight().longValue(), totalInsertedEntries); - assertEquals(earliestPositionAndCounter.getRight().longValue(), totalInsertedEntries - earliestCursor.getNumberOfEntriesInBacklog()); + assertEquals(earliestPositionAndCounter.getRight().longValue(), totalInsertedEntries - earliestCursor.getNumberOfEntriesInBacklog(false)); ledger.close(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java index 957e63bfa3e6a..26cad0a02260f 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java @@ -221,31 +221,31 @@ void testNumberOfEntriesInBacklog() throws Exception { Position p4 = ledger.addEntry("dummy-entry-4".getBytes(Encoding)); ManagedCursor c5 = ledger.newNonDurableCursor(PositionImpl.latest); - assertEquals(c1.getNumberOfEntriesInBacklog(), 4); - assertEquals(c2.getNumberOfEntriesInBacklog(), 3); - assertEquals(c3.getNumberOfEntriesInBacklog(), 2); - assertEquals(c4.getNumberOfEntriesInBacklog(), 1); - assertEquals(c5.getNumberOfEntriesInBacklog(), 0); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 4); + assertEquals(c2.getNumberOfEntriesInBacklog(false), 3); + assertEquals(c3.getNumberOfEntriesInBacklog(false), 2); + assertEquals(c4.getNumberOfEntriesInBacklog(false), 1); + assertEquals(c5.getNumberOfEntriesInBacklog(false), 0); List entries = c1.readEntries(2); assertEquals(entries.size(), 2); entries.forEach(e -> e.release()); assertEquals(c1.getNumberOfEntries(), 2); - assertEquals(c1.getNumberOfEntriesInBacklog(), 4); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 4); c1.markDelete(p1); assertEquals(c1.getNumberOfEntries(), 2); - assertEquals(c1.getNumberOfEntriesInBacklog(), 3); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 3); c1.delete(p3); assertEquals(c1.getNumberOfEntries(), 1); - assertEquals(c1.getNumberOfEntriesInBacklog(), 2); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 2); c1.markDelete(p4); assertEquals(c1.getNumberOfEntries(), 0); - assertEquals(c1.getNumberOfEntriesInBacklog(), 0); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 0); } @Test(timeOut = 20000) @@ -371,34 +371,34 @@ void rewind() throws Exception { log.debug("p4: {}", p4); assertEquals(c1.getNumberOfEntries(), 4); - assertEquals(c1.getNumberOfEntriesInBacklog(), 4); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 4); c1.markDelete(p1); assertEquals(c1.getNumberOfEntries(), 3); - assertEquals(c1.getNumberOfEntriesInBacklog(), 3); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 3); List entries = c1.readEntries(10); assertEquals(entries.size(), 3); entries.forEach(e -> e.release()); assertEquals(c1.getNumberOfEntries(), 0); - assertEquals(c1.getNumberOfEntriesInBacklog(), 3); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 3); c1.rewind(); assertEquals(c1.getNumberOfEntries(), 3); - assertEquals(c1.getNumberOfEntriesInBacklog(), 3); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 3); c1.markDelete(p2); assertEquals(c1.getNumberOfEntries(), 2); - assertEquals(c1.getNumberOfEntriesInBacklog(), 2); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 2); entries = c1.readEntries(10); assertEquals(entries.size(), 2); entries.forEach(e -> e.release()); assertEquals(c1.getNumberOfEntries(), 0); - assertEquals(c1.getNumberOfEntriesInBacklog(), 2); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 2); c1.rewind(); assertEquals(c1.getNumberOfEntries(), 2); c1.markDelete(p4); assertEquals(c1.getNumberOfEntries(), 0); - assertEquals(c1.getNumberOfEntriesInBacklog(), 0); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 0); c1.rewind(); assertEquals(c1.getNumberOfEntries(), 0); ledger.addEntry("dummy-entry-5".getBytes(Encoding)); @@ -573,13 +573,13 @@ void subscribeToEarliestPositionWithDeferredDeletion() throws Exception { assertEquals(c1.getReadPosition(), p1); assertEquals(c1.getMarkDeletedPosition(), new PositionImpl(3, -1)); assertEquals(c1.getNumberOfEntries(), 6); - assertEquals(c1.getNumberOfEntriesInBacklog(), 6); + assertEquals(c1.getNumberOfEntriesInBacklog(false), 6); ManagedCursor c2 = ledger.newNonDurableCursor(p1); assertEquals(c2.getReadPosition(), p2); assertEquals(c2.getMarkDeletedPosition(), p1); assertEquals(c2.getNumberOfEntries(), 5); - assertEquals(c2.getNumberOfEntriesInBacklog(), 5); + assertEquals(c2.getNumberOfEntriesInBacklog(false), 5); } @Test diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index f015a92b66952..86da1027f34d6 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -1376,6 +1376,14 @@ public class ServiceConfiguration implements PulsarConfiguration { doc = "Classname of Pluggable JVM GC metrics logger that can log GC specific metrics") private String jvmGCMetricsLoggerClassName; + @FieldContext( + category = CATEGORY_METRICS, + doc = "Enable expose the precise backlog stats.\n" + + " Set false to use published counter and consumed counter to calculate,\n" + + " this would be more efficient but may be inaccurate. Default is false." + ) + private boolean exposePreciseBacklogInPrometheus = false; + /**** --- Functions --- ****/ @FieldContext( category = CATEGORY_FUNCTIONS, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 4a1021f9aa366..e33f94a3f744d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -808,14 +808,14 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut } } - protected TopicStats internalGetStats(boolean authoritative) { + protected TopicStats internalGetStats(boolean authoritative, boolean getPreciseBacklog) { validateAdminAndClientPermission(); if (topicName.isGlobal()) { validateGlobalNamespaceOwnership(namespaceName); } validateTopicOwnership(topicName, authoritative); Topic topic = getTopicReference(topicName); - return topic.getStats(); + return topic.getStats(getPreciseBacklog); } protected PersistentTopicInternalStats internalGetInternalStats(boolean authoritative) { @@ -850,7 +850,7 @@ public void getInfoFailed(ManagedLedgerException exception, Object ctx) { } protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean authoritative, - boolean perPartition) { + boolean perPartition, boolean getPreciseBacklog) { PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); if (partitionMetadata.partitions == 0) { throw new RestException(Status.NOT_FOUND, "Partitioned Topic not found"); @@ -864,7 +864,7 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean for (int i = 0; i < partitionMetadata.partitions; i++) { try { topicStatsFutureList - .add(pulsar().getAdminClient().topics().getStatsAsync((topicName.getPartition(i).toString()))); + .add(pulsar().getAdminClient().topics().getStatsAsync((topicName.getPartition(i).toString()), getPreciseBacklog)); } catch (PulsarServerException e) { asyncResponse.resume(new RestException(e)); return; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java index 81759cc840675..4bc0ddfd154d9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java @@ -99,7 +99,7 @@ public NonPersistentTopicStats getStats(@PathParam("property") String property, validateTopicName(property, cluster, namespace, encodedTopic); validateAdminOperationOnTopic(authoritative); Topic topic = getTopicReference(topicName); - return ((NonPersistentTopic) topic).getStats(); + return ((NonPersistentTopic) topic).getStats(false); } @GET diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java index ae139a1f0894a..836ca14282781 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java @@ -282,7 +282,7 @@ public TopicStats getStats(@PathParam("property") String property, @PathParam("c @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(property, cluster, namespace, encodedTopic); - return internalGetStats(authoritative); + return internalGetStats(authoritative, false); } @GET @@ -327,7 +327,7 @@ public void getPartitionedStats(@Suspended final AsyncResponse asyncResponse, @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { try { validateTopicName(property, cluster, namespace, encodedTopic); - internalGetPartitionedStats(asyncResponse, authoritative, perPartition); + internalGetPartitionedStats(asyncResponse, authoritative, perPartition, false); } catch (WebApplicationException wae) { asyncResponse.resume(wae); } catch (Exception e) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index 480e63ba0da8a..7e88eed2fe5ab 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -115,11 +115,13 @@ public NonPersistentTopicStats getStats( @ApiParam(value = "Specify topic name", required = true) @PathParam("topic") @Encoded String encodedTopic, @ApiParam(value = "Is authentication required to perform this operation") - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, + @ApiParam(value = "Is return precise backlog or imprecise backlog") + @QueryParam("getPreciseBacklog") @DefaultValue("false") boolean getPreciseBacklog) { validateTopicName(tenant, namespace, encodedTopic); validateAdminOperationOnTopic(topicName, authoritative); Topic topic = getTopicReference(topicName); - return ((NonPersistentTopic) topic).getStats(); + return ((NonPersistentTopic) topic).getStats(getPreciseBacklog); } @GET diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index fbf74c971cdf1..8c59fa5b6b079 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -46,8 +46,6 @@ import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; -import org.apache.pulsar.common.policies.data.PartitionedTopicInternalStats; -import org.apache.pulsar.common.policies.data.PartitionedTopicStats; import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.TopicStats; @@ -469,9 +467,11 @@ public TopicStats getStats( @ApiParam(value = "Specify topic name", required = true) @PathParam("topic") @Encoded String encodedTopic, @ApiParam(value = "Is authentication required to perform this operation") - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, + @ApiParam(value = "Is return precise backlog or imprecise backlog") + @QueryParam("getPreciseBacklog") @DefaultValue("false") boolean getPreciseBacklog) { validateTopicName(tenant, namespace, encodedTopic); - return internalGetStats(authoritative); + return internalGetStats(authoritative, getPreciseBacklog); } @GET @@ -542,10 +542,12 @@ public void getPartitionedStats( @ApiParam(value = "Get per partition stats") @QueryParam("perPartition") @DefaultValue("true") boolean perPartition, @ApiParam(value = "Is authentication required to perform this operation") - @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { + @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, + @ApiParam(value = "Is return precise backlog or imprecise backlog") + @QueryParam("getPreciseBacklog") @DefaultValue("false") boolean getPreciseBacklog) { try { validatePartitionedTopicName(tenant, namespace, encodedTopic); - internalGetPartitionedStats(asyncResponse, authoritative, perPartition); + internalGetPartitionedStats(asyncResponse, authoritative, perPartition, getPreciseBacklog); } catch (WebApplicationException wae) { asyncResponse.resume(wae); } catch (Exception e) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java index b0afc73620abc..fd47425bfaac5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java @@ -144,7 +144,7 @@ private void dropBacklog(PersistentTopic persistentTopic, BacklogQuota quota) { } // Calculate number of messages to be skipped using the current backlog and the skip factor. - long entriesInBacklog = slowestConsumer.getNumberOfEntriesInBacklog(); + long entriesInBacklog = slowestConsumer.getNumberOfEntriesInBacklog(false); int messagesToSkip = (int) (messageSkipFactor * entriesInBacklog); try { // If there are no messages to skip, break out of the loop diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index db055b69b9ddc..7d48f76b71e90 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -1407,7 +1407,7 @@ public String generateUniqueProducerName() { public Map getTopicStats() { HashMap stats = new HashMap<>(); - forEachTopic(topic -> stats.put(topic.getName(), topic.getStats())); + forEachTopic(topic -> stats.put(topic.getName(), topic.getStats(false))); return stats; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index 50a5107c05fd9..bef63fc7f5b0a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -454,7 +454,7 @@ CommandConsumerStatsResponse.Builder createConsumerStatsResponse(Consumer consum commandConsumerStatsResponseBuilder.setConnectedSince(consumerStats.getConnectedSince()); Subscription subscription = consumer.getSubscription(); - commandConsumerStatsResponseBuilder.setMsgBacklog(subscription.getNumberOfEntriesInBacklog()); + commandConsumerStatsResponseBuilder.setMsgBacklog(subscription.getNumberOfEntriesInBacklog(false)); commandConsumerStatsResponseBuilder.setMsgRateExpired(subscription.getExpiredMessageRate()); commandConsumerStatsResponseBuilder.setType(subscription.getTypeString()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java index 18c7c49ae37f5..f7d9687523960 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java @@ -53,7 +53,7 @@ default void removeConsumer(Consumer consumer) throws BrokerServiceException { Dispatcher getDispatcher(); - long getNumberOfEntriesInBacklog(); + long getNumberOfEntriesInBacklog(boolean getPreciseBacklog); default long getNumberOfEntriesDelayed() { return 0; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java index 26af1c1c5c8bf..6b3685deba927 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java @@ -162,7 +162,7 @@ void updateRates(NamespaceStats nsStats, NamespaceBundleStats currentBundleStats ConcurrentOpenHashMap getReplicators(); - TopicStats getStats(); + TopicStats getStats(boolean getPreciseBacklog); PersistentTopicInternalStats getInternalStats(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java index 652042000b6ac..f653ee52b4953 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java @@ -250,7 +250,7 @@ public CompletableFuture peekNthMessage(int messagePosition) { } @Override - public long getNumberOfEntriesInBacklog() { + public long getNumberOfEntriesInBacklog(boolean getPreciseBacklog) { // No-op return 0; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java index 8e3b14fa7f43a..27dc4bb3111bd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java @@ -697,7 +697,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats topicStatsStream.endList(); // Populate subscription specific stats here - topicStatsStream.writePair("msgBacklog", subscription.getNumberOfEntriesInBacklog()); + topicStatsStream.writePair("msgBacklog", subscription.getNumberOfEntriesInBacklog(false)); topicStatsStream.writePair("msgRateExpired", subscription.getExpiredMessageRate()); topicStatsStream.writePair("msgRateOut", subMsgRateOut); topicStatsStream.writePair("msgThroughputOut", subMsgThroughputOut); @@ -714,7 +714,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats topicStats.aggMsgRateOut += subMsgRateOut; topicStats.aggMsgThroughputOut += subMsgThroughputOut; - nsStats.msgBacklog += subscription.getNumberOfEntriesInBacklog(); + nsStats.msgBacklog += subscription.getNumberOfEntriesInBacklog(false); } catch (Exception e) { log.error("Got exception when creating consumer stats for subscription {}: {}", subscriptionName, e.getMessage(), e); @@ -751,7 +751,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats topicStatsStream.endObject(); } - public NonPersistentTopicStats getStats() { + public NonPersistentTopicStats getStats(boolean getPreciseBacklog) { NonPersistentTopicStats stats = new NonPersistentTopicStats(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java index 0c3feb31bfd86..5d39c20a2441d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java @@ -89,7 +89,7 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { } }, null); - if (topic.getManagedLedger().isTerminated() && cursor.getNumberOfEntriesInBacklog() == 0) { + if (topic.getManagedLedger().isTerminated() && cursor.getNumberOfEntriesInBacklog(false) == 0) { // Notify all consumer that the end of topic was reached dispatcher.getConsumers().forEach(Consumer::reachedEndOfTopic); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 9c6b1efa9810f..9457af38d398a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -540,7 +540,7 @@ public synchronized void readEntriesFailed(ManagedLedgerException exception, Obj long waitTimeMillis = readFailureBackoff.next(); if (exception instanceof NoMoreEntriesToReadException) { - if (cursor.getNumberOfEntriesInBacklog() == 0) { + if (cursor.getNumberOfEntriesInBacklog(false) == 0) { // Topic has been terminated and there are no more entries to read // Notify the consumer only if all the messages were already acknowledged consumerList.forEach(Consumer::reachedEndOfTopic); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java index c29fbc85b7879..fe73760171286 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java @@ -470,7 +470,7 @@ private synchronized void internalReadEntriesFailed(ManagedLedgerException excep long waitTimeMillis = readFailureBackoff.next(); if (exception instanceof NoMoreEntriesToReadException) { - if (cursor.getNumberOfEntriesInBacklog() == 0) { + if (cursor.getNumberOfEntriesInBacklog(false) == 0) { // Topic has been terminated and there are no more entries to read // Notify the consumer only if all the messages were already acknowledged consumers.forEach(Consumer::reachedEndOfTopic); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java index deb274457bf92..1b761d8fb7af6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java @@ -99,7 +99,7 @@ public double getMessageExpiryRate() { private final MarkDeleteCallback markDeleteCallback = new MarkDeleteCallback() { @Override public void markDeleteComplete(Object ctx) { - long numMessagesExpired = (long) ctx - cursor.getNumberOfEntriesInBacklog(); + long numMessagesExpired = (long) ctx - cursor.getNumberOfEntriesInBacklog(false); msgExpired.recordMultipleEvents(numMessagesExpired, 0 /* no value stats */); updateRates(); @@ -119,7 +119,7 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { public void findEntryComplete(Position position, Object ctx) { if (position != null) { log.info("[{}][{}] Expiring all messages until position {}", topicName, subName, position); - cursor.asyncMarkDelete(position, markDeleteCallback, cursor.getNumberOfEntriesInBacklog()); + cursor.asyncMarkDelete(position, markDeleteCallback, cursor.getNumberOfEntriesInBacklog(false)); } else { if (log.isDebugEnabled()) { log.debug("[{}][{}] No messages to expire", topicName, subName); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index 2bc18ca384709..c4a19c460b529 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -149,7 +149,7 @@ protected Position getReplicatorReadPosition() { @Override protected long getNumberOfEntriesInBacklog() { - return cursor.getNumberOfEntriesInBacklog(); + return cursor.getNumberOfEntriesInBacklog(false); } @Override @@ -507,7 +507,7 @@ public CompletableFuture clearBacklog() { if (log.isDebugEnabled()) { log.debug("[{}][{} -> {}] Backlog size before clearing: {}", topicName, localCluster, remoteCluster, - cursor.getNumberOfEntriesInBacklog()); + cursor.getNumberOfEntriesInBacklog(false)); } cursor.asyncClearBacklog(new ClearBacklogCallback() { @@ -515,7 +515,7 @@ public CompletableFuture clearBacklog() { public void clearBacklogComplete(Object ctx) { if (log.isDebugEnabled()) { log.debug("[{}][{} -> {}] Backlog size after clearing: {}", topicName, localCluster, remoteCluster, - cursor.getNumberOfEntriesInBacklog()); + cursor.getNumberOfEntriesInBacklog(false)); } future.complete(null); } @@ -535,7 +535,7 @@ public CompletableFuture skipMessages(int numMessagesToSkip) { if (log.isDebugEnabled()) { log.debug("[{}][{} -> {}] Skipping {} messages, current backlog {}", topicName, localCluster, remoteCluster, - numMessagesToSkip, cursor.getNumberOfEntriesInBacklog()); + numMessagesToSkip, cursor.getNumberOfEntriesInBacklog(false)); } cursor.asyncSkipEntries(numMessagesToSkip, IndividualDeletedEntries.Exclude, new AsyncCallbacks.SkipEntriesCallback() { @@ -543,7 +543,7 @@ public CompletableFuture skipMessages(int numMessagesToSkip) { public void skipEntriesComplete(Object ctx) { if (log.isDebugEnabled()) { log.debug("[{}][{} -> {}] Skipped {} messages, new backlog {}", topicName, localCluster, - remoteCluster, numMessagesToSkip, cursor.getNumberOfEntriesInBacklog()); + remoteCluster, numMessagesToSkip, cursor.getNumberOfEntriesInBacklog(false)); } future.complete(null); } @@ -605,7 +605,7 @@ public void updateRates() { } public ReplicatorStats getStats() { - stats.replicationBacklog = cursor.getNumberOfEntriesInBacklog(); + stats.replicationBacklog = cursor.getNumberOfEntriesInBacklog(false); stats.connected = producer != null && producer.isConnected(); stats.replicationDelayInSeconds = getReplicationDelayInSeconds(); @@ -633,8 +633,8 @@ private long getReplicationDelayInSeconds() { } public void expireMessages(int messageTTLInSeconds) { - if ((cursor.getNumberOfEntriesInBacklog() == 0) - || (cursor.getNumberOfEntriesInBacklog() < MINIMUM_BACKLOG_FOR_EXPIRY_CHECK + if ((cursor.getNumberOfEntriesInBacklog(false) == 0) + || (cursor.getNumberOfEntriesInBacklog(false) < MINIMUM_BACKLOG_FOR_EXPIRY_CHECK && !topic.isOldestMessageExpired(cursor, messageTTLInSeconds))) { // don't do anything for almost caught-up connected subscriptions return; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 88d8e45c8b3cc..7055ac0071e62 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.stream.Collectors; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.ClearBacklogCallback; @@ -391,7 +392,7 @@ public void acknowledgeMessage(List positions, AckType ackType, Map clearBacklog() { if (log.isDebugEnabled()) { log.debug("[{}][{}] Backlog size before clearing: {}", topicName, subName, - cursor.getNumberOfEntriesInBacklog()); + cursor.getNumberOfEntriesInBacklog(false)); } cursor.asyncClearBacklog(new ClearBacklogCallback() { @@ -581,7 +582,7 @@ public CompletableFuture clearBacklog() { public void clearBacklogComplete(Object ctx) { if (log.isDebugEnabled()) { log.debug("[{}][{}] Backlog size after clearing: {}", topicName, subName, - cursor.getNumberOfEntriesInBacklog()); + cursor.getNumberOfEntriesInBacklog(false)); } future.complete(null); } @@ -602,7 +603,7 @@ public CompletableFuture skipMessages(int numMessagesToSkip) { if (log.isDebugEnabled()) { log.debug("[{}][{}] Skipping {} messages, current backlog {}", topicName, subName, numMessagesToSkip, - cursor.getNumberOfEntriesInBacklog()); + cursor.getNumberOfEntriesInBacklog(false)); } cursor.asyncSkipEntries(numMessagesToSkip, IndividualDeletedEntries.Exclude, new AsyncCallbacks.SkipEntriesCallback() { @@ -610,7 +611,7 @@ public CompletableFuture skipMessages(int numMessagesToSkip) { public void skipEntriesComplete(Object ctx) { if (log.isDebugEnabled()) { log.debug("[{}][{}] Skipped {} messages, new backlog {}", topicName, subName, - numMessagesToSkip, cursor.getNumberOfEntriesInBacklog()); + numMessagesToSkip, cursor.getNumberOfEntriesInBacklog(false)); } future.complete(null); } @@ -776,8 +777,8 @@ public void readEntryComplete(Entry entry, Object ctx) { } @Override - public long getNumberOfEntriesInBacklog() { - return cursor.getNumberOfEntriesInBacklog(); + public long getNumberOfEntriesInBacklog(boolean getPreciseBacklog) { + return cursor.getNumberOfEntriesInBacklog(getPreciseBacklog); } @Override @@ -917,8 +918,8 @@ public List getConsumers() { @Override public void expireMessages(int messageTTLInSeconds) { this.lastExpireTimestamp = System.currentTimeMillis(); - if ((getNumberOfEntriesInBacklog() == 0) || (dispatcher != null && dispatcher.isConsumerConnected() - && getNumberOfEntriesInBacklog() < MINIMUM_BACKLOG_FOR_EXPIRY_CHECK + if ((getNumberOfEntriesInBacklog(false) == 0) || (dispatcher != null && dispatcher.isConsumerConnected() + && getNumberOfEntriesInBacklog(false) < MINIMUM_BACKLOG_FOR_EXPIRY_CHECK && !topic.isOldestMessageExpired(cursor, messageTTLInSeconds))) { // don't do anything for almost caught-up connected subscriptions return; @@ -934,7 +935,7 @@ public long estimateBacklogSize() { return cursor.getEstimatedSizeSinceMarkDeletePosition(); } - public SubscriptionStats getStats() { + public SubscriptionStats getStats(Boolean getPreciseBacklog) { SubscriptionStats subStats = new SubscriptionStats(); subStats.lastExpireTimestamp = lastExpireTimestamp; subStats.lastConsumedFlowTimestamp = lastConsumedFlowTimestamp; @@ -967,7 +968,8 @@ public SubscriptionStats getStats() { subStats.msgDelayed = d.getNumberOfDelayedMessages(); } } - subStats.msgBacklog = getNumberOfEntriesInBacklog(); + subStats.msgBacklog = getNumberOfEntriesInBacklog(getPreciseBacklog); + subStats.msgBacklogNoDelayed = subStats.msgBacklog - subStats.msgDelayed; subStats.msgRateExpired = expiryMonitor.getMessageExpiryRate(); subStats.isReplicated = isReplicated(); return subStats; @@ -1047,7 +1049,7 @@ public void markTopicWithBatchMessagePublished() { } void topicTerminated() { - if (cursor.getNumberOfEntriesInBacklog() == 0) { + if (cursor.getNumberOfEntriesInBacklog(false) == 0) { // notify the consumers if there are consumers connected to this topic. if (null != dispatcher) { // Immediately notify the consumer that there are no more available messages @@ -1197,5 +1199,10 @@ public void processReplicatedSubscriptionSnapshot(ReplicatedSubscriptionsSnapsho } } + @VisibleForTesting + public ManagedCursor getCursor() { + return cursor; + } + private static final Logger log = LoggerFactory.getLogger(PersistentSubscription.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index f624ae8011268..335be8cf3a847 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -1435,7 +1435,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats topicStatsStream.endList(); // Populate subscription specific stats here - topicStatsStream.writePair("msgBacklog", subscription.getNumberOfEntriesInBacklog()); + topicStatsStream.writePair("msgBacklog", subscription.getNumberOfEntriesInBacklog(false)); topicStatsStream.writePair("msgRateExpired", subscription.getExpiredMessageRate()); topicStatsStream.writePair("msgRateOut", subMsgRateOut); topicStatsStream.writePair("msgThroughputOut", subMsgThroughputOut); @@ -1456,7 +1456,7 @@ public void updateRates(NamespaceStats nsStats, NamespaceBundleStats bundleStats topicStatsHelper.aggMsgRateOut += subMsgRateOut; topicStatsHelper.aggMsgThroughputOut += subMsgThroughputOut; - nsStats.msgBacklog += subscription.getNumberOfEntriesInBacklog(); + nsStats.msgBacklog += subscription.getNumberOfEntriesInBacklog(false); } catch (Exception e) { log.error("Got exception when creating consumer stats for subscription {}: {}", subscriptionName, e.getMessage(), e); @@ -1508,7 +1508,7 @@ public double getLastUpdatedAvgPublishRateInByte() { return lastUpdatedAvgPublishRateInByte; } - public TopicStats getStats() { + public TopicStats getStats(boolean getPreciseBacklog) { TopicStats stats = new TopicStats(); @@ -1531,7 +1531,7 @@ public TopicStats getStats() { stats.bytesInCounter = getBytesInCounter(); subscriptions.forEach((name, subscription) -> { - SubscriptionStats subStats = subscription.getStats(); + SubscriptionStats subStats = subscription.getStats(getPreciseBacklog); stats.msgRateOut += subStats.msgRateOut; stats.msgThroughputOut += subStats.msgThroughputOut; @@ -1641,7 +1641,7 @@ public boolean isActive(InactiveTopicDeleteMode deleteMode) { } private boolean hasBacklogs() { - return subscriptions.values().stream().anyMatch(sub -> sub.getNumberOfEntriesInBacklog() > 0); + return subscriptions.values().stream().anyMatch(sub -> sub.getNumberOfEntriesInBacklog(false) > 0); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java index cc5388e8ced02..ea05ed074c09e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java @@ -93,6 +93,7 @@ void updateStats(TopicStats stats) { msgDelayed += as.msgDelayed; subsStats.blockedSubscriptionOnUnackedMsgs = as.blockedSubscriptionOnUnackedMsgs; subsStats.msgBacklog += as.msgBacklog; + subsStats.msgBacklogNoDelayed += as.msgBacklogNoDelayed; subsStats.msgDelayed += as.msgDelayed; subsStats.msgRateRedeliver += as.msgRateRedeliver; subsStats.unackedMessages += as.unackedMessages; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedSubscriptionStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedSubscriptionStats.java index 1f3c51302e980..d5b53537dcc9a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedSubscriptionStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedSubscriptionStats.java @@ -27,6 +27,8 @@ public class AggregatedSubscriptionStats { public long msgBacklog; + public long msgBacklogNoDelayed; + public boolean blockedSubscriptionOnUnackedMsgs; public double msgRateRedeliver; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java index 19aa043ae823a..a7b35b8c63337 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java @@ -61,7 +61,7 @@ public static void generate(PulsarService pulsar, boolean includeTopicMetrics, b bundlesMap.forEach((bundle, topicsMap) -> { topicsMap.forEach((name, topic) -> { - getTopicStats(topic, topicStats, includeConsumerMetrics); + getTopicStats(topic, topicStats, includeConsumerMetrics, pulsar.getConfiguration().isExposePreciseBacklogInPrometheus()); if (includeTopicMetrics) { topicsCount.add(1); @@ -82,7 +82,7 @@ public static void generate(PulsarService pulsar, boolean includeTopicMetrics, b }); } - private static void getTopicStats(Topic topic, TopicStats stats, boolean includeConsumerMetrics) { + private static void getTopicStats(Topic topic, TopicStats stats, boolean includeConsumerMetrics, boolean getPreciseBacklog) { stats.reset(); if (topic instanceof PersistentTopic) { @@ -104,8 +104,8 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include stats.storageReadRate = mlStats.getReadEntriesRate(); } - stats.msgInCounter = topic.getStats().msgInCounter; - stats.bytesInCounter = topic.getStats().bytesInCounter; + stats.msgInCounter = topic.getStats(getPreciseBacklog).msgInCounter; + stats.bytesInCounter = topic.getStats(getPreciseBacklog).bytesInCounter; stats.producersCount = 0; topic.getProducers().values().forEach(producer -> { @@ -125,12 +125,13 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include topic.getSubscriptions().forEach((name, subscription) -> { stats.subscriptionsCount++; - stats.msgBacklog += subscription.getNumberOfEntriesInBacklog(); + stats.msgBacklog += subscription.getNumberOfEntriesInBacklog(getPreciseBacklog); AggregatedSubscriptionStats subsStats = stats.subscriptionStats .computeIfAbsent(name, k -> new AggregatedSubscriptionStats()); - subsStats.msgBacklog = subscription.getNumberOfEntriesInBacklog(); + subsStats.msgBacklog = subscription.getNumberOfEntriesInBacklog(getPreciseBacklog); subsStats.msgDelayed = subscription.getNumberOfEntriesDelayed(); + subsStats.msgBacklogNoDelayed = subsStats.msgBacklog - subsStats.msgDelayed; subscription.getConsumers().forEach(consumer -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java index 846104441fac3..caf0ce842e2b3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/TopicStats.java @@ -134,6 +134,7 @@ static void printTopicStats(SimpleTextOutputStream stream, String cluster, Strin stats.subscriptionStats.forEach((n, subsStats) -> { metric(stream, cluster, namespace, topic, n, "pulsar_subscription_back_log", subsStats.msgBacklog); + metric(stream, cluster, namespace, topic, n, "pulsar_subscription_back_log_no_delayed", subsStats.msgBacklogNoDelayed); metric(stream, cluster, namespace, topic, n, "pulsar_subscription_delayed", subsStats.msgDelayed); metric(stream, cluster, namespace, topic, n, "pulsar_subscription_msg_rate_redeliver", subsStats.msgRateRedeliver); metric(stream, cluster, namespace, topic, n, "pulsar_subscription_unacked_messages", subsStats.unackedMessages); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java index fb61825ff86e3..558ce39d09f27 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest2.java @@ -53,12 +53,14 @@ import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.PulsarAdminException.PreconditionFailedException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; @@ -1062,4 +1064,185 @@ public void testConsumerStatsLastTimestamp() throws PulsarClientException, Pulsa consumer.close(); producer.close(); } + + @Test(timeOut = 30000) + public void testPreciseBacklog() throws PulsarClientException, PulsarAdminException, InterruptedException { + final String topic = "persistent://prop-xyz/ns1/precise-back-log"; + final String subName = "sub-name"; + + @Cleanup + PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getWebServiceAddress()).build(); + + @Cleanup + Consumer consumer = client.newConsumer() + .topic(topic) + .subscriptionName(subName) + .subscribe(); + + @Cleanup + Producer producer = client.newProducer() + .topic(topic) + .enableBatching(false) + .create(); + + producer.send("message-1".getBytes(StandardCharsets.UTF_8)); + Message message = consumer.receive(); + assertNotNull(message); + + // Mock the entries added count. Default is disable the precise backlog, so the backlog is entries added count - consumed count + // Since message have not acked, so the backlog is 10 + PersistentSubscription subscription = (PersistentSubscription)pulsar.getBrokerService().getTopicReference(topic).get().getSubscription(subName); + assertNotNull(subscription); + ((ManagedLedgerImpl)subscription.getCursor().getManagedLedger()).setEntriesAddedCounter(10L); + TopicStats topicStats = admin.topics().getStats(topic); + assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 10); + + topicStats = admin.topics().getStats(topic, true); + assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 1); + consumer.acknowledge(message); + + // wait for ack send + Thread.sleep(500); + + // Consumer acks the message, so the precise backlog is 0 + topicStats = admin.topics().getStats(topic, true); + assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 0); + + topicStats = admin.topics().getStats(topic); + assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 9); + } + + @Test(timeOut = 30000) + public void testBacklogNoDelayed() throws PulsarClientException, PulsarAdminException, InterruptedException { + final String topic = "persistent://prop-xyz/ns1/precise-back-log-no-delayed"; + final String subName = "sub-name"; + + @Cleanup + PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getWebServiceAddress()).build(); + + @Cleanup + Consumer consumer = client.newConsumer() + .topic(topic) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + + @Cleanup + Producer producer = client.newProducer() + .topic(topic) + .enableBatching(false) + .create(); + + for (int i = 0; i < 10; i++) { + if (i > 4) { + producer.newMessage() + .value("message-1".getBytes(StandardCharsets.UTF_8)) + .deliverAfter(10, TimeUnit.SECONDS) + .send(); + } else { + producer.send("message-1".getBytes(StandardCharsets.UTF_8)); + } + } + + TopicStats topicStats = admin.topics().getStats(topic, true); + assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 10); + assertEquals(topicStats.subscriptions.get(subName).msgBacklogNoDelayed, 5); + + for (int i = 0; i < 5; i++) { + consumer.acknowledge(consumer.receive()); + } + // Wait the ack send. + Thread.sleep(500); + topicStats = admin.topics().getStats(topic, true); + assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 5); + assertEquals(topicStats.subscriptions.get(subName).msgBacklogNoDelayed, 0); + } + + @Test + public void testPreciseBacklogForPartitionedTopic() throws PulsarClientException, PulsarAdminException, InterruptedException { + final String topic = "persistent://prop-xyz/ns1/precise-back-log-for-partitioned-topic"; + admin.topics().createPartitionedTopic(topic, 2); + final String subName = "sub-name"; + + @Cleanup + PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getWebServiceAddress()).build(); + + @Cleanup + Consumer consumer = client.newConsumer() + .topic(topic) + .subscriptionName(subName) + .subscribe(); + + @Cleanup + Producer producer = client.newProducer() + .topic(topic) + .enableBatching(false) + .create(); + + producer.send("message-1".getBytes(StandardCharsets.UTF_8)); + Message message = consumer.receive(); + assertNotNull(message); + + // Mock the entries added count. Default is disable the precise backlog, so the backlog is entries added count - consumed count + // Since message have not acked, so the backlog is 10 + for (int i = 0; i < 2; i++) { + PersistentSubscription subscription = (PersistentSubscription)pulsar.getBrokerService().getTopicReference(topic + "-partition-" + i).get().getSubscription(subName); + assertNotNull(subscription); + ((ManagedLedgerImpl)subscription.getCursor().getManagedLedger()).setEntriesAddedCounter(10L); + } + + TopicStats topicStats = admin.topics().getPartitionedStats(topic, false); + assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 20); + + topicStats = admin.topics().getPartitionedStats(topic, false, true); + assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 1); + } + + @Test(timeOut = 30000) + public void testBacklogNoDelayedForPartitionedTopic() throws PulsarClientException, PulsarAdminException, InterruptedException { + final String topic = "persistent://prop-xyz/ns1/precise-back-log-no-delayed-partitioned-topic"; + admin.topics().createPartitionedTopic(topic, 2); + final String subName = "sub-name"; + + @Cleanup + PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getWebServiceAddress()).build(); + + @Cleanup + Consumer consumer = client.newConsumer() + .topic(topic) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + + @Cleanup + Producer producer = client.newProducer() + .topic(topic) + .enableBatching(false) + .create(); + + for (int i = 0; i < 10; i++) { + if (i > 4) { + producer.newMessage() + .value("message-1".getBytes(StandardCharsets.UTF_8)) + .deliverAfter(10, TimeUnit.SECONDS) + .send(); + } else { + producer.send("message-1".getBytes(StandardCharsets.UTF_8)); + } + } + + TopicStats topicStats = admin.topics().getPartitionedStats(topic, false, true); + assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 10); + assertEquals(topicStats.subscriptions.get(subName).msgBacklogNoDelayed, 5); + + for (int i = 0; i < 5; i++) { + consumer.acknowledge(consumer.receive()); + } + // Wait the ack send. + Thread.sleep(500); + topicStats = admin.topics().getPartitionedStats(topic, false, true); + assertEquals(topicStats.subscriptions.get(subName).msgBacklog, 5); + assertEquals(topicStats.subscriptions.get(subName).msgBacklogNoDelayed, 0); + } + } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java index 7dd152827ad4d..3d5f95603b057 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BatchMessageTest.java @@ -121,7 +121,7 @@ public void testSimpleBatchProducerWithFixedBatchSize(CompressionType compressio assertTrue(topic.getProducers().values().iterator().next().getStats().msgRateIn > 0.0); // we expect 2 messages in the backlog since we sent 50 messages with the batch size set to 25. We have set the // batch time high enough for it to not affect the number of messages in the batch - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), 2); + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), 2); consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName(subscriptionName).subscribe(); for (int i = 0; i < numMsgs; i++) { @@ -170,7 +170,7 @@ public void testSimpleBatchProducerWithFixedBatchBytes(CompressionType compressi assertTrue(topic.getProducers().values().iterator().next().getStats().msgRateIn > 0.0); // we expect 2 messages in the backlog since we sent 50 messages with the batch size set to 25. We have set the // batch time high enough for it to not affect the number of messages in the batch - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), 2); + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), 2); consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName(subscriptionName).subscribe(); for (int i = 0; i < numMsgs; i++) { @@ -215,8 +215,8 @@ public void testSimpleBatchProducerWithFixedBatchTime(CompressionType compressio rolloverPerIntervalStats(); assertTrue(topic.getProducers().values().iterator().next().getStats().msgRateIn > 0.0); LOG.info("Sent {} messages, backlog is {} messages", numMsgs, - topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog()); - assertTrue(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog() < numMsgs); + topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false)); + assertTrue(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false) < numMsgs); producer.close(); } @@ -251,8 +251,8 @@ public void testSimpleBatchProducerWithFixedBatchSizeAndTime(CompressionType com rolloverPerIntervalStats(); assertTrue(topic.getProducers().values().iterator().next().getStats().msgRateIn > 0.0); LOG.info("Sent {} messages, backlog is {} messages", numMsgs, - topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog()); - assertTrue(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog() < numMsgs); + topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false)); + assertTrue(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false) < numMsgs); producer.close(); } @@ -298,7 +298,7 @@ public void testBatchProducerWithLargeMessage(CompressionType compressionType, B assertTrue(topic.getProducers().values().iterator().next().getStats().msgRateIn > 0.0); // we expect 3 messages in the backlog since the large message in the middle should // close out the batch and be sent in a batch of its own - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), 3); + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), 3); consumer = pulsarClient.newConsumer() .topic(topicName) .subscriptionName(subscriptionName) @@ -312,7 +312,7 @@ public void testBatchProducerWithLargeMessage(CompressionType compressionType, B consumer.acknowledge(msg); } Thread.sleep(100); - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), 0); + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), 0); consumer.close(); producer.close(); } @@ -354,7 +354,7 @@ public void testSimpleBatchProducerConsumer(CompressionType compressionType, Bat rolloverPerIntervalStats(); assertTrue(topic.getProducers().values().iterator().next().getStats().msgRateIn > 0.0); - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), numMsgs / numMsgsInBatch); + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), numMsgs / numMsgsInBatch); consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName(subscriptionName).subscribe(); Message lastunackedMsg = null; @@ -371,7 +371,7 @@ public void testSimpleBatchProducerConsumer(CompressionType compressionType, Bat consumer.acknowledgeCumulative(lastunackedMsg); } Thread.sleep(100); - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), 0); + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), 0); consumer.close(); producer.close(); } @@ -403,7 +403,7 @@ public void testSimpleBatchSyncProducerWithFixedBatchSize(BatcherBuilder builder assertTrue(topic.getProducers().values().iterator().next().getStats().msgRateIn > 0.0); // we expect 10 messages in the backlog since we sent 10 messages with the batch size set to 5. // However, we are using synchronous send and so each message will go as an individual message - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), 10); + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), 10); consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName(subscriptionName).subscribe(); for (int i = 0; i < numMsgs; i++) { @@ -458,7 +458,7 @@ public void testSimpleBatchProducerConsumer1kMessages(BatcherBuilder builder) th // allow stats to be updated.. LOG.info("[{}] checking backlog stats.."); rolloverPerIntervalStats(); - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), numMsgs / numMsgsInBatch); + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), numMsgs / numMsgsInBatch); consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName(subscriptionName).subscribe(); Message lastunackedMsg = null; @@ -473,7 +473,7 @@ public void testSimpleBatchProducerConsumer1kMessages(BatcherBuilder builder) th consumer.close(); producer.close(); - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), 0); + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), 0); } // test for ack holes @@ -507,7 +507,7 @@ public void testOutOfOrderAcksForBatchMessage() throws Exception { PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); rolloverPerIntervalStats(); - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), numMsgs / numMsgsInBatch); + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), numMsgs / numMsgsInBatch); consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName(subscriptionName).subscribe(); Set individualAcks = new HashSet<>(); for (int i = 15; i < 20; i++) { @@ -528,7 +528,7 @@ public void testOutOfOrderAcksForBatchMessage() throws Exception { Thread.sleep(1000); rolloverPerIntervalStats(); Thread.sleep(1000); - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), 3); + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), 3); } else if (individualAcks.contains(i)) { consumer.acknowledge(msg); } else { @@ -537,12 +537,12 @@ public void testOutOfOrderAcksForBatchMessage() throws Exception { } Thread.sleep(1000); rolloverPerIntervalStats(); - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), 2); + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), 2); if (lastunackedMsg != null) { consumer.acknowledgeCumulative(lastunackedMsg); } Thread.sleep(100); - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), 0); + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), 0); consumer.close(); producer.close(); } @@ -580,7 +580,7 @@ public void testNonBatchCumulativeAckAfterBatchPublish(BatcherBuilder builder) t rolloverPerIntervalStats(); assertTrue(topic.getProducers().values().iterator().next().getStats().msgRateIn > 0.0); - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), 2); + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), 2); consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName(subscriptionName).subscribe(); Message lastunackedMsg = null; @@ -594,7 +594,7 @@ public void testNonBatchCumulativeAckAfterBatchPublish(BatcherBuilder builder) t } Thread.sleep(100); rolloverPerIntervalStats(); - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), 0); + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), 0); consumer.close(); producer.close(); noBatchProducer.close(); @@ -637,7 +637,7 @@ public void testBatchAndNonBatchCumulativeAcks(BatcherBuilder builder) throws Ex rolloverPerIntervalStats(); assertTrue(topic.getProducers().values().iterator().next().getStats().msgRateIn > 0.0); - assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(), + assertEquals(topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false), (numMsgs / 2) / numMsgsInBatch + numMsgs / 2); consumer = pulsarClient.newConsumer() .topic(topicName) @@ -662,7 +662,7 @@ public void testBatchAndNonBatchCumulativeAcks(BatcherBuilder builder) throws Ex consumer.acknowledgeCumulative(lastunackedMsg); } - retryStrategically(t -> topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog() == 0, 100, 100); + retryStrategically(t -> topic.getSubscription(subscriptionName).getNumberOfEntriesInBacklog(false) == 0, 100, 100); consumer.close(); producer.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index 505038bede387..a2f27822d0008 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java @@ -153,7 +153,7 @@ public void testBrokerServicePersistentTopicStats() throws Exception { assertNotNull(topicRef); rolloverPerIntervalStats(); - stats = topicRef.getStats(); + stats = topicRef.getStats(false); subStats = stats.subscriptions.values().iterator().next(); // subscription stats @@ -171,7 +171,7 @@ public void testBrokerServicePersistentTopicStats() throws Exception { Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); rolloverPerIntervalStats(); - stats = topicRef.getStats(); + stats = topicRef.getStats(false); subStats = stats.subscriptions.values().iterator().next(); // publisher stats @@ -208,7 +208,7 @@ public void testBrokerServicePersistentTopicStats() throws Exception { Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); rolloverPerIntervalStats(); - stats = topicRef.getStats(); + stats = topicRef.getStats(false); subStats = stats.subscriptions.values().iterator().next(); assertEquals(subStats.msgBacklog, 0); @@ -221,13 +221,13 @@ public void testStatsOfStorageSizeWithSubscription() throws Exception { PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); assertNotNull(topicRef); - assertEquals(topicRef.getStats().storageSize, 0); + assertEquals(topicRef.getStats(false).storageSize, 0); for (int i = 0; i < 10; i++) { producer.send(new byte[10]); } - assertTrue(topicRef.getStats().storageSize > 0); + assertTrue(topicRef.getStats(false).storageSize > 0); } @Test @@ -246,7 +246,7 @@ public void testBrokerServicePersistentRedeliverTopicStats() throws Exception { assertNotNull(topicRef); rolloverPerIntervalStats(); - stats = topicRef.getStats(); + stats = topicRef.getStats(false); subStats = stats.subscriptions.values().iterator().next(); // subscription stats @@ -264,7 +264,7 @@ public void testBrokerServicePersistentRedeliverTopicStats() throws Exception { Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); rolloverPerIntervalStats(); - stats = topicRef.getStats(); + stats = topicRef.getStats(false); subStats = stats.subscriptions.values().iterator().next(); // publisher stats @@ -299,7 +299,7 @@ public void testBrokerServicePersistentRedeliverTopicStats() throws Exception { Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); rolloverPerIntervalStats(); - stats = topicRef.getStats(); + stats = topicRef.getStats(false); subStats = stats.subscriptions.values().iterator().next(); assertTrue(subStats.msgRateRedeliver > 0.0); assertEquals(subStats.msgRateRedeliver, subStats.consumers.get(0).msgRateRedeliver); @@ -313,7 +313,7 @@ public void testBrokerServicePersistentRedeliverTopicStats() throws Exception { Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); rolloverPerIntervalStats(); - stats = topicRef.getStats(); + stats = topicRef.getStats(false); subStats = stats.subscriptions.values().iterator().next(); assertEquals(subStats.msgBacklog, 0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java index 21f3c03fa6097..eccaea6237244 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java @@ -187,7 +187,7 @@ public void testSimpleConsumerEventsWithoutPartition() throws Exception { rolloverPerIntervalStats(); - assertEquals(subRef.getNumberOfEntriesInBacklog(), numMsgs); + assertEquals(subRef.getNumberOfEntriesInBacklog(false), numMsgs); Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); // 3. consumer1 should have all the messages while consumer2 should have no messages @@ -204,7 +204,7 @@ public void testSimpleConsumerEventsWithoutPartition() throws Exception { // 4. messages deleted on individual acks Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); - assertEquals(subRef.getNumberOfEntriesInBacklog(), 0); + assertEquals(subRef.getNumberOfEntriesInBacklog(false), 0); for (int i = 0; i < numMsgs; i++) { String message = "my-message-" + i; @@ -241,7 +241,7 @@ public void testSimpleConsumerEventsWithoutPartition() throws Exception { rolloverPerIntervalStats(); Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); - assertEquals(subRef.getNumberOfEntriesInBacklog(), 0); + assertEquals(subRef.getNumberOfEntriesInBacklog(false), 0); // 8. unsubscribe not allowed if multiple consumers connected try { @@ -517,7 +517,7 @@ public void testActiveConsumerFailoverWithDelay() throws Exception { // wait for all messages to be dequeued int retry = 20; for (int i = 0; i < retry; i++) { - if (receivedMessages.size() >= numMsgs && subRef.getNumberOfEntriesInBacklog() == 0) { + if (receivedMessages.size() >= numMsgs && subRef.getNumberOfEntriesInBacklog(false) == 0) { break; } else if (i != retry - 1) { Thread.sleep(100); @@ -526,7 +526,7 @@ public void testActiveConsumerFailoverWithDelay() throws Exception { // check if message duplication has occurred assertEquals(receivedMessages.size(), numMsgs); - assertEquals(subRef.getNumberOfEntriesInBacklog(), 0); + assertEquals(subRef.getNumberOfEntriesInBacklog(false), 0); for (int i = 0; i < receivedMessages.size(); i++) { Assert.assertNotNull(receivedMessages.get(i)); Assert.assertEquals(new String(receivedMessages.get(i).getData()), "my-message-" + i); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentQueueE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentQueueE2ETest.java index 7ffb00565c63c..495a9973b5322 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentQueueE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentQueueE2ETest.java @@ -117,7 +117,7 @@ public void testSimpleConsumerEvents() throws Exception { rolloverPerIntervalStats(); - assertEquals(subRef.getNumberOfEntriesInBacklog(), numMsgs * 2); + assertEquals(subRef.getNumberOfEntriesInBacklog(false), numMsgs * 2); Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); // both consumers will together consumer all messages @@ -141,7 +141,7 @@ public void testSimpleConsumerEvents() throws Exception { // 3. messages deleted on individual acks Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); - assertEquals(subRef.getNumberOfEntriesInBacklog(), 0); + assertEquals(subRef.getNumberOfEntriesInBacklog(false), 0); // 4. shared consumer unsubscribe not allowed try { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java index 38a080f0035e0..42190c7a18024 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicE2ETest.java @@ -161,7 +161,7 @@ public void testSimpleConsumerEvents() throws Exception { assertTrue(subRef.getDispatcher().isConsumerConnected()); rolloverPerIntervalStats(); - assertEquals(subRef.getNumberOfEntriesInBacklog(), numMsgs * 2); + assertEquals(subRef.getNumberOfEntriesInBacklog(false), numMsgs * 2); // 2. messages pushed before client receive Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); @@ -179,7 +179,7 @@ public void testSimpleConsumerEvents() throws Exception { // 4. messages deleted on individual acks Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); - assertEquals(subRef.getNumberOfEntriesInBacklog(), numMsgs); + assertEquals(subRef.getNumberOfEntriesInBacklog(false), numMsgs); for (int i = 0; i < numMsgs; i++) { msg = consumer.receive(); @@ -192,7 +192,7 @@ public void testSimpleConsumerEvents() throws Exception { // 5. messages deleted on cumulative acks Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); - assertEquals(subRef.getNumberOfEntriesInBacklog(), 0); + assertEquals(subRef.getNumberOfEntriesInBacklog(false), 0); // 6. consumer unsubscribe consumer.unsubscribe(); @@ -365,7 +365,7 @@ public Void call() throws Exception { PersistentSubscription subRef = topicRef.getSubscription(subName); // 1. cumulatively all threads drain the backlog - assertEquals(subRef.getNumberOfEntriesInBacklog(), 0); + assertEquals(subRef.getNumberOfEntriesInBacklog(false), 0); // 2. flow control works the same as single consumer single thread Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); @@ -848,13 +848,13 @@ public void testMessageExpiry() throws Exception { } rolloverPerIntervalStats(); - assertEquals(subRef.getNumberOfEntriesInBacklog(), numMsgs); + assertEquals(subRef.getNumberOfEntriesInBacklog(false), numMsgs); Thread.sleep(TimeUnit.SECONDS.toMillis(messageTTLSecs)); runMessageExpiryCheck(); // 1. check all messages expired for this unconnected subscription - assertEquals(subRef.getNumberOfEntriesInBacklog(), 0); + assertEquals(subRef.getNumberOfEntriesInBacklog(false), 0); // clean-up producer.close(); @@ -895,17 +895,17 @@ public void testMessageExpiryWithFewExpiredBacklog() throws Exception { } rolloverPerIntervalStats(); - assertEquals(subRef.getNumberOfEntriesInBacklog(), numMsgs); + assertEquals(subRef.getNumberOfEntriesInBacklog(false), numMsgs); Thread.sleep(TimeUnit.SECONDS.toMillis(messageTTLSecs)); runMessageExpiryCheck(); - assertEquals(subRef.getNumberOfEntriesInBacklog(), numMsgs); + assertEquals(subRef.getNumberOfEntriesInBacklog(false), numMsgs); Thread.sleep(TimeUnit.SECONDS.toMillis(messageTTLSecs / 2)); runMessageExpiryCheck(); - assertEquals(subRef.getNumberOfEntriesInBacklog(), 0); + assertEquals(subRef.getNumberOfEntriesInBacklog(false), 0); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java index bbcb6b194b19b..c9cc07ccaed00 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java @@ -76,19 +76,19 @@ public void testSeek() throws Exception { } PersistentSubscription sub = topicRef.getSubscription("my-subscription"); - assertEquals(sub.getNumberOfEntriesInBacklog(), 10); + assertEquals(sub.getNumberOfEntriesInBacklog(false), 10); consumer.seek(MessageId.latest); - assertEquals(sub.getNumberOfEntriesInBacklog(), 0); + assertEquals(sub.getNumberOfEntriesInBacklog(false), 0); // Wait for consumer to reconnect Thread.sleep(500); consumer.seek(MessageId.earliest); - assertEquals(sub.getNumberOfEntriesInBacklog(), 10); + assertEquals(sub.getNumberOfEntriesInBacklog(false), 10); Thread.sleep(500); consumer.seek(messageIds.get(5)); - assertEquals(sub.getNumberOfEntriesInBacklog(), 5); + assertEquals(sub.getNumberOfEntriesInBacklog(false), 5); } @Test @@ -131,16 +131,16 @@ public void testSeekTime() throws Exception { producer.send(message.getBytes()); } - assertEquals(sub.getNumberOfEntriesInBacklog(), 10); + assertEquals(sub.getNumberOfEntriesInBacklog(false), 10); long currentTimestamp = System.currentTimeMillis(); consumer.seek(currentTimestamp); - assertEquals(sub.getNumberOfEntriesInBacklog(), 1); + assertEquals(sub.getNumberOfEntriesInBacklog(false), 1); // Wait for consumer to reconnect Thread.sleep(1000); consumer.seek(currentTimestamp - resetTimeInMillis); - assertEquals(sub.getNumberOfEntriesInBacklog(), 10); + assertEquals(sub.getNumberOfEntriesInBacklog(false), 10); } @Test @@ -176,7 +176,7 @@ public void testSeekTimeOnPartitionedTopic() throws Exception { long backlogs = 0; for (PersistentSubscription sub : subs) { - backlogs += sub.getNumberOfEntriesInBacklog(); + backlogs += sub.getNumberOfEntriesInBacklog(false); } assertEquals(backlogs, 10); @@ -185,7 +185,7 @@ public void testSeekTimeOnPartitionedTopic() throws Exception { long currentTimestamp = System.currentTimeMillis(); consumer.seek(currentTimestamp); for (PersistentSubscription sub : subs) { - backlogs += sub.getNumberOfEntriesInBacklog(); + backlogs += sub.getNumberOfEntriesInBacklog(false); } assertEquals(backlogs, 2); @@ -195,7 +195,7 @@ public void testSeekTimeOnPartitionedTopic() throws Exception { backlogs = 0; for (PersistentSubscription sub : subs) { - backlogs += sub.getNumberOfEntriesInBacklog(); + backlogs += sub.getNumberOfEntriesInBacklog(false); } assertEquals(backlogs, 10); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java index bb010a61d05a2..c01173ee2511c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java @@ -54,6 +54,7 @@ public void testSimpleAggregation() throws Exception { subStats1.msgBacklog = 50; subStats1.msgRateRedeliver = 1.5; subStats1.unackedMessages = 2; + subStats1.msgBacklogNoDelayed = 30; topicStats1.subscriptionStats.put(namespace, subStats1); TopicStats topicStats2 = new TopicStats(); @@ -81,6 +82,7 @@ public void testSimpleAggregation() throws Exception { subStats2.msgBacklog = 27; subStats2.msgRateRedeliver = 0.7; subStats2.unackedMessages = 0; + subStats2.msgBacklogNoDelayed = 20; topicStats2.subscriptionStats.put(namespace, subStats2); AggregatedNamespaceStats nsStats = new AggregatedNamespaceStats(); @@ -111,6 +113,7 @@ public void testSimpleAggregation() throws Exception { AggregatedSubscriptionStats nsSubStats = nsStats.subscriptionStats.get(namespace); assertNotNull(nsSubStats); assertEquals(nsSubStats.msgBacklog, 77); + assertEquals(nsSubStats.msgBacklogNoDelayed, 50); assertEquals(nsSubStats.msgRateRedeliver, 2.2); assertEquals(nsSubStats.unackedMessages, 2); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DispatcherBlockConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DispatcherBlockConsumerTest.java index bb7aa65d2398a..5cb4fc1383dae 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DispatcherBlockConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/DispatcherBlockConsumerTest.java @@ -544,7 +544,7 @@ public void testBlockDispatcherStats() throws Exception { assertNotNull(topicRef); rolloverPerIntervalStats(); - stats = topicRef.getStats(); + stats = topicRef.getStats(false); subStats = stats.subscriptions.values().iterator().next(); // subscription stats @@ -562,7 +562,7 @@ public void testBlockDispatcherStats() throws Exception { Thread.sleep(timeWaitToSync); rolloverPerIntervalStats(); - stats = topicRef.getStats(); + stats = topicRef.getStats(false); subStats = stats.subscriptions.values().iterator().next(); assertTrue(subStats.msgBacklog > 0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java index 1e46fe86ca41e..9232642c02f9b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPersistentTopicTest.java @@ -438,7 +438,7 @@ public void testTopicStats() throws Exception { assertNotNull(topicRef); rolloverPerIntervalStats(pulsar); - stats = topicRef.getStats(); + stats = topicRef.getStats(false); subStats = stats.getSubscriptions().values().iterator().next(); // subscription stats @@ -456,7 +456,7 @@ public void testTopicStats() throws Exception { Thread.sleep(timeWaitToSync); rolloverPerIntervalStats(pulsar); - stats = topicRef.getStats(); + stats = topicRef.getStats(false); subStats = stats.getSubscriptions().values().iterator().next(); assertTrue(subStats.msgRateOut > 0); @@ -520,7 +520,7 @@ public void testReplicator() throws Exception { assertNotNull(replicatorR3); rolloverPerIntervalStats(replicationPulasr); - stats = topicRef.getStats(); + stats = topicRef.getStats(false); subStats = stats.getSubscriptions().values().iterator().next(); // subscription stats @@ -591,7 +591,7 @@ public void testReplicator() throws Exception { Thread.sleep(timeWaitToSync); rolloverPerIntervalStats(replicationPulasr); - stats = topicRef.getStats(); + stats = topicRef.getStats(false); subStats = stats.getSubscriptions().values().iterator().next(); assertTrue(subStats.msgRateOut > 0); @@ -812,7 +812,7 @@ public void testMsgDropStat() throws Exception { NonPersistentTopic topic = (NonPersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); pulsar.getBrokerService().updateRates(); - NonPersistentTopicStats stats = topic.getStats(); + NonPersistentTopicStats stats = topic.getStats(false); NonPersistentPublisherStats npStats = stats.getPublishers().get(0); NonPersistentSubscriptionStats sub1Stats = stats.getSubscriptions().get("subscriber-1"); NonPersistentSubscriptionStats sub2Stats = stats.getSubscriptions().get("subscriber-2"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageParserTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageParserTest.java index 87278a4d0745b..9fcb129a373c8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageParserTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageParserTest.java @@ -124,7 +124,7 @@ public void testWithBatches() throws Exception { producer.send("hello-" + (n - 1)); // Read through raw data - assertEquals(cursor.getNumberOfEntriesInBacklog(), 1); + assertEquals(cursor.getNumberOfEntriesInBacklog(false), 1); Entry entry = cursor.readEntriesOrWait(1).get(0); List messages = Lists.newArrayList(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java index c823e608dbe7d..c6883c63e0670 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java @@ -803,9 +803,9 @@ public void testDefaultBacklogTTL() throws Exception { topic.get().checkMessageExpiry(); - retryStrategically((test) -> subscription.getNumberOfEntriesInBacklog() == 0, 5, 200); + retryStrategically((test) -> subscription.getNumberOfEntriesInBacklog(false) == 0, 5, 200); - assertEquals(subscription.getNumberOfEntriesInBacklog(), 0); + assertEquals(subscription.getNumberOfEntriesInBacklog(false), 0); } @Test(timeOut = testTimeout) diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Topics.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Topics.java index c5ec04120c022..0a004628de6b0 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Topics.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/Topics.java @@ -565,6 +565,8 @@ List getListInBundle(String namespace, String bundleRange) * * @param topic * topic name + * @param getPreciseBacklog + * Set to true to get precise backlog, Otherwise get imprecise backlog. * @return the topic statistics * * @throws NotAuthorizedException @@ -574,7 +576,11 @@ List getListInBundle(String namespace, String bundleRange) * @throws PulsarAdminException * Unexpected error */ - TopicStats getStats(String topic) throws PulsarAdminException; + TopicStats getStats(String topic, boolean getPreciseBacklog) throws PulsarAdminException; + + default TopicStats getStats(String topic) throws PulsarAdminException { + return getStats(topic, false); + } /** * Get the stats for the topic asynchronously. All the rates are computed over a 1 minute window and are relative @@ -582,11 +588,17 @@ List getListInBundle(String namespace, String bundleRange) * * @param topic * topic name + * @param getPreciseBacklog + * Set to true to get precise backlog, Otherwise get imprecise backlog. * * @return a future that can be used to track when the topic statistics are returned * */ - CompletableFuture getStatsAsync(String topic); + CompletableFuture getStatsAsync(String topic, boolean getPreciseBacklog); + + default CompletableFuture getStatsAsync(String topic) { + return getStatsAsync(topic, false); + } /** * Get the internal stats for the topic. @@ -716,7 +728,11 @@ List getListInBundle(String namespace, String bundleRange) * Unexpected error * */ - PartitionedTopicStats getPartitionedStats(String topic, boolean perPartition) throws PulsarAdminException; + PartitionedTopicStats getPartitionedStats(String topic, boolean perPartition, boolean getPreciseBacklog) throws PulsarAdminException; + + default PartitionedTopicStats getPartitionedStats(String topic, boolean perPartition) throws PulsarAdminException { + return getPartitionedStats(topic, perPartition, false); + } /** * Get the stats for the partitioned topic asynchronously @@ -727,7 +743,11 @@ List getListInBundle(String namespace, String bundleRange) * flag to get stats per partition * @return a future that can be used to track when the partitioned topic statistics are returned */ - CompletableFuture getPartitionedStatsAsync(String topic, boolean perPartition); + CompletableFuture getPartitionedStatsAsync(String topic, boolean perPartition, boolean getPreciseBacklog); + + default CompletableFuture getPartitionedStatsAsync(String topic, boolean perPartition) { + return getPartitionedStatsAsync(topic, perPartition, false); + } /** * Get the stats for the partitioned topic diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java index 75c3d59a55d7c..c61359a74f0b5 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java @@ -437,9 +437,9 @@ public void failed(Throwable throwable) { } @Override - public TopicStats getStats(String topic) throws PulsarAdminException { + public TopicStats getStats(String topic, boolean getPreciseBacklog) throws PulsarAdminException { try { - return getStatsAsync(topic).get(this.readTimeoutMs, TimeUnit.MILLISECONDS); + return getStatsAsync(topic, getPreciseBacklog).get(this.readTimeoutMs, TimeUnit.MILLISECONDS); } catch (ExecutionException e) { throw (PulsarAdminException) e.getCause(); } catch (InterruptedException e) { @@ -451,9 +451,9 @@ public TopicStats getStats(String topic) throws PulsarAdminException { } @Override - public CompletableFuture getStatsAsync(String topic) { + public CompletableFuture getStatsAsync(String topic, boolean getPreciseBacklog) { TopicName tn = validateTopic(topic); - WebTarget path = topicPath(tn, "stats"); + WebTarget path = topicPath(tn, "stats").queryParam("getPreciseBacklog", getPreciseBacklog); final CompletableFuture future = new CompletableFuture<>(); asyncGetRequest(path, new InvocationCallback() { @@ -542,10 +542,10 @@ public void failed(Throwable throwable) { } @Override - public PartitionedTopicStats getPartitionedStats(String topic, boolean perPartition) + public PartitionedTopicStats getPartitionedStats(String topic, boolean perPartition, boolean getPreciseBacklog) throws PulsarAdminException { try { - return getPartitionedStatsAsync(topic, perPartition).get(this.readTimeoutMs, TimeUnit.MILLISECONDS); + return getPartitionedStatsAsync(topic, perPartition, getPreciseBacklog).get(this.readTimeoutMs, TimeUnit.MILLISECONDS); } catch (ExecutionException e) { throw (PulsarAdminException) e.getCause(); } catch (InterruptedException e) { @@ -558,10 +558,10 @@ public PartitionedTopicStats getPartitionedStats(String topic, boolean perPartit @Override public CompletableFuture getPartitionedStatsAsync(String topic, - boolean perPartition) { + boolean perPartition, boolean getPreciseBacklog) { TopicName tn = validateTopic(topic); WebTarget path = topicPath(tn, "partitioned-stats"); - path = path.queryParam("perPartition", perPartition); + path = path.queryParam("perPartition", perPartition).queryParam("getPreciseBacklog", getPreciseBacklog); final CompletableFuture future = new CompletableFuture<>(); asyncGetRequest(path, new InvocationCallback() { diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java index 371f752138a3e..283c5fe7c12ac 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java @@ -642,7 +642,7 @@ void topics() throws Exception { verify(mockTopics).deleteSubscription("persistent://myprop/clust/ns1/ds1", "sub1"); cmdTopics.run(split("stats persistent://myprop/clust/ns1/ds1")); - verify(mockTopics).getStats("persistent://myprop/clust/ns1/ds1"); + verify(mockTopics).getStats("persistent://myprop/clust/ns1/ds1", false); cmdTopics.run(split("stats-internal persistent://myprop/clust/ns1/ds1")); verify(mockTopics).getInternalStats("persistent://myprop/clust/ns1/ds1"); @@ -651,7 +651,7 @@ void topics() throws Exception { verify(mockTopics).getInternalInfo("persistent://myprop/clust/ns1/ds1"); cmdTopics.run(split("partitioned-stats persistent://myprop/clust/ns1/ds1 --per-partition")); - verify(mockTopics).getPartitionedStats("persistent://myprop/clust/ns1/ds1", true); + verify(mockTopics).getPartitionedStats("persistent://myprop/clust/ns1/ds1", true, false); cmdTopics.run(split("clear-backlog persistent://myprop/clust/ns1/ds1 -s sub1")); verify(mockTopics).skipAllMessages("persistent://myprop/clust/ns1/ds1", "sub1"); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java index af5aa02e7a2a3..f5954cf38bac4 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java @@ -370,10 +370,14 @@ private class GetStats extends CliCommand { @Parameter(description = "persistent://tenant/namespace/topic\n", required = true) private java.util.List params; + @Parameter(names = { "-gpb", + "--get-precise-backlog" }, description = "Set true to get precise backlog") + private boolean getPreciseBacklog = false; + @Override void run() throws PulsarAdminException { String topic = validateTopicName(params); - print(topics.getStats(topic)); + print(topics.getStats(topic, getPreciseBacklog)); } } @@ -412,10 +416,14 @@ private class GetPartitionedStats extends CliCommand { @Parameter(names = "--per-partition", description = "Get per partition stats") private boolean perPartition = false; + @Parameter(names = { "-gpb", + "--get-precise-backlog" }, description = "Set true to get precise backlog") + private boolean getPreciseBacklog = false; + @Override void run() throws Exception { String topic = validateTopicName(params); - print(topics.getPartitionedStats(topic, perPartition)); + print(topics.getPartitionedStats(topic, perPartition, getPreciseBacklog)); } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java index a4c299493be79..7064883a04c2a 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/SubscriptionStats.java @@ -40,6 +40,9 @@ public class SubscriptionStats { /** Number of messages in the subscription backlog. */ public long msgBacklog; + /** Number of messages in the subscription backlog that do not contain the delay messages. */ + public long msgBacklogNoDelayed; + /** Flag to verify if subscription is blocked due to reaching threshold of unacked messages. */ public boolean blockedSubscriptionOnUnackedMsgs; @@ -85,6 +88,7 @@ public void reset() { msgThroughputOut = 0; msgRateRedeliver = 0; msgBacklog = 0; + msgBacklogNoDelayed = 0; unackedMessages = 0; msgRateExpired = 0; lastExpireTimestamp = 0L; @@ -99,6 +103,7 @@ public SubscriptionStats add(SubscriptionStats stats) { this.msgThroughputOut += stats.msgThroughputOut; this.msgRateRedeliver += stats.msgRateRedeliver; this.msgBacklog += stats.msgBacklog; + this.msgBacklogNoDelayed += stats.msgBacklogNoDelayed; this.unackedMessages += stats.unackedMessages; this.msgRateExpired += stats.msgRateExpired; this.isReplicated |= stats.isReplicated; diff --git a/site2/docs/admin-api-non-partitioned-topics.md b/site2/docs/admin-api-non-partitioned-topics.md index 61f7409b47938..a8cfe0fc8d183 100644 --- a/site2/docs/admin-api-non-partitioned-topics.md +++ b/site2/docs/admin-api-non-partitioned-topics.md @@ -101,3 +101,59 @@ persistent://tenant/namespace/topic2 ```java admin.topics().getList(namespace); ``` + +### Stats + +It shows current statistics of a given topic. Here's an example payload: + +The following stats are available: + +|Stat|Description| +|----|-----------| +|msgRateIn|The sum of all local and replication publishers’ publish rates in messages per second| +|msgThroughputIn|Same as msgRateIn but in bytes per second instead of messages per second| +|msgRateOut|The sum of all local and replication consumers’ dispatch rates in messages per second| +|msgThroughputOut|Same as msgRateOut but in bytes per second instead of messages per second| +|averageMsgSize|Average message size, in bytes, from this publisher within the last interval| +|storageSize|The sum of the ledgers’ storage size for this topic| +|publishers|The list of all local publishers into the topic. There can be anywhere from zero to thousands.| +|producerId|Internal identifier for this producer on this topic| +|producerName|Internal identifier for this producer, generated by the client library| +|address|IP address and source port for the connection of this producer| +|connectedSince|Timestamp this producer was created or last reconnected| +|subscriptions|The list of all local subscriptions to the topic| +|my-subscription|The name of this subscription (client defined)| +|msgBacklog|The count of messages in backlog for this subscription| +|msgBacklogNoDelayed|The count of messages in backlog without delayed messages for this subscription| +|type|This subscription type| +|msgRateExpired|The rate at which messages were discarded instead of dispatched from this subscription due to TTL| +|consumers|The list of connected consumers for this subscription| +|consumerName|Internal identifier for this consumer, generated by the client library| +|availablePermits|The number of messages this consumer has space for in the client library’s listen queue. A value of 0 means the client library’s queue is full and receive() isn’t being called. A nonzero value means this consumer is ready to be dispatched messages.| +|replication|This section gives the stats for cross-colo replication of this topic| +|replicationBacklog|The outbound replication backlog in messages| +|connected|Whether the outbound replicator is connected| +|replicationDelayInSeconds|How long the oldest message has been waiting to be sent through the connection, if connected is true| +|inboundConnection|The IP and port of the broker in the remote cluster’s publisher connection to this broker| +|inboundConnectedSince|The TCP connection being used to publish messages to the remote cluster. If there are no local publishers connected, this connection is automatically closed after a minute.| + +#### pulsar-admin + +The stats for the topic and its connected producers and consumers can be fetched by using the +[`stats`](reference-pulsar-admin.md#stats) command, specifying the topic by name: + +```shell +$ pulsar-admin topics stats \ + persistent://test-tenant/namespace/topic \ + --get-precise-backlog +``` + +#### REST API + +{@inject: endpoint|GET|/admin/v2/persistent/:tenant/:namespace/:topic/stats|operation/getStats} + +#### Java + +```java +admin.topics().getStats(persistentTopic, false /* is precise backlog */); +``` diff --git a/site2/docs/admin-api-partitioned-topics.md b/site2/docs/admin-api-partitioned-topics.md index a6507e073b759..d413425c282f1 100644 --- a/site2/docs/admin-api-partitioned-topics.md +++ b/site2/docs/admin-api-partitioned-topics.md @@ -254,6 +254,7 @@ The following stats are available: |subscriptions|The list of all local subscriptions to the topic| |my-subscription|The name of this subscription (client defined)| |msgBacklog|The count of messages in backlog for this subscription| +|msgBacklogNoDelayed|The count of messages in backlog without delayed messages for this subscription| |type|This subscription type| |msgRateExpired|The rate at which messages were discarded instead of dispatched from this subscription due to TTL| |consumers|The list of connected consumers for this subscription| @@ -274,7 +275,7 @@ The stats for the partitioned topic and its connected producers and consumers ca ```shell $ pulsar-admin topics partitioned-stats \ persistent://test-tenant/namespace/topic \ - --per-partition + --per-partition ``` #### REST API @@ -284,7 +285,7 @@ $ pulsar-admin topics partitioned-stats \ #### Java ```java -admin.persistentTopics().getStats(persistentTopic); +admin.topics().getPartitionedStats(persistentTopic, true /* per partition */, false /* is precise backlog */); ``` ### Internal stats diff --git a/site2/docs/reference-configuration.md b/site2/docs/reference-configuration.md index d72778a4d8097..cdfe0d419a70d 100644 --- a/site2/docs/reference-configuration.md +++ b/site2/docs/reference-configuration.md @@ -165,6 +165,7 @@ Pulsar brokers are responsible for handling incoming messages from producers, di |brokerClientAuthenticationPlugin| Authentication settings of the broker itself. Used when the broker connects to other brokers, either in same or other clusters || |brokerClientAuthenticationParameters||| |athenzDomainNames| Supported Athenz provider domain names(comma separated) for authentication || +|exposePreciseBacklogInPrometheus| Enable expose the precise backlog stats, set false to use published counter and consumed counter to calculate, this would be more efficient but may be inaccurate. |false| |bookkeeperClientAuthenticationPlugin| Authentication plugin to use when connecting to bookies || |bookkeeperClientAuthenticationParametersName| BookKeeper auth plugin implementatation specifics parameters name and values || |bookkeeperClientAuthenticationParameters||| @@ -353,6 +354,7 @@ The [`pulsar-client`](reference-cli-tools.md#pulsar-client) CLI tool can be used |brokerClientAuthenticationPlugin| The authentication settings of the broker itself. Used when the broker connects to other brokers either in the same cluster or from other clusters. || |brokerClientAuthenticationParameters| The parameters that go along with the plugin specified using brokerClientAuthenticationPlugin. || |athenzDomainNames| Supported Athenz authentication provider domain names as a comma-separated list. || +|exposePreciseBacklogInPrometheus| Enable expose the precise backlog stats, set false to use published counter and consumed counter to calculate, this would be more efficient but may be inaccurate. |false| |bookkeeperClientAuthenticationPlugin| Authentication plugin to be used when connecting to bookies (BookKeeper servers). || |bookkeeperClientAuthenticationParametersName| BookKeeper authentication plugin implementation parameters and values. || |bookkeeperClientAuthenticationParameters| Parameters associated with the bookkeeperClientAuthenticationParametersName || From 3cf6be139e191f3fa058f01c469e1dbd4c8534be Mon Sep 17 00:00:00 2001 From: ran Date: Mon, 17 Feb 2020 18:02:05 +0800 Subject: [PATCH 06/74] KeyValue schema support for pulsar sql (#6325) Fixes #5560 ### Motivation Currently, Pulsar SQL can't read the keyValue schema data. This PR added support Pulsar SQL reading messages with a key-value schema. ### Modifications Add KeyValue schema support for Pulsar SQL. Add prefix __key. for the key field name. --- .../pulsar/common/api/raw/RawMessage.java | 15 + .../pulsar/common/api/raw/RawMessageImpl.java | 24 +- .../pulsar/sql/presto/AvroSchemaHandler.java | 5 + .../pulsar/sql/presto/JSONSchemaHandler.java | 5 + .../sql/presto/KeyValueSchemaHandler.java | 91 +++++ .../pulsar/sql/presto/PulsarColumnHandle.java | 51 ++- .../sql/presto/PulsarColumnMetadata.java | 33 +- .../sql/presto/PulsarInternalColumn.java | 5 +- .../pulsar/sql/presto/PulsarMetadata.java | 86 +++-- .../presto/PulsarPrimitiveSchemaHandler.java | 5 + .../pulsar/sql/presto/PulsarRecordCursor.java | 11 +- .../sql/presto/PulsarSchemaHandlers.java | 4 +- .../apache/pulsar/sql/presto/PulsarSplit.java | 40 ++- .../pulsar/sql/presto/PulsarSplitManager.java | 16 +- .../pulsar/sql/presto/SchemaHandler.java | 2 + .../sql/presto/TestPulsarConnector.java | 51 +-- .../TestPulsarKeyValueSchemaHandler.java | 331 ++++++++++++++++++ .../TestPulsarPrimitiveSchemaHandler.java | 2 +- 18 files changed, 705 insertions(+), 72 deletions(-) create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/KeyValueSchemaHandler.java create mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarKeyValueSchemaHandler.java diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessage.java b/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessage.java index ce5d9404ccb28..07a8098a838df 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessage.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessage.java @@ -101,4 +101,19 @@ public interface RawMessage { * @return the key of the message */ Optional getKey(); + + /** + * Get byteBuf of the key. + * + * @return the byte array with the key payload + */ + Optional getKeyBytes(); + + /** + * Check whether the key has been base64 encoded. + * + * @return true if the key is base64 encoded, false otherwise + */ + boolean hasBase64EncodedKey(); + } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessageImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessageImpl.java index 282945f08ad9d..335bf967bf099 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessageImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessageImpl.java @@ -19,9 +19,10 @@ package org.apache.pulsar.common.api.raw; import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; import io.netty.util.Recycler; import io.netty.util.Recycler.Handle; - +import java.util.Base64; import java.util.Collections; import java.util.Map; import java.util.Optional; @@ -140,4 +141,25 @@ public Optional getKey() { return Optional.empty(); } } + + @Override + public Optional getKeyBytes() { + if (getKey().isPresent()) { + if (hasBase64EncodedKey()) { + return Optional.of(Unpooled.wrappedBuffer(Base64.getDecoder().decode(getKey().get()))); + } else { + return Optional.of(Unpooled.wrappedBuffer(getKey().get().getBytes())); + } + } + return Optional.empty(); + } + + @Override + public boolean hasBase64EncodedKey() { + if (singleMessageMetadata != null) { + return singleMessageMetadata.getPartitionKeyB64Encoded(); + } + return msgMetadata.get().getPartitionKeyB64Encoded(); + } + } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/AvroSchemaHandler.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/AvroSchemaHandler.java index f6807a2b19a14..3e3685ebd6175 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/AvroSchemaHandler.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/AvroSchemaHandler.java @@ -77,6 +77,11 @@ public Object deserialize(ByteBuf payload) { return null; } + @Override + public Object deserialize(ByteBuf keyPayload, ByteBuf dataPayload) { + return null; + } + @Override public Object extractField(int index, Object currentRecord) { try { diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/JSONSchemaHandler.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/JSONSchemaHandler.java index 8649e41fc3c7a..99863a36dae5c 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/JSONSchemaHandler.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/JSONSchemaHandler.java @@ -73,6 +73,11 @@ public Object deserialize(ByteBuf payload) { } } + @Override + public Object deserialize(ByteBuf keyPayload, ByteBuf dataPayload) { + return null; + } + @Override public Object extractField(int index, Object currentRecord) { try { diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/KeyValueSchemaHandler.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/KeyValueSchemaHandler.java new file mode 100644 index 0000000000000..2f056d3abb8a3 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/KeyValueSchemaHandler.java @@ -0,0 +1,91 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.sql.presto; + +import io.airlift.log.Logger; +import io.netty.buffer.ByteBuf; +import java.util.List; +import java.util.Objects; +import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; +import org.apache.pulsar.common.schema.KeyValue; +import org.apache.pulsar.common.schema.KeyValueEncodingType; +import org.apache.pulsar.common.schema.SchemaInfo; + + +/** + * Schema handler for payload in the KeyValue format. + */ +public class KeyValueSchemaHandler implements SchemaHandler { + + private static final Logger log = Logger.get(KeyValueSchemaHandler.class); + + private final List columnHandles; + + private final SchemaHandler keySchemaHandler; + + private final SchemaHandler valueSchemaHandler; + + private KeyValueEncodingType keyValueEncodingType; + + public KeyValueSchemaHandler(SchemaInfo schemaInfo, List columnHandles) { + this.columnHandles = columnHandles; + KeyValue kvSchemaInfo = KeyValueSchemaInfo.decodeKeyValueSchemaInfo(schemaInfo); + keySchemaHandler = PulsarSchemaHandlers.newPulsarSchemaHandler(kvSchemaInfo.getKey(), columnHandles); + valueSchemaHandler = PulsarSchemaHandlers.newPulsarSchemaHandler(kvSchemaInfo.getValue(), columnHandles); + keyValueEncodingType = KeyValueSchemaInfo.decodeKeyValueEncodingType(schemaInfo); + } + + @Override + public Object deserialize(ByteBuf payload) { + return null; + } + + @Override + public Object deserialize(ByteBuf keyPayload, ByteBuf dataPayload) { + ByteBuf keyByteBuf; + ByteBuf valueByteBuf; + if (Objects.equals(keyValueEncodingType, KeyValueEncodingType.INLINE)) { + dataPayload.resetReaderIndex(); + int keyLength = dataPayload.readInt(); + keyByteBuf = dataPayload.readSlice(keyLength); + + int valueLength = dataPayload.readInt(); + valueByteBuf = dataPayload.readSlice(valueLength); + } else { + keyByteBuf = keyPayload; + valueByteBuf = dataPayload; + } + Object keyObj = keySchemaHandler.deserialize(keyByteBuf); + Object valueObj = valueSchemaHandler.deserialize(valueByteBuf); + return new KeyValue<>(keyObj, valueObj); + } + + @Override + public Object extractField(int index, Object currentRecord) { + PulsarColumnHandle pulsarColumnHandle = this.columnHandles.get(index); + KeyValue keyValue = (KeyValue) currentRecord; + if (pulsarColumnHandle.isKey()) { + return keySchemaHandler.extractField(index, keyValue.getKey()); + } else if (pulsarColumnHandle.isValue()) { + return valueSchemaHandler.extractField(index, keyValue.getValue()); + } + return null; + } + +} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnHandle.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnHandle.java index 2a1bd43e07e97..2a07f52e548bd 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnHandle.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnHandle.java @@ -24,8 +24,10 @@ import com.facebook.presto.spi.ColumnMetadata; import com.facebook.presto.spi.type.Type; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Arrays; +import java.util.Objects; /** * This class represents the basic information about a presto column. @@ -58,6 +60,26 @@ public class PulsarColumnHandle implements ColumnHandle { private final Integer[] positionIndices; + private HandleKeyValueType handleKeyValueType; + + /** + * Column Handle keyValue type, used for keyValue schema. + */ + public enum HandleKeyValueType { + /** + * The handle not for keyValue schema. + */ + NONE, + /** + * The key schema handle for keyValue schema. + */ + KEY, + /** + * The value schema handle for keyValue schema. + */ + VALUE + } + @JsonCreator public PulsarColumnHandle( @JsonProperty("connectorId") String connectorId, @@ -66,7 +88,8 @@ public PulsarColumnHandle( @JsonProperty("hidden") boolean hidden, @JsonProperty("internal") boolean internal, @JsonProperty("fieldNames") String[] fieldNames, - @JsonProperty("positionIndices") Integer[] positionIndices) { + @JsonProperty("positionIndices") Integer[] positionIndices, + @JsonProperty("handleKeyValueType") HandleKeyValueType handleKeyValueType) { this.connectorId = requireNonNull(connectorId, "connectorId is null"); this.name = requireNonNull(name, "name is null"); this.type = requireNonNull(type, "type is null"); @@ -74,6 +97,11 @@ public PulsarColumnHandle( this.internal = internal; this.fieldNames = fieldNames; this.positionIndices = positionIndices; + if (handleKeyValueType == null) { + this.handleKeyValueType = HandleKeyValueType.NONE; + } else { + this.handleKeyValueType = handleKeyValueType; + } } @JsonProperty @@ -111,6 +139,20 @@ public Integer[] getPositionIndices() { return positionIndices; } + @JsonProperty + public HandleKeyValueType getHandleKeyValueType() { + return handleKeyValueType; + } + + @JsonIgnore + public boolean isKey() { + return Objects.equals(handleKeyValueType, HandleKeyValueType.KEY); + } + + @JsonIgnore + public boolean isValue() { + return Objects.equals(handleKeyValueType, HandleKeyValueType.VALUE); + } ColumnMetadata getColumnMetadata() { return new ColumnMetadata(name, type, null, hidden); @@ -145,7 +187,10 @@ public boolean equals(Object o) { if (!Arrays.deepEquals(fieldNames, that.fieldNames)) { return false; } - return Arrays.deepEquals(positionIndices, that.positionIndices); + if (!Arrays.deepEquals(positionIndices, that.positionIndices)) { + return false; + } + return Objects.equals(handleKeyValueType, that.handleKeyValueType); } @Override @@ -157,6 +202,7 @@ public int hashCode() { result = 31 * result + (internal ? 1 : 0); result = 31 * result + Arrays.hashCode(fieldNames); result = 31 * result + Arrays.hashCode(positionIndices); + result = 31 * result + (handleKeyValueType != null ? handleKeyValueType.hashCode() : 0); return result; } @@ -170,6 +216,7 @@ public String toString() { + ", internal=" + internal + ", fieldNames=" + Arrays.toString(fieldNames) + ", positionIndices=" + Arrays.toString(positionIndices) + + ", handleKeyValueType=" + handleKeyValueType + '}'; } } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnMetadata.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnMetadata.java index 5b033fac86fec..789ea7f580ac4 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnMetadata.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnMetadata.java @@ -21,6 +21,7 @@ import com.facebook.presto.spi.ColumnMetadata; import com.facebook.presto.spi.type.Type; import java.util.Arrays; +import java.util.Objects; /** * Description of the column metadata. @@ -32,15 +33,19 @@ public class PulsarColumnMetadata extends ColumnMetadata { private String nameWithCase; private String[] fieldNames; private Integer[] positionIndices; + private PulsarColumnHandle.HandleKeyValueType handleKeyValueType; + public final static String KEY_SCHEMA_COLUMN_PREFIX = "__key."; public PulsarColumnMetadata(String name, Type type, String comment, String extraInfo, boolean hidden, boolean isInternal, - String[] fieldNames, Integer[] positionIndices) { + String[] fieldNames, Integer[] positionIndices, + PulsarColumnHandle.HandleKeyValueType handleKeyValueType) { super(name, type, comment, extraInfo, hidden); this.nameWithCase = name; this.isInternal = isInternal; this.fieldNames = fieldNames; this.positionIndices = positionIndices; + this.handleKeyValueType = handleKeyValueType; } public String getNameWithCase() { @@ -59,6 +64,25 @@ public Integer[] getPositionIndices() { return positionIndices; } + public PulsarColumnHandle.HandleKeyValueType getHandleKeyValueType() { + return handleKeyValueType; + } + + public boolean isKey() { + return Objects.equals(handleKeyValueType, PulsarColumnHandle.HandleKeyValueType.KEY); + } + + public boolean isValue() { + return Objects.equals(handleKeyValueType, PulsarColumnHandle.HandleKeyValueType.VALUE); + } + + public static String getColumnName(PulsarColumnHandle.HandleKeyValueType handleKeyValueType, String name) { + if (Objects.equals(PulsarColumnHandle.HandleKeyValueType.KEY, handleKeyValueType)) { + return KEY_SCHEMA_COLUMN_PREFIX + name; + } + return name; + } + @Override public String toString() { return "PulsarColumnMetadata{" @@ -66,6 +90,7 @@ public String toString() { + ", nameWithCase='" + nameWithCase + '\'' + ", fieldNames=" + Arrays.toString(fieldNames) + ", positionIndices=" + Arrays.toString(positionIndices) + + ", handleKeyValueType=" + handleKeyValueType + '}'; } @@ -92,7 +117,10 @@ public boolean equals(Object o) { if (!Arrays.deepEquals(fieldNames, that.fieldNames)) { return false; } - return Arrays.deepEquals(positionIndices, that.positionIndices); + if (!Arrays.deepEquals(positionIndices, that.positionIndices)) { + return false; + } + return Objects.equals(handleKeyValueType, that.handleKeyValueType); } @Override @@ -102,6 +130,7 @@ public int hashCode() { result = 31 * result + (nameWithCase != null ? nameWithCase.hashCode() : 0); result = 31 * result + Arrays.hashCode(fieldNames); result = 31 * result + Arrays.hashCode(positionIndices); + result = 31 * result + (handleKeyValueType != null ? handleKeyValueType.hashCode() : 0); return result; } } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java index 71107f86b3d64..4f2587bf0210f 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java @@ -218,11 +218,12 @@ PulsarColumnHandle getColumnHandle(String connectorId, boolean hidden) { getName(), getType(), hidden, - true, null, null); + true, null, null, PulsarColumnHandle.HandleKeyValueType.NONE); } PulsarColumnMetadata getColumnMetadata(boolean hidden) { - return new PulsarColumnMetadata(name, type, comment, null, hidden, true, null, null); + return new PulsarColumnMetadata(name, type, comment, null, hidden, true, null, null, + PulsarColumnHandle.HandleKeyValueType.NONE); } public static Set getInternalFields() { diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarMetadata.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarMetadata.java index dc9a479aa3503..420232ba0e439 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarMetadata.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarMetadata.java @@ -80,7 +80,9 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.schema.KeyValue; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; @@ -222,7 +224,8 @@ public Map getColumnHandles(ConnectorSession session, Conn pulsarColumnMetadata.isHidden(), pulsarColumnMetadata.isInternal(), pulsarColumnMetadata.getFieldNames(), - pulsarColumnMetadata.getPositionIndices()); + pulsarColumnMetadata.getPositionIndices(), + pulsarColumnMetadata.getHandleKeyValueType()); columnHandles.put( columnMetadata.getName(), @@ -324,7 +327,7 @@ private ConnectorTableMetadata getTableMetadata(SchemaTableName schemaTableName, } } List handles = getPulsarColumns( - topicName, schemaInfo, withInternalColumns + topicName, schemaInfo, withInternalColumns, PulsarColumnHandle.HandleKeyValueType.NONE ); @@ -336,12 +339,15 @@ private ConnectorTableMetadata getTableMetadata(SchemaTableName schemaTableName, */ static List getPulsarColumns(TopicName topicName, SchemaInfo schemaInfo, - boolean withInternalColumns) { + boolean withInternalColumns, + PulsarColumnHandle.HandleKeyValueType handleKeyValueType) { SchemaType schemaType = schemaInfo.getType(); if (schemaType.isStruct()) { - return getPulsarColumnsFromStructSchema(topicName, schemaInfo, withInternalColumns); + return getPulsarColumnsFromStructSchema(topicName, schemaInfo, withInternalColumns, handleKeyValueType); } else if (schemaType.isPrimitive()) { - return getPulsarColumnsFromPrimitiveSchema(topicName, schemaInfo, withInternalColumns); + return getPulsarColumnsFromPrimitiveSchema(topicName, schemaInfo, withInternalColumns, handleKeyValueType); + } else if (schemaType.equals(SchemaType.KEY_VALUE)) { + return getPulsarColumnsFromKeyValueSchema(topicName, schemaInfo, withInternalColumns); } else { throw new IllegalArgumentException("Unsupported schema : " + schemaInfo); } @@ -349,15 +355,16 @@ static List getPulsarColumns(TopicName topicName, static List getPulsarColumnsFromPrimitiveSchema(TopicName topicName, SchemaInfo schemaInfo, - boolean withInternalColumns) { + boolean withInternalColumns, + PulsarColumnHandle.HandleKeyValueType handleKeyValueType) { ImmutableList.Builder builder = ImmutableList.builder(); ColumnMetadata valueColumn = new PulsarColumnMetadata( - "__value__", + PulsarColumnMetadata.getColumnName(handleKeyValueType, "__value__"), convertPulsarType(schemaInfo.getType()), "The value of the message with primitive type schema", null, false, false, new String[0], - new Integer[0]); + new Integer[0], handleKeyValueType); builder.add(valueColumn); @@ -372,8 +379,8 @@ static List getPulsarColumnsFromPrimitiveSchema(TopicName topicN static List getPulsarColumnsFromStructSchema(TopicName topicName, SchemaInfo schemaInfo, - boolean withInternalColumns) { - + boolean withInternalColumns, + PulsarColumnHandle.HandleKeyValueType handleKeyValueType) { String schemaJson = new String(schemaInfo.getSchema()); if (StringUtils.isBlank(schemaJson)) { throw new PrestoException(NOT_SUPPORTED, "Topic " + topicName.toString() @@ -389,7 +396,7 @@ static List getPulsarColumnsFromStructSchema(TopicName topicName ImmutableList.Builder builder = ImmutableList.builder(); - builder.addAll(getColumns(null, schema, new HashSet<>(), new Stack<>(), new Stack<>())); + builder.addAll(getColumns(null, schema, new HashSet<>(), new Stack<>(), new Stack<>(), handleKeyValueType)); if (withInternalColumns) { PulsarInternalColumn.getInternalFields() @@ -398,6 +405,29 @@ static List getPulsarColumnsFromStructSchema(TopicName topicName } return builder.build(); } + + static List getPulsarColumnsFromKeyValueSchema(TopicName topicName, + SchemaInfo schemaInfo, + boolean withInternalColumns) { + ImmutableList.Builder builder = ImmutableList.builder(); + KeyValue kvSchemaInfo = KeyValueSchemaInfo.decodeKeyValueSchemaInfo(schemaInfo); + SchemaInfo keySchemaInfo = kvSchemaInfo.getKey(); + List keyColumnMetadataList = getPulsarColumns(topicName, keySchemaInfo, false, + PulsarColumnHandle.HandleKeyValueType.KEY); + builder.addAll(keyColumnMetadataList); + + SchemaInfo valueSchemaInfo = kvSchemaInfo.getValue(); + List valueColumnMetadataList = getPulsarColumns(topicName, valueSchemaInfo, false, + PulsarColumnHandle.HandleKeyValueType.VALUE); + builder.addAll(valueColumnMetadataList); + + if (withInternalColumns) { + PulsarInternalColumn.getInternalFields() + .forEach(pulsarInternalColumn -> builder.add(pulsarInternalColumn.getColumnMetadata(false))); + } + return builder.build(); + } + @VisibleForTesting static Type convertPulsarType(SchemaType pulsarType) { switch (pulsarType) { @@ -435,18 +465,20 @@ static Type convertPulsarType(SchemaType pulsarType) { @VisibleForTesting static List getColumns(String fieldName, Schema fieldSchema, - Set fieldTypes, - Stack fieldNames, - Stack positionIndices) { + Set fieldTypes, + Stack fieldNames, + Stack positionIndices, + PulsarColumnHandle.HandleKeyValueType handleKeyValueType) { List columnMetadataList = new LinkedList<>(); if (isPrimitiveType(fieldSchema.getType())) { - columnMetadataList.add(new PulsarColumnMetadata(fieldName, + columnMetadataList.add(new PulsarColumnMetadata( + PulsarColumnMetadata.getColumnName(handleKeyValueType, fieldName), convertType(fieldSchema.getType(), fieldSchema.getLogicalType()), null, null, false, false, fieldNames.toArray(new String[fieldNames.size()]), - positionIndices.toArray(new Integer[positionIndices.size()]))); + positionIndices.toArray(new Integer[positionIndices.size()]), handleKeyValueType)); } else if (fieldSchema.getType() == Schema.Type.UNION) { boolean canBeNull = false; for (Schema type : fieldSchema.getTypes()) { @@ -454,17 +486,19 @@ static List getColumns(String fieldName, Schema fieldSchem PulsarColumnMetadata columnMetadata; if (type.getType() != Schema.Type.NULL) { if (!canBeNull) { - columnMetadata = new PulsarColumnMetadata(fieldName, + columnMetadata = new PulsarColumnMetadata( + PulsarColumnMetadata.getColumnName(handleKeyValueType, fieldName), convertType(type.getType(), type.getLogicalType()), null, null, false, false, fieldNames.toArray(new String[fieldNames.size()]), - positionIndices.toArray(new Integer[positionIndices.size()])); + positionIndices.toArray(new Integer[positionIndices.size()]), handleKeyValueType); } else { - columnMetadata = new PulsarColumnMetadata(fieldName, + columnMetadata = new PulsarColumnMetadata( + PulsarColumnMetadata.getColumnName(handleKeyValueType, fieldName), convertType(type.getType(), type.getLogicalType()), "field can be null", null, false, false, fieldNames.toArray(new String[fieldNames.size()]), - positionIndices.toArray(new Integer[positionIndices.size()])); + positionIndices.toArray(new Integer[positionIndices.size()]), handleKeyValueType); } columnMetadataList.add(columnMetadata); } else { @@ -472,7 +506,7 @@ static List getColumns(String fieldName, Schema fieldSchem } } else { List columns = getColumns(fieldName, type, fieldTypes, fieldNames, - positionIndices); + positionIndices, handleKeyValueType); columnMetadataList.addAll(columns); } } @@ -488,10 +522,11 @@ static List getColumns(String fieldName, Schema fieldSchem positionIndices.push(i); List columns; if (fieldName == null) { - columns = getColumns(field.name(), field.schema(), fieldTypes, fieldNames, positionIndices); + columns = getColumns(field.name(), field.schema(), fieldTypes, fieldNames, positionIndices, + handleKeyValueType); } else { columns = getColumns(String.format("%s.%s", fieldName, field.name()), field.schema(), - fieldTypes, fieldNames, positionIndices); + fieldTypes, fieldNames, positionIndices, handleKeyValueType); } positionIndices.pop(); @@ -507,11 +542,12 @@ static List getColumns(String fieldName, Schema fieldSchem } else if (fieldSchema.getType() == Schema.Type.MAP) { } else if (fieldSchema.getType() == Schema.Type.ENUM) { - PulsarColumnMetadata columnMetadata = new PulsarColumnMetadata(fieldName, + PulsarColumnMetadata columnMetadata = new PulsarColumnMetadata( + PulsarColumnMetadata.getColumnName(handleKeyValueType, fieldName), convertType(fieldSchema.getType(), fieldSchema.getLogicalType()), null, null, false, false, fieldNames.toArray(new String[fieldNames.size()]), - positionIndices.toArray(new Integer[positionIndices.size()])); + positionIndices.toArray(new Integer[positionIndices.size()]), handleKeyValueType); columnMetadataList.add(columnMetadata); } else if (fieldSchema.getType() == Schema.Type.FIXED) { diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarPrimitiveSchemaHandler.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarPrimitiveSchemaHandler.java index 28980a9f13be4..f26587bfcbf80 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarPrimitiveSchemaHandler.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarPrimitiveSchemaHandler.java @@ -56,6 +56,11 @@ public Object deserialize(ByteBuf byteBuf) { } } + @Override + public Object deserialize(ByteBuf keyPayload, ByteBuf dataPayload) { + return null; + } + @Override public Object extractField(int index, Object currentRecord) { return currentRecord; diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java index eca99af4dbb4b..6899f97bbd1c2 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java @@ -40,6 +40,7 @@ import io.airlift.log.Logger; import io.airlift.slice.Slice; import io.airlift.slice.Slices; +import io.netty.buffer.ByteBuf; import java.io.IOException; import java.util.List; import java.util.Map; @@ -418,7 +419,15 @@ public boolean advanceNextPosition() { //start time for deseralizing record metricsTracker.start_RECORD_DESERIALIZE_TIME(); - currentRecord = this.schemaHandler.deserialize(this.currentMessage.getData()); + if (this.schemaHandler instanceof KeyValueSchemaHandler) { + ByteBuf keyByteBuf = null; + if (this.currentMessage.getKeyBytes().isPresent()) { + keyByteBuf = this.currentMessage.getKeyBytes().get(); + } + currentRecord = this.schemaHandler.deserialize(keyByteBuf, this.currentMessage.getData()); + } else { + currentRecord = this.schemaHandler.deserialize(this.currentMessage.getData()); + } metricsTracker.incr_NUM_RECORD_DESERIALIZED(); // stats for time spend deserializing diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSchemaHandlers.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSchemaHandlers.java index c0282df14007e..3b091ad74c77a 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSchemaHandlers.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSchemaHandlers.java @@ -23,9 +23,9 @@ import com.facebook.presto.spi.PrestoException; import java.util.List; - import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; class PulsarSchemaHandlers { @@ -45,6 +45,8 @@ static SchemaHandler newPulsarSchemaHandler(SchemaInfo schemaInfo, throw new PrestoException(NOT_SUPPORTED, "Not supported schema type: " + schemaInfo.getType()); } + } else if (schemaInfo.getType().equals(SchemaType.KEY_VALUE)) { + return new KeyValueSchemaHandler(schemaInfo, columnHandles); } else { throw new PrestoException( NOT_SUPPORTED, diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java index dbebbf54a3eba..9893d8ffab0ea 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplit.java @@ -26,7 +26,10 @@ import com.facebook.presto.spi.predicate.TupleDomain; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; +import java.io.IOException; import java.util.List; import java.util.Map; import org.apache.bookkeeper.mledger.impl.PositionImpl; @@ -39,9 +42,12 @@ */ public class PulsarSplit implements ConnectorSplit { + private static final Logger log = Logger.get(PulsarSplit.class); + private final long splitId; private final String connectorId; private final String schemaName; + private final String originSchemaName; private final String tableName; private final long splitSize; private final String schema; @@ -55,6 +61,7 @@ public class PulsarSplit implements ConnectorSplit { private final PositionImpl startPosition; private final PositionImpl endPosition; + private final String schemaInfoProperties; private final OffloadPolicies offloadPolicies; @@ -63,6 +70,7 @@ public PulsarSplit( @JsonProperty("splitId") long splitId, @JsonProperty("connectorId") String connectorId, @JsonProperty("schemaName") String schemaName, + @JsonProperty("originSchemaName") String originSchemaName, @JsonProperty("tableName") String tableName, @JsonProperty("splitSize") long splitSize, @JsonProperty("schema") String schema, @@ -72,16 +80,11 @@ public PulsarSplit( @JsonProperty("startPositionLedgerId") long startPositionLedgerId, @JsonProperty("endPositionLedgerId") long endPositionLedgerId, @JsonProperty("tupleDomain") TupleDomain tupleDomain, - @JsonProperty("properties") Map schemaInfoProperties, - @JsonProperty("offloadPolicies") OffloadPolicies offloadPolicies) { + @JsonProperty("schemaInfoProperties") String schemaInfoProperties, + @JsonProperty("offloadPolicies") OffloadPolicies offloadPolicies) throws IOException { this.splitId = splitId; requireNonNull(schemaName, "schema name is null"); - this.schemaInfo = SchemaInfo.builder() - .type(schemaType) - .name(schemaName) - .schema(schema.getBytes()) - .properties(schemaInfoProperties) - .build(); + this.originSchemaName = originSchemaName; this.schemaName = requireNonNull(schemaName, "schema name is null"); this.connectorId = requireNonNull(connectorId, "connector id is null"); this.tableName = requireNonNull(tableName, "table name is null"); @@ -95,7 +98,16 @@ public PulsarSplit( this.tupleDomain = requireNonNull(tupleDomain, "tupleDomain is null"); this.startPosition = PositionImpl.get(startPositionLedgerId, startPositionEntryId); this.endPosition = PositionImpl.get(endPositionLedgerId, endPositionEntryId); + this.schemaInfoProperties = schemaInfoProperties; this.offloadPolicies = offloadPolicies; + + ObjectMapper objectMapper = new ObjectMapper(); + this.schemaInfo = SchemaInfo.builder() + .name(originSchemaName) + .type(schemaType) + .schema(schema.getBytes("ISO8859-1")) + .properties(objectMapper.readValue(schemaInfoProperties, Map.class)) + .build(); } @JsonProperty @@ -128,6 +140,11 @@ public long getSplitSize() { return splitSize; } + @JsonProperty + public String getOriginSchemaName() { + return originSchemaName; + } + @JsonProperty public String getSchema() { return schema; @@ -166,6 +183,11 @@ public PositionImpl getEndPosition() { return endPosition; } + @JsonProperty + public String getSchemaInfoProperties() { + return schemaInfoProperties; + } + @JsonProperty public OffloadPolicies getOffloadPolicies() { return offloadPolicies; @@ -191,6 +213,7 @@ public String toString() { return "PulsarSplit{" + "splitId=" + splitId + ", connectorId='" + connectorId + '\'' + + ", originSchemaName='" + originSchemaName + '\'' + ", schemaName='" + schemaName + '\'' + ", tableName='" + tableName + '\'' + ", splitSize=" + splitSize @@ -200,6 +223,7 @@ public String toString() { + ", endPositionEntryId=" + endPositionEntryId + ", startPositionLedgerId=" + startPositionLedgerId + ", endPositionLedgerId=" + endPositionLedgerId + + ", schemaInfoProperties=" + schemaInfoProperties + (offloadPolicies == null ? "" : offloadPolicies.toString()) + '}'; } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java index 848b9ccd1789c..cce071ae15f12 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java @@ -35,9 +35,11 @@ import com.facebook.presto.spi.predicate.Domain; import com.facebook.presto.spi.predicate.Range; import com.facebook.presto.spi.predicate.TupleDomain; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Predicate; import io.airlift.log.Logger; +import java.io.IOException; import java.sql.Timestamp; import java.util.ArrayList; import java.util.Collection; @@ -75,6 +77,8 @@ public class PulsarSplitManager implements ConnectorSplitManager { private static final Logger log = Logger.get(PulsarSplitManager.class); + private ObjectMapper objectMapper = new ObjectMapper(); + @Inject public PulsarSplitManager(PulsarConnectorId connectorId, PulsarConnectorConfig pulsarConnectorConfig) { this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); @@ -250,7 +254,7 @@ Collection getSplitsForTopic(String topicNamePersistenceEncoding, SchemaInfo schemaInfo, String tableName, TupleDomain tupleDomain, OffloadPolicies offloadPolicies) - throws ManagedLedgerException, InterruptedException { + throws ManagedLedgerException, InterruptedException, IOException { ReadOnlyCursor readOnlyCursor = null; try { @@ -295,19 +299,21 @@ Collection getSplitsForTopic(String topicNamePersistenceEncoding, readOnlyCursor.skipEntries(Math.toIntExact(entriesForSplit)); PositionImpl endPosition = (PositionImpl) readOnlyCursor.getReadPosition(); - splits.add(new PulsarSplit(i, this.connectorId, + PulsarSplit pulsarSplit = new PulsarSplit(i, this.connectorId, restoreNamespaceDelimiterIfNeeded(tableHandle.getSchemaName(), pulsarConnectorConfig), + schemaInfo.getName(), tableName, entriesForSplit, - new String(schemaInfo.getSchema()), + new String(schemaInfo.getSchema(), "ISO8859-1"), schemaInfo.getType(), startPosition.getEntryId(), endPosition.getEntryId(), startPosition.getLedgerId(), endPosition.getLedgerId(), tupleDomain, - schemaInfo.getProperties(), - offloadPolicies)); + objectMapper.writeValueAsString(schemaInfo.getProperties()), + offloadPolicies); + splits.add(pulsarSplit); } return splits; } finally { diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/SchemaHandler.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/SchemaHandler.java index 37fce04329555..7529034accea6 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/SchemaHandler.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/SchemaHandler.java @@ -27,6 +27,8 @@ public interface SchemaHandler { Object deserialize(ByteBuf payload); + Object deserialize(ByteBuf keyPayload, ByteBuf dataPayload); + Object extractField(int index, Object currentRecord); } diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnector.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnector.java index 6c911a407541a..adeababe481e8 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnector.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnector.java @@ -26,6 +26,7 @@ import com.facebook.presto.spi.type.RealType; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.VarcharType; +import com.fasterxml.jackson.databind.ObjectMapper; import io.airlift.log.Logger; import io.netty.buffer.ByteBuf; import org.apache.bookkeeper.mledger.AsyncCallbacks; @@ -118,6 +119,7 @@ public abstract class TestPulsarConnector { protected static Map topicsToSchemas; protected static Map topicsToNumEntries; + private final static ObjectMapper objectMapper = new ObjectMapper(); protected static final NamespaceName NAMESPACE_NAME_1 = NamespaceName.get("tenant-1", "ns-1"); protected static final NamespaceName NAMESPACE_NAME_2 = NamespaceName.get("tenant-1", "ns-2"); @@ -299,7 +301,7 @@ public static class Boo { false, false, fooFieldNames.get("field1"), - fooPositionIndices.get("field1"))); + fooPositionIndices.get("field1"), null)); String[] fieldNames2 = {"field2"}; @@ -312,7 +314,7 @@ public static class Boo { false, false, fieldNames2, - positionIndices2)); + positionIndices2, null)); String[] fieldNames3 = {"field3"}; Integer[] positionIndices3 = {2}; @@ -324,7 +326,7 @@ public static class Boo { false, false, fieldNames3, - positionIndices3)); + positionIndices3, null)); String[] fieldNames4 = {"field4"}; Integer[] positionIndices4 = {3}; @@ -336,7 +338,7 @@ public static class Boo { false, false, fieldNames4, - positionIndices4)); + positionIndices4, null)); String[] fieldNames5 = {"field5"}; @@ -349,7 +351,7 @@ public static class Boo { false, false, fieldNames5, - positionIndices5)); + positionIndices5, null)); String[] fieldNames6 = {"field6"}; Integer[] positionIndices6 = {5}; @@ -361,7 +363,7 @@ public static class Boo { false, false, fieldNames6, - positionIndices6)); + positionIndices6, null)); String[] fieldNames7 = {"timestamp"}; Integer[] positionIndices7 = {6}; @@ -373,7 +375,7 @@ public static class Boo { false, false, fieldNames7, - positionIndices7)); + positionIndices7, null)); String[] fieldNames8 = {"time"}; Integer[] positionIndices8 = {7}; @@ -385,7 +387,7 @@ public static class Boo { false, false, fieldNames8, - positionIndices8)); + positionIndices8, null)); String[] fieldNames9 = {"date"}; Integer[] positionIndices9 = {8}; @@ -397,7 +399,7 @@ public static class Boo { false, false, fieldNames9, - positionIndices9)); + positionIndices9, null)); String[] bar_fieldNames1 = {"bar", "field1"}; Integer[] bar_positionIndices1 = {9, 0}; @@ -409,7 +411,7 @@ public static class Boo { false, false, bar_fieldNames1, - bar_positionIndices1)); + bar_positionIndices1, null)); String[] bar_fieldNames2 = {"bar", "field2"}; Integer[] bar_positionIndices2 = {9, 1}; @@ -421,7 +423,7 @@ public static class Boo { false, false, bar_fieldNames2, - bar_positionIndices2)); + bar_positionIndices2, null)); String[] bar_test_fieldNames4 = {"bar", "test", "field4"}; Integer[] bar_test_positionIndices4 = {9, 2, 0}; @@ -433,7 +435,7 @@ public static class Boo { false, false, bar_test_fieldNames4, - bar_test_positionIndices4)); + bar_test_positionIndices4, null)); String[] bar_test_fieldNames5 = {"bar", "test", "field5"}; Integer[] bar_test_positionIndices5 = {9, 2, 1}; @@ -445,7 +447,7 @@ public static class Boo { false, false, bar_test_fieldNames5, - bar_test_positionIndices5)); + bar_test_positionIndices5, null)); String[] bar_test_fieldNames6 = {"bar", "test", "field6"}; Integer[] bar_test_positionIndices6 = {9, 2, 2}; @@ -457,7 +459,7 @@ public static class Boo { false, false, bar_test_fieldNames6, - bar_test_positionIndices6)); + bar_test_positionIndices6, null)); String[] bar_test_foobar_fieldNames1 = {"bar", "test", "foobar", "field1"}; Integer[] bar_test_foobar_positionIndices1 = {9, 2, 6, 0}; @@ -469,7 +471,7 @@ public static class Boo { false, false, bar_test_foobar_fieldNames1, - bar_test_foobar_positionIndices1)); + bar_test_foobar_positionIndices1, null)); String[] bar_field3 = {"bar", "field3"}; Integer[] bar_positionIndices3 = {9, 3}; @@ -481,7 +483,7 @@ public static class Boo { false, false, bar_field3, - bar_positionIndices3)); + bar_positionIndices3, null)); String[] bar_test2_fieldNames4 = {"bar", "test2", "field4"}; Integer[] bar_test2_positionIndices4 = {9, 4, 0}; @@ -493,7 +495,7 @@ public static class Boo { false, false, bar_test2_fieldNames4, - bar_test2_positionIndices4)); + bar_test2_positionIndices4, null)); String[] bar_test2_fieldNames5 = {"bar", "test2", "field5"}; Integer[] bar_test2_positionIndices5 = {9, 4, 1}; @@ -505,7 +507,7 @@ public static class Boo { false, false, bar_test2_fieldNames5, - bar_test2_positionIndices5)); + bar_test2_positionIndices5, null)); String[] bar_test2_fieldNames6 = {"bar", "test2", "field6"}; Integer[] bar_test2_positionIndices6 = {9, 4, 2}; @@ -517,7 +519,7 @@ public static class Boo { false, false, bar_test2_fieldNames6, - bar_test2_positionIndices6)); + bar_test2_positionIndices6, null)); String[] bar_test2_foobar_fieldNames1 = {"bar", "test2", "foobar", "field1"}; Integer[] bar_test2_foobar_positionIndices1 = {9, 4, 6, 0}; @@ -529,7 +531,7 @@ public static class Boo { false, false, bar_test2_foobar_fieldNames1, - bar_test2_foobar_positionIndices1)); + bar_test2_foobar_positionIndices1, null)); String[] fieldNames10 = {"field7"}; Integer[] positionIndices10 = {10}; @@ -541,7 +543,7 @@ public static class Boo { false, false, fieldNames10, - positionIndices10)); + positionIndices10, null)); fooColumnHandles.addAll(PulsarInternalColumn.getInternalFields().stream() .map(pulsarInternalColumn -> pulsarInternalColumn.getColumnHandle(pulsarConnectorId.toString(), false)) @@ -556,12 +558,14 @@ public static class Boo { for (TopicName topicName : allTopics) { if (topicsToSchemas.containsKey(topicName.getSchemaName())) { splits.put(topicName, new PulsarSplit(0, pulsarConnectorId.toString(), - topicName.getNamespace(), topicName.getLocalName(), + topicName.getNamespace(), topicName.getLocalName(), topicName.getLocalName(), topicsToNumEntries.get(topicName.getSchemaName()), new String(topicsToSchemas.get(topicName.getSchemaName()).getSchema()), topicsToSchemas.get(topicName.getSchemaName()).getType(), 0, topicsToNumEntries.get(topicName.getSchemaName()), - 0, 0, TupleDomain.all(), new HashMap<>(), null)); + 0, 0, TupleDomain.all(), + objectMapper.writeValueAsString( + topicsToSchemas.get(topicName.getSchemaName()).getProperties()), null)); } } @@ -601,7 +605,6 @@ public static class Boo { } catch (Throwable e) { System.out.println("Error: " + e); System.out.println("Stacktrace: " + Arrays.asList(e.getStackTrace())); - throw e; } } diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarKeyValueSchemaHandler.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarKeyValueSchemaHandler.java new file mode 100644 index 0000000000000..193f68e00b069 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarKeyValueSchemaHandler.java @@ -0,0 +1,331 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.sql.presto; + +import com.facebook.presto.spi.ColumnMetadata; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import lombok.Data; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.schema.KeyValueSchema; +import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; +import org.apache.pulsar.common.api.raw.RawMessage; +import org.apache.pulsar.common.api.raw.RawMessageImpl; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.schema.KeyValue; +import org.apache.pulsar.common.schema.KeyValueEncodingType; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.Test; + + +/** + * Unit test for KeyValueSchemaHandler + */ +@Slf4j +public class TestPulsarKeyValueSchemaHandler { + + private final static ObjectMapper objectMapper = new ObjectMapper(); + + private Schema> schema1 = + Schema.KeyValue(Schema.STRING, Schema.INT32, KeyValueEncodingType.INLINE); + + private Schema> schema2 = + Schema.KeyValue(Schema.STRING, Schema.JSON(Foo.class), KeyValueEncodingType.INLINE); + + private Schema> schema3 = + Schema.KeyValue(Schema.AVRO(Boo.class), Schema.INT64, KeyValueEncodingType.SEPARATED); + + private Schema> schema4 = + Schema.KeyValue(Schema.JSON(Boo.class), Schema.AVRO(Foo.class), KeyValueEncodingType.SEPARATED); + + private final static TopicName topicName = TopicName.get("persistent://public/default/kv-test"); + + private final static Foo foo; + + private final static Boo boo; + + private final Integer KEY_FIELD_NAME_PREFIX_LENGTH = PulsarColumnMetadata.KEY_SCHEMA_COLUMN_PREFIX.length(); + + static { + foo = new Foo(); + foo.field1 = "field1-value"; + foo.field2 = 20; + + boo = new Boo(); + boo.field1 = "field1-value"; + boo.field2 = true; + boo.field3 = 10.2; + } + + + @Test + public void testSchema1() throws IOException { + final String keyData = "test-key"; + final Integer valueData = 10; + List columnMetadataList = + PulsarMetadata.getPulsarColumns(topicName, schema1.getSchemaInfo(), + true, null); + int keyCount = 0; + int valueCount = 0; + for (ColumnMetadata columnMetadata : columnMetadataList) { + PulsarColumnMetadata pulsarColumnMetadata = (PulsarColumnMetadata) columnMetadata; + if (pulsarColumnMetadata.isKey()) { + keyCount++; + } else if (pulsarColumnMetadata.isValue()) { + valueCount++; + } + } + Assert.assertEquals(keyCount, 1); + Assert.assertEquals(valueCount, 1); + + List columnHandleList = getColumnHandlerList(columnMetadataList); + + KeyValueSchemaHandler keyValueSchemaHandler = + new KeyValueSchemaHandler(schema1.getSchemaInfo(), columnHandleList); + + RawMessageImpl message = Mockito.mock(RawMessageImpl.class); + Mockito.when(message.getData()).thenReturn( + Unpooled.wrappedBuffer(schema1.encode(new KeyValue<>(keyData, valueData))) + ); + + KeyValue byteBufKeyValue = getKeyValueByteBuf(message, schema1); + Object object = keyValueSchemaHandler.deserialize(byteBufKeyValue.getKey(), byteBufKeyValue.getValue()); + Assert.assertEquals(keyValueSchemaHandler.extractField(0, object), keyData); + Assert.assertEquals(keyValueSchemaHandler.extractField(1, object), valueData); + } + + @Test + public void testSchema2() throws IOException { + final String keyData = "test-key"; + + List columnMetadataList = + PulsarMetadata.getPulsarColumns(topicName, schema2.getSchemaInfo(), + true, null); + int keyCount = 0; + int valueCount = 0; + for (ColumnMetadata columnMetadata : columnMetadataList) { + PulsarColumnMetadata pulsarColumnMetadata = (PulsarColumnMetadata) columnMetadata; + if (pulsarColumnMetadata.isKey()) { + keyCount++; + } else if (pulsarColumnMetadata.isValue()) { + valueCount++; + } + } + Assert.assertEquals(keyCount, 1); + Assert.assertEquals(valueCount, 2); + + List columnHandleList = getColumnHandlerList(columnMetadataList); + + RawMessage message = Mockito.mock(RawMessage.class); + Mockito.when(message.getData()).thenReturn( + Unpooled.wrappedBuffer(schema2.encode(new KeyValue<>(keyData, foo))) + ); + + + KeyValueSchemaHandler keyValueSchemaHandler = + new KeyValueSchemaHandler(schema2.getSchemaInfo(), columnHandleList); + + KeyValue byteBufKeyValue = getKeyValueByteBuf(message, schema2); + Object object = keyValueSchemaHandler.deserialize(byteBufKeyValue.getKey(), byteBufKeyValue.getValue()); + Assert.assertEquals(keyValueSchemaHandler.extractField(0, object), keyData); + Assert.assertEquals(keyValueSchemaHandler.extractField(1, object), + foo.getValue(columnHandleList.get(1).getName())); + Assert.assertEquals(keyValueSchemaHandler.extractField(2, object), + foo.getValue(columnHandleList.get(2).getName())); + } + + @Test + public void testSchema3() throws IOException { + final Boo boo = new Boo(); + boo.field1 = "field1-value"; + boo.field2 = true; + boo.field3 = 10.2; + final Long valueData = 999999L; + + List columnMetadataList = + PulsarMetadata.getPulsarColumns(topicName, schema3.getSchemaInfo(), + true, null); + int keyCount = 0; + int valueCount = 0; + for (ColumnMetadata columnMetadata : columnMetadataList) { + PulsarColumnMetadata pulsarColumnMetadata = (PulsarColumnMetadata) columnMetadata; + if (pulsarColumnMetadata.isKey()) { + keyCount++; + } else if (pulsarColumnMetadata.isValue()) { + valueCount++; + } + } + Assert.assertEquals(keyCount, 3); + Assert.assertEquals(valueCount, 1); + + List columnHandleList = getColumnHandlerList(columnMetadataList); + + KeyValueSchemaHandler keyValueSchemaHandler = + new KeyValueSchemaHandler(schema3.getSchemaInfo(), columnHandleList); + + RawMessage message = Mockito.mock(RawMessage.class); + Mockito.when(message.getKeyBytes()).thenReturn( + Optional.of(Unpooled.wrappedBuffer( + ((KeyValueSchema) schema3).getKeySchema().encode(boo) + )) + ); + Mockito.when(message.getData()).thenReturn( + Unpooled.wrappedBuffer(schema3.encode(new KeyValue<>(boo, valueData))) + ); + + KeyValue byteBufKeyValue = getKeyValueByteBuf(message, schema3); + Object object = keyValueSchemaHandler.deserialize(byteBufKeyValue.getKey(), byteBufKeyValue.getValue()); + Assert.assertEquals(keyValueSchemaHandler.extractField(0, object).toString(), + boo.getValue(columnHandleList.get(0).getName().substring(KEY_FIELD_NAME_PREFIX_LENGTH))); + Assert.assertEquals(keyValueSchemaHandler.extractField(1, object), + boo.getValue(columnHandleList.get(1).getName().substring(KEY_FIELD_NAME_PREFIX_LENGTH))); + Assert.assertEquals(keyValueSchemaHandler.extractField(2, object), + boo.getValue(columnHandleList.get(2).getName().substring(KEY_FIELD_NAME_PREFIX_LENGTH))); + Assert.assertEquals(keyValueSchemaHandler.extractField(3, object), valueData); + } + + @Test + public void testSchema4() throws IOException { + List columnMetadataList = + PulsarMetadata.getPulsarColumns(topicName, schema4.getSchemaInfo(), + true, null); + int keyCount = 0; + int valueCount = 0; + for (ColumnMetadata columnMetadata : columnMetadataList) { + PulsarColumnMetadata pulsarColumnMetadata = (PulsarColumnMetadata) columnMetadata; + if (pulsarColumnMetadata.isKey()) { + keyCount++; + } else if (pulsarColumnMetadata.isValue()) { + valueCount++; + } + } + Assert.assertEquals(keyCount, 3); + Assert.assertEquals(valueCount, 2); + + List columnHandleList = getColumnHandlerList(columnMetadataList); + + KeyValueSchemaHandler keyValueSchemaHandler = + new KeyValueSchemaHandler(schema4.getSchemaInfo(), columnHandleList); + + RawMessage message = Mockito.mock(RawMessage.class); + Mockito.when(message.getKeyBytes()).thenReturn( + Optional.of(Unpooled.wrappedBuffer( + ((KeyValueSchema) schema4).getKeySchema().encode(boo) + )) + ); + Mockito.when(message.getData()).thenReturn( + Unpooled.wrappedBuffer(schema4.encode(new KeyValue<>(boo, foo))) + ); + + KeyValue byteBufKeyValue = getKeyValueByteBuf(message, schema4); + Object object = keyValueSchemaHandler.deserialize(byteBufKeyValue.getKey(), byteBufKeyValue.getValue()); + Assert.assertEquals(keyValueSchemaHandler.extractField(0, object).toString(), + boo.getValue(columnHandleList.get(0).getName().substring(KEY_FIELD_NAME_PREFIX_LENGTH))); + Assert.assertEquals(keyValueSchemaHandler.extractField(1, object), + boo.getValue(columnHandleList.get(1).getName().substring(KEY_FIELD_NAME_PREFIX_LENGTH))); + Assert.assertEquals(keyValueSchemaHandler.extractField(2, object), + boo.getValue(columnHandleList.get(2).getName().substring(KEY_FIELD_NAME_PREFIX_LENGTH))); + Assert.assertEquals(keyValueSchemaHandler.extractField(3, object).toString(), + foo.getValue(columnHandleList.get(3).getName())); + Assert.assertEquals(keyValueSchemaHandler.extractField(4, object).toString(), + foo.getValue(columnHandleList.get(4).getName()) + ""); + } + + private List getColumnHandlerList(List columnMetadataList) { + List columnHandleList = new LinkedList<>(); + + columnMetadataList.forEach(columnMetadata -> { + PulsarColumnMetadata pulsarColumnMetadata = (PulsarColumnMetadata) columnMetadata; + PulsarColumnHandle pulsarColumnHandle = new PulsarColumnHandle( + "connectorId", + pulsarColumnMetadata.getNameWithCase(), + pulsarColumnMetadata.getType(), + pulsarColumnMetadata.isHidden(), + pulsarColumnMetadata.isInternal(), + pulsarColumnMetadata.getFieldNames(), + pulsarColumnMetadata.getPositionIndices(), + pulsarColumnMetadata.getHandleKeyValueType()); + columnHandleList.add(pulsarColumnHandle); + }); + + return columnHandleList; + } + + public KeyValue getKeyValueByteBuf(RawMessage message, Schema schema) { + KeyValueEncodingType encodingType = KeyValueSchemaInfo.decodeKeyValueEncodingType(schema.getSchemaInfo()); + ByteBuf keyByteBuf = null; + if (Objects.equals(KeyValueEncodingType.SEPARATED, encodingType)) { + if (message.getKeyBytes().isPresent()) { + keyByteBuf = message.getKeyBytes().get(); + } else { + keyByteBuf = null; + } + } else { + keyByteBuf = null; + } + return new KeyValue<>(keyByteBuf, Unpooled.wrappedBuffer(message.getData())); + } + + @Data + static class Foo { + private String field1; + private Integer field2; + + public Object getValue(String fieldName) { + switch (fieldName) { + case "field1": + return field1; + case "field2": + return field2 == null ? null : new Long(field2); + default: + return null; + } + } + } + + @Data + static class Boo { + private String field1; + private Boolean field2; + private Double field3; + + public Object getValue(String fieldName) { + switch (fieldName) { + case "field1": + return field1; + case "field2": + return field2; + case "field3": + return field3 == null ? null : field3.doubleValue(); + default: + return null; + } + } + + } + +} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarPrimitiveSchemaHandler.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarPrimitiveSchemaHandler.java index 87e148db84f1c..ac2e1f4bd3815 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarPrimitiveSchemaHandler.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarPrimitiveSchemaHandler.java @@ -153,7 +153,7 @@ public void testNewPulsarPrimitiveSchemaHandler() { @Test public void testNewColumnMetadata() { List columnMetadataList = PulsarMetadata.getPulsarColumns(stringTopicName, - StringSchema.utf8().getSchemaInfo(), false); + StringSchema.utf8().getSchemaInfo(), false, null); Assert.assertEquals(columnMetadataList.size(), 1); ColumnMetadata columnMetadata = columnMetadataList.get(0); Assert.assertEquals("__value__", columnMetadata.getName()); From 26d569b9a894b1ab051bdf215cb99a107f8abee5 Mon Sep 17 00:00:00 2001 From: lipenghui Date: Mon, 17 Feb 2020 18:02:43 +0800 Subject: [PATCH 07/74] Avoid get partition metadata while the topic name is a partition name. (#6339) Motivation To avoid get partition metadata while the topic name is a partition name. Currently, if users want to skip all messages for a partitioned topic or unload a partitioned topic, the broker will call get topic metadata many times. For a topic with the partition name, it is not necessary to call get partitioned topic metadata again. --- .../admin/impl/PersistentTopicsBase.java | 934 +++++++++--------- 1 file changed, 493 insertions(+), 441 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index e33f94a3f744d..503b21b430d58 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -638,19 +638,15 @@ protected void internalDeletePartitionedTopic(AsyncResponse asyncResponse, boole zkSync(path); log.info("[{}] Deleted partitioned topic {}", clientAppId(), topicName); asyncResponse.resume(Response.noContent().build()); - return; } catch (KeeperException.NoNodeException nne) { asyncResponse.resume(new RestException(Status.NOT_FOUND, "Partitioned topic does not exist")); - return; } catch (KeeperException.BadVersionException e) { log.warn("[{}] Failed to delete partitioned topic {}: concurrent modification", clientAppId(), topicName); asyncResponse.resume(new RestException(Status.CONFLICT, "Concurrent modification")); - return; } catch (Exception e) { log.error("[{}] Failed to delete partitioned topic {}", clientAppId(), topicName, e); asyncResponse.resume(new RestException(e)); - return; } }); } @@ -660,57 +656,65 @@ protected void internalUnloadTopic(AsyncResponse asyncResponse, boolean authorit if (topicName.isGlobal()) { validateGlobalNamespaceOwnership(namespaceName); } - - getPartitionedTopicMetadataAsync(topicName, authoritative, false).whenComplete((meta, t) -> { - if (meta.partitions > 0) { - final List> futures = Lists.newArrayList(); - - for (int i = 0; i < meta.partitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - futures.add(pulsar().getAdminClient().topics().unloadAsync(topicNamePartition.toString())); - } catch (Exception e) { - log.error("[{}] Failed to unload topic {}", clientAppId(), topicNamePartition, e); - asyncResponse.resume(new RestException(e)); - return; + // If the topic name is a partition name, no need to get partition topic metadata again + if (topicName.isPartitioned()) { + internalUnloadNonPartitionedTopic(asyncResponse, authoritative); + } else { + getPartitionedTopicMetadataAsync(topicName, authoritative, false).whenComplete((meta, t) -> { + if (meta.partitions > 0) { + final List> futures = Lists.newArrayList(); + + for (int i = 0; i < meta.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + futures.add(pulsar().getAdminClient().topics().unloadAsync(topicNamePartition.toString())); + } catch (Exception e) { + log.error("[{}] Failed to unload topic {}", clientAppId(), topicNamePartition, e); + asyncResponse.resume(new RestException(e)); + return; + } } - } - FutureUtil.waitForAll(futures).handle((result, exception) -> { - if (exception != null) { - Throwable th = exception.getCause(); - if (th instanceof NotFoundException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, th.getMessage())); - } else { - log.error("[{}] Failed to unload topic {}", clientAppId(), topicName, exception); - asyncResponse.resume(new RestException(exception)); + FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable th = exception.getCause(); + if (th instanceof NotFoundException) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, th.getMessage())); + } else { + log.error("[{}] Failed to unload topic {}", clientAppId(), topicName, exception); + asyncResponse.resume(new RestException(exception)); + } + return null; } + + asyncResponse.resume(Response.noContent().build()); return null; - } + }); + } else { + internalUnloadNonPartitionedTopic(asyncResponse, authoritative); + } + }).exceptionally(t -> { + Throwable th = t.getCause(); + asyncResponse.resume(new RestException(th)); + return null; + }); + } + } - asyncResponse.resume(Response.noContent().build()); - return null; - }); - } else { - validateAdminAccessForTenant(topicName.getTenant()); - validateTopicOwnership(topicName, authoritative); + private void internalUnloadNonPartitionedTopic(AsyncResponse asyncResponse, boolean authoritative) { + validateAdminAccessForTenant(topicName.getTenant()); + validateTopicOwnership(topicName, authoritative); - Topic topic = getTopicReference(topicName); - topic.close(false).whenComplete((r, ex) -> { - if (ex != null) { - log.error("[{}] Failed to unload topic {}, {}", clientAppId(), topicName, ex.getMessage(), ex); - asyncResponse.resume(new RestException(ex)); + Topic topic = getTopicReference(topicName); + topic.close(false).whenComplete((r, ex) -> { + if (ex != null) { + log.error("[{}] Failed to unload topic {}, {}", clientAppId(), topicName, ex.getMessage(), ex); + asyncResponse.resume(new RestException(ex)); - } else { - log.info("[{}] Successfully unloaded topic {}", clientAppId(), topicName); - asyncResponse.resume(Response.noContent().build()); - } - }); + } else { + log.info("[{}] Successfully unloaded topic {}", clientAppId(), topicName); + asyncResponse.resume(Response.noContent().build()); } - }).exceptionally(t -> { - Throwable th = t.getCause(); - asyncResponse.resume(new RestException(th)); - return null; }); } @@ -753,25 +757,26 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut if (topicName.isGlobal()) { validateGlobalNamespaceOwnership(namespaceName); } - - final List subscriptions = Lists.newArrayList(); - - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - if (partitionMetadata.partitions > 0) { - try { - // get the subscriptions only from the 1st partition since all the other partitions will have the same - // subscriptions - pulsar().getAdminClient().topics().getSubscriptionsAsync(topicName.getPartition(0).toString()) + // If the topic name is a partition name, no need to get partition topic metadata again + if (topicName.isPartitioned()) { + internalGetSubscriptionsForNonPartitionedTopic(asyncResponse, authoritative); + } else { + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); + if (partitionMetadata.partitions > 0) { + try { + // get the subscriptions only from the 1st partition since all the other partitions will have the same + // subscriptions + pulsar().getAdminClient().topics().getSubscriptionsAsync(topicName.getPartition(0).toString()) .whenComplete((r, ex) -> { if (ex != null) { log.warn("[{}] Failed to get list of subscriptions for {}: {}", clientAppId(), - topicName, ex.getMessage()); + topicName, ex.getMessage()); if (ex instanceof PulsarAdminException) { PulsarAdminException pae = (PulsarAdminException) ex; if (pae.getStatusCode() == Status.NOT_FOUND.getStatusCode()) { asyncResponse.resume(new RestException(Status.NOT_FOUND, - "Internal topics have not been generated yet")); + "Internal topics have not been generated yet")); return; } else { asyncResponse.resume(new RestException(pae)); @@ -782,29 +787,34 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut return; } } - + final List subscriptions = Lists.newArrayList(); subscriptions.addAll(r); asyncResponse.resume(subscriptions); return; }); - } catch (Exception e) { - log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), topicName, e); - asyncResponse.resume(e); - return; + } catch (Exception e) { + log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), topicName, e); + asyncResponse.resume(e); + return; + } + } else { + internalGetSubscriptionsForNonPartitionedTopic(asyncResponse, authoritative); } - } else { - validateAdminOperationOnTopic(authoritative); - Topic topic = getTopicReference(topicName); + } + } - try { - topic.getSubscriptions().forEach((subName, sub) -> subscriptions.add(subName)); - asyncResponse.resume(subscriptions); - return; - } catch (Exception e) { - log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), topicName, e); - asyncResponse.resume(new RestException(e)); - return; - } + private void internalGetSubscriptionsForNonPartitionedTopic(AsyncResponse asyncResponse, boolean authoritative) { + validateAdminOperationOnTopic(authoritative); + Topic topic = getTopicReference(topicName); + try { + final List subscriptions = Lists.newArrayList(); + topic.getSubscriptions().forEach((subName, sub) -> subscriptions.add(subName)); + asyncResponse.resume(subscriptions); + return; + } catch (Exception e) { + log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), topicName, e); + asyncResponse.resume(new RestException(e)); + return; } } @@ -952,67 +962,72 @@ protected void internalDeleteSubscription(AsyncResponse asyncResponse, String su if (topicName.isGlobal()) { validateGlobalNamespaceOwnership(namespaceName); } - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - if (partitionMetadata.partitions > 0) { - final List> futures = Lists.newArrayList(); + // If the topic name is a partition name, no need to get partition topic metadata again + if (topicName.isPartitioned()) { + internalDeleteSubscriptionForNonPartitionedTopic(asyncResponse, subName, authoritative); + } else { + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); + if (partitionMetadata.partitions > 0) { + final List> futures = Lists.newArrayList(); - for (int i = 0; i < partitionMetadata.partitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - futures.add(pulsar().getAdminClient().topics() + for (int i = 0; i < partitionMetadata.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + futures.add(pulsar().getAdminClient().topics() .deleteSubscriptionAsync(topicNamePartition.toString(), subName)); - } catch (Exception e) { - log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicNamePartition, subName, + } catch (Exception e) { + log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicNamePartition, subName, e); - asyncResponse.resume(new RestException(e)); - return; + asyncResponse.resume(new RestException(e)); + return; + } } - } - FutureUtil.waitForAll(futures).handle((result, exception) -> { - if (exception != null) { - Throwable t = exception.getCause(); - if (t instanceof NotFoundException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); - return null; - } else if (t instanceof PreconditionFailedException) { - asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, + FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable t = exception.getCause(); + if (t instanceof NotFoundException) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); + return null; + } else if (t instanceof PreconditionFailedException) { + asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, "Subscription has active connected consumers")); - return null; - } else { - log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicName, subName, t); - asyncResponse.resume(new RestException(t)); - return null; + return null; + } else { + log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicName, subName, t); + asyncResponse.resume(new RestException(t)); + return null; + } } - } - asyncResponse.resume(Response.noContent().build()); - return null; - }); - } else { - validateAdminAccessForSubscriber(subName, authoritative); - Topic topic = getTopicReference(topicName); - try { - Subscription sub = topic.getSubscription(subName); - checkNotNull(sub); - sub.delete().get(); - log.info("[{}][{}] Deleted subscription {}", clientAppId(), topicName, subName); - asyncResponse.resume(Response.noContent().build()); - return; - } catch (Exception e) { - Throwable t = e.getCause(); - if (e instanceof NullPointerException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); - return; - } else if (t instanceof SubscriptionBusyException) { - asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, - "Subscription has active connected consumers")); - return; - } else { - log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicName, subName, e); - asyncResponse.resume(new RestException(t)); - return; - } + asyncResponse.resume(Response.noContent().build()); + return null; + }); + } else { + internalDeleteSubscriptionForNonPartitionedTopic(asyncResponse, subName, authoritative); + } + } + } + + private void internalDeleteSubscriptionForNonPartitionedTopic(AsyncResponse asyncResponse, String subName, boolean authoritative) { + validateAdminAccessForSubscriber(subName, authoritative); + Topic topic = getTopicReference(topicName); + try { + Subscription sub = topic.getSubscription(subName); + checkNotNull(sub); + sub.delete().get(); + log.info("[{}][{}] Deleted subscription {}", clientAppId(), topicName, subName); + asyncResponse.resume(Response.noContent().build()); + } catch (Exception e) { + Throwable t = e.getCause(); + if (e instanceof NullPointerException) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); + } else if (t instanceof SubscriptionBusyException) { + asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, + "Subscription has active connected consumers")); + } else { + log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicName, subName, e); + asyncResponse.resume(new RestException(t)); } } } @@ -1021,67 +1036,76 @@ protected void internalSkipAllMessages(AsyncResponse asyncResponse, String subNa if (topicName.isGlobal()) { validateGlobalNamespaceOwnership(namespaceName); } - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - if (partitionMetadata.partitions > 0) { - final List> futures = Lists.newArrayList(); + // If the topic name is a partition name, no need to get partition topic metadata again + if (topicName.isPartitioned()) { + internalSkipAllMessagesForNonPartitionedTopic(asyncResponse, subName, authoritative); + } else { + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); + if (partitionMetadata.partitions > 0) { + final List> futures = Lists.newArrayList(); - for (int i = 0; i < partitionMetadata.partitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - futures.add(pulsar().getAdminClient().topics().skipAllMessagesAsync(topicNamePartition.toString(), + for (int i = 0; i < partitionMetadata.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + futures.add(pulsar().getAdminClient().topics().skipAllMessagesAsync(topicNamePartition.toString(), subName)); - } catch (Exception e) { - log.error("[{}] Failed to skip all messages {} {}", clientAppId(), topicNamePartition, subName, e); - asyncResponse.resume(new RestException(e)); - return; + } catch (Exception e) { + log.error("[{}] Failed to skip all messages {} {}", clientAppId(), topicNamePartition, subName, e); + asyncResponse.resume(new RestException(e)); + return; + } } - } - FutureUtil.waitForAll(futures).handle((result, exception) -> { - if (exception != null) { - Throwable t = exception.getCause(); - if (t instanceof NotFoundException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); - return null; - } else { - log.error("[{}] Failed to skip all messages {} {}", clientAppId(), topicName, subName, t); - asyncResponse.resume(new RestException(t)); - return null; + FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable t = exception.getCause(); + if (t instanceof NotFoundException) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); + return null; + } else { + log.error("[{}] Failed to skip all messages {} {}", clientAppId(), topicName, subName, t); + asyncResponse.resume(new RestException(t)); + return null; + } } - } - asyncResponse.resume(Response.noContent().build()); - return null; - }); - } else { - validateAdminAccessForSubscriber(subName, authoritative); - PersistentTopic topic = (PersistentTopic) getTopicReference(topicName); - BiConsumer biConsumer = (v, ex) -> { - if (ex != null) { - asyncResponse.resume(new RestException(ex)); - log.error("[{}] Failed to skip all messages {} {}", clientAppId(), topicName, subName, ex); - } else { asyncResponse.resume(Response.noContent().build()); - log.info("[{}] Cleared backlog on {} {}", clientAppId(), topicName, subName); - } - }; - try { - if (subName.startsWith(topic.getReplicatorPrefix())) { - String remoteCluster = PersistentReplicator.getRemoteCluster(subName); - PersistentReplicator repl = (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); - checkNotNull(repl); - repl.clearBacklog().whenComplete(biConsumer); - } else { - PersistentSubscription sub = topic.getSubscription(subName); - checkNotNull(sub); - sub.clearBacklog().whenComplete(biConsumer); - } - } catch (Exception e) { - if (e instanceof NullPointerException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); - } else { - asyncResponse.resume(new RestException(e)); - } + return null; + }); + } else { + internalSkipAllMessagesForNonPartitionedTopic(asyncResponse, subName, authoritative); + } + } + } + + private void internalSkipAllMessagesForNonPartitionedTopic(AsyncResponse asyncResponse, String subName, boolean authoritative) { + validateAdminAccessForSubscriber(subName, authoritative); + PersistentTopic topic = (PersistentTopic) getTopicReference(topicName); + BiConsumer biConsumer = (v, ex) -> { + if (ex != null) { + asyncResponse.resume(new RestException(ex)); + log.error("[{}] Failed to skip all messages {} {}", clientAppId(), topicName, subName, ex); + } else { + asyncResponse.resume(Response.noContent().build()); + log.info("[{}] Cleared backlog on {} {}", clientAppId(), topicName, subName); + } + }; + try { + if (subName.startsWith(topic.getReplicatorPrefix())) { + String remoteCluster = PersistentReplicator.getRemoteCluster(subName); + PersistentReplicator repl = (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); + checkNotNull(repl); + repl.clearBacklog().whenComplete(biConsumer); + } else { + PersistentSubscription sub = topic.getSubscription(subName); + checkNotNull(sub); + sub.clearBacklog().whenComplete(biConsumer); + } + } catch (Exception e) { + if (e instanceof NullPointerException) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); + } else { + asyncResponse.resume(new RestException(e)); } } } @@ -1122,73 +1146,82 @@ protected void internalExpireMessagesForAllSubscriptions(AsyncResponse asyncResp if (topicName.isGlobal()) { validateGlobalNamespaceOwnership(namespaceName); } - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - if (partitionMetadata.partitions > 0) { - final List> futures = Lists.newArrayList(); + // If the topic name is a partition name, no need to get partition topic metadata again + if (topicName.isPartitioned()) { + internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(asyncResponse, expireTimeInSeconds, authoritative); + } else { + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); + if (partitionMetadata.partitions > 0) { + final List> futures = Lists.newArrayList(); - // expire messages for each partition topic - for (int i = 0; i < partitionMetadata.partitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - futures.add(pulsar().getAdminClient().topics().expireMessagesForAllSubscriptionsAsync( + // expire messages for each partition topic + for (int i = 0; i < partitionMetadata.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + futures.add(pulsar().getAdminClient().topics().expireMessagesForAllSubscriptionsAsync( topicNamePartition.toString(), expireTimeInSeconds)); - } catch (Exception e) { - log.error("[{}] Failed to expire messages up to {} on {}", clientAppId(), expireTimeInSeconds, + } catch (Exception e) { + log.error("[{}] Failed to expire messages up to {} on {}", clientAppId(), expireTimeInSeconds, topicNamePartition, e); - asyncResponse.resume(new RestException(e)); - return; + asyncResponse.resume(new RestException(e)); + return; + } } - } - FutureUtil.waitForAll(futures).handle((result, exception) -> { - if (exception != null) { - Throwable t = exception.getCause(); - log.error("[{}] Failed to expire messages up to {} on {}", clientAppId(), expireTimeInSeconds, + FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable t = exception.getCause(); + log.error("[{}] Failed to expire messages up to {} on {}", clientAppId(), expireTimeInSeconds, topicName, t); - asyncResponse.resume(new RestException(t)); - return null; - } + asyncResponse.resume(new RestException(t)); + return null; + } - asyncResponse.resume(Response.noContent().build()); - return null; - }); - } else { - // validate ownership and redirect if current broker is not owner - validateAdminOperationOnTopic(authoritative); + asyncResponse.resume(Response.noContent().build()); + return null; + }); + } else { + internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(asyncResponse, expireTimeInSeconds, authoritative); + } + } + } - PersistentTopic topic = (PersistentTopic) getTopicReference(topicName); - final AtomicReference exception = new AtomicReference<>(); + private void internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(AsyncResponse asyncResponse, int expireTimeInSeconds, + boolean authoritative) { + // validate ownership and redirect if current broker is not owner + validateAdminOperationOnTopic(authoritative); - topic.getReplicators().forEach((subName, replicator) -> { - try { - internalExpireMessagesForSinglePartition(subName, expireTimeInSeconds, authoritative); - } catch (Throwable t) { - exception.set(t); - } - }); + PersistentTopic topic = (PersistentTopic) getTopicReference(topicName); + final AtomicReference exception = new AtomicReference<>(); - topic.getSubscriptions().forEach((subName, subscriber) -> { - try { - internalExpireMessagesForSinglePartition(subName, expireTimeInSeconds, authoritative); - } catch (Throwable t) { - exception.set(t); - } - }); + topic.getReplicators().forEach((subName, replicator) -> { + try { + internalExpireMessagesForSinglePartition(subName, expireTimeInSeconds, authoritative); + } catch (Throwable t) { + exception.set(t); + } + }); - if (exception.get() != null) { - if (exception.get() instanceof WebApplicationException) { - WebApplicationException wae = (WebApplicationException) exception.get(); - asyncResponse.resume(wae); - return; - } else { - asyncResponse.resume(new RestException(exception.get())); - return; - } + topic.getSubscriptions().forEach((subName, subscriber) -> { + try { + internalExpireMessagesForSinglePartition(subName, expireTimeInSeconds, authoritative); + } catch (Throwable t) { + exception.set(t); } + }); - asyncResponse.resume(Response.noContent().build()); - return; + if (exception.get() != null) { + if (exception.get() instanceof WebApplicationException) { + WebApplicationException wae = (WebApplicationException) exception.get(); + asyncResponse.resume(wae); + return; + } else { + asyncResponse.resume(new RestException(exception.get())); + return; + } } + + asyncResponse.resume(Response.noContent().build()); } protected void internalResetCursor(AsyncResponse asyncResponse, String subName, long timestamp, @@ -1196,108 +1229,111 @@ protected void internalResetCursor(AsyncResponse asyncResponse, String subName, if (topicName.isGlobal()) { validateGlobalNamespaceOwnership(namespaceName); } - - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - final int numPartitions = partitionMetadata.partitions; - if (numPartitions > 0) { - final CompletableFuture future = new CompletableFuture<>(); - final AtomicInteger count = new AtomicInteger(numPartitions); - final AtomicInteger failureCount = new AtomicInteger(0); - final AtomicReference partitionException = new AtomicReference<>(); - - for (int i = 0; i < numPartitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - pulsar().getAdminClient().topics() + // If the topic name is a partition name, no need to get partition topic metadata again + if (topicName.isPartitioned()) { + internalResetCursorForNonPartitionedTopic(asyncResponse, subName, timestamp, authoritative); + } else { + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); + final int numPartitions = partitionMetadata.partitions; + if (numPartitions > 0) { + final CompletableFuture future = new CompletableFuture<>(); + final AtomicInteger count = new AtomicInteger(numPartitions); + final AtomicInteger failureCount = new AtomicInteger(0); + final AtomicReference partitionException = new AtomicReference<>(); + + for (int i = 0; i < numPartitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + pulsar().getAdminClient().topics() .resetCursorAsync(topicNamePartition.toString(), subName, timestamp).handle((r, ex) -> { - if (ex != null) { - if (ex instanceof PreconditionFailedException) { - // throw the last exception if all partitions get this error - // any other exception on partition is reported back to user - failureCount.incrementAndGet(); - partitionException.set(ex); - } else { - log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", - clientAppId(), topicNamePartition, subName, timestamp, ex); - future.completeExceptionally(ex); - return null; - } + if (ex != null) { + if (ex instanceof PreconditionFailedException) { + // throw the last exception if all partitions get this error + // any other exception on partition is reported back to user + failureCount.incrementAndGet(); + partitionException.set(ex); + } else { + log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", + clientAppId(), topicNamePartition, subName, timestamp, ex); + future.completeExceptionally(ex); + return null; } + } - if (count.decrementAndGet() == 0) { - future.complete(null); - } + if (count.decrementAndGet() == 0) { + future.complete(null); + } - return null; - }); - } catch (Exception e) { - log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), + return null; + }); + } catch (Exception e) { + log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), topicNamePartition, subName, timestamp, e); - future.completeExceptionally(e); + future.completeExceptionally(e); + } } - } - future.whenComplete((r, ex) -> { - if (ex != null) { - if (ex instanceof PulsarAdminException) { - asyncResponse.resume(new RestException((PulsarAdminException) ex)); - return; - } else { - asyncResponse.resume(new RestException(ex)); - return; + future.whenComplete((r, ex) -> { + if (ex != null) { + if (ex instanceof PulsarAdminException) { + asyncResponse.resume(new RestException((PulsarAdminException) ex)); + return; + } else { + asyncResponse.resume(new RestException(ex)); + return; + } } - } - // report an error to user if unable to reset for all partitions - if (failureCount.get() == numPartitions) { - log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), topicName, + // report an error to user if unable to reset for all partitions + if (failureCount.get() == numPartitions) { + log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), topicName, subName, timestamp, partitionException.get()); - asyncResponse.resume( + asyncResponse.resume( new RestException(Status.PRECONDITION_FAILED, partitionException.get().getMessage())); - return; - } else if (failureCount.get() > 0) { - log.warn("[{}] [{}] Partial errors for reset cursor on subscription {} to time {}", clientAppId(), + return; + } else if (failureCount.get() > 0) { + log.warn("[{}] [{}] Partial errors for reset cursor on subscription {} to time {}", clientAppId(), topicName, subName, timestamp, partitionException.get()); - } + } - asyncResponse.resume(Response.noContent().build()); - return; - }); - } else { - validateAdminAccessForSubscriber(subName, authoritative); - log.info("[{}] [{}] Received reset cursor on subscription {} to time {}", clientAppId(), topicName, subName, - timestamp); - PersistentTopic topic = (PersistentTopic) getTopicReference(topicName); - if (topic == null) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, "Topic not found")); - return; + asyncResponse.resume(Response.noContent().build()); + }); + } else { + internalResetCursorForNonPartitionedTopic(asyncResponse, subName, timestamp, authoritative); } - try { - PersistentSubscription sub = topic.getSubscription(subName); - checkNotNull(sub); - sub.resetCursor(timestamp).get(); - log.info("[{}] [{}] Reset cursor on subscription {} to time {}", clientAppId(), topicName, subName, - timestamp); - asyncResponse.resume(Response.noContent().build()); - return; - } catch (Exception e) { - Throwable t = e.getCause(); - log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), topicName, - subName, timestamp, e); - if (e instanceof NullPointerException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); - return; - } else if (e instanceof NotAllowedException) { - asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, e.getMessage())); - return; - } else if (t instanceof SubscriptionInvalidCursorPosition) { - asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, - "Unable to find position for timestamp specified -" + t.getMessage())); - return; - } else { - asyncResponse.resume(new RestException(e)); - return; - } + } + } + + private void internalResetCursorForNonPartitionedTopic(AsyncResponse asyncResponse, String subName, long timestamp, + boolean authoritative) { + validateAdminAccessForSubscriber(subName, authoritative); + log.info("[{}] [{}] Received reset cursor on subscription {} to time {}", clientAppId(), topicName, subName, + timestamp); + PersistentTopic topic = (PersistentTopic) getTopicReference(topicName); + if (topic == null) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Topic not found")); + return; + } + try { + PersistentSubscription sub = topic.getSubscription(subName); + checkNotNull(sub); + sub.resetCursor(timestamp).get(); + log.info("[{}] [{}] Reset cursor on subscription {} to time {}", clientAppId(), topicName, subName, + timestamp); + asyncResponse.resume(Response.noContent().build()); + } catch (Exception e) { + Throwable t = e.getCause(); + log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), topicName, + subName, timestamp, e); + if (e instanceof NullPointerException) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); + } else if (e instanceof NotAllowedException) { + asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, e.getMessage())); + } else if (t instanceof SubscriptionInvalidCursorPosition) { + asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, + "Unable to find position for timestamp specified -" + t.getMessage())); + } else { + asyncResponse.resume(new RestException(e)); } } } @@ -1310,27 +1346,30 @@ protected void internalCreateSubscription(AsyncResponse asyncResponse, String su final MessageIdImpl targetMessageId = messageId == null ? (MessageIdImpl) MessageId.earliest : messageId; log.info("[{}][{}] Creating subscription {} at message id {}", clientAppId(), topicName, subscriptionName, targetMessageId); - - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - final int numPartitions = partitionMetadata.partitions; - if (numPartitions > 0) { - final CompletableFuture future = new CompletableFuture<>(); - final AtomicInteger count = new AtomicInteger(numPartitions); - final AtomicInteger failureCount = new AtomicInteger(0); - final AtomicReference partitionException = new AtomicReference<>(); - - // Create the subscription on each partition - for (int i = 0; i < numPartitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - pulsar().getAdminClient().topics() + // If the topic name is a partition name, no need to get partition topic metadata again + if (topicName.isPartitioned()) { + internalCreateSubscriptionForNonPartitionedTopic(asyncResponse, subscriptionName, targetMessageId, authoritative, replicated); + } else { + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); + final int numPartitions = partitionMetadata.partitions; + if (numPartitions > 0) { + final CompletableFuture future = new CompletableFuture<>(); + final AtomicInteger count = new AtomicInteger(numPartitions); + final AtomicInteger failureCount = new AtomicInteger(0); + final AtomicReference partitionException = new AtomicReference<>(); + + // Create the subscription on each partition + for (int i = 0; i < numPartitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + pulsar().getAdminClient().topics() .createSubscriptionAsync(topicNamePartition.toString(), subscriptionName, targetMessageId) .handle((r, ex) -> { if (ex != null) { // fail the operation on unknown exception or if all the partitioned failed due to // subscription-already-exist if (failureCount.incrementAndGet() == numPartitions - || !(ex instanceof PulsarAdminException.ConflictException)) { + || !(ex instanceof PulsarAdminException.ConflictException)) { partitionException.set(ex); } } @@ -1341,75 +1380,79 @@ protected void internalCreateSubscription(AsyncResponse asyncResponse, String su return null; }); - } catch (Exception e) { - log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(), + } catch (Exception e) { + log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(), topicNamePartition, subscriptionName, targetMessageId, e); - future.completeExceptionally(e); + future.completeExceptionally(e); + } } - } - future.whenComplete((r, ex) -> { - if (ex != null) { - if (ex instanceof PulsarAdminException) { - asyncResponse.resume(new RestException((PulsarAdminException) ex)); - return; - } else { - asyncResponse.resume(new RestException(ex)); - return; + future.whenComplete((r, ex) -> { + if (ex != null) { + if (ex instanceof PulsarAdminException) { + asyncResponse.resume(new RestException((PulsarAdminException) ex)); + return; + } else { + asyncResponse.resume(new RestException(ex)); + return; + } } - } - if (partitionException.get() != null) { - log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(), topicName, + if (partitionException.get() != null) { + log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(), topicName, subscriptionName, targetMessageId, partitionException.get()); - if (partitionException.get() instanceof PulsarAdminException) { - asyncResponse.resume(new RestException((PulsarAdminException) partitionException.get())); - return; - } else { - asyncResponse.resume(new RestException(partitionException.get())); - return; + if (partitionException.get() instanceof PulsarAdminException) { + asyncResponse.resume(new RestException((PulsarAdminException) partitionException.get())); + return; + } else { + asyncResponse.resume(new RestException(partitionException.get())); + return; + } } - } - - asyncResponse.resume(Response.noContent().build()); - return; - }); - } else { - validateAdminAccessForSubscriber(subscriptionName, authoritative); - - PersistentTopic topic = (PersistentTopic) getOrCreateTopic(topicName); - if (topic.getSubscriptions().containsKey(subscriptionName)) { - asyncResponse.resume(new RestException(Status.CONFLICT, "Subscription already exists for topic")); - return; + asyncResponse.resume(Response.noContent().build()); + }); + } else { + internalCreateSubscriptionForNonPartitionedTopic(asyncResponse, subscriptionName, targetMessageId, authoritative, replicated); } + } + } - try { - PersistentSubscription subscription = (PersistentSubscription) topic - .createSubscription(subscriptionName, InitialPosition.Latest, replicated).get(); - // Mark the cursor as "inactive" as it was created without a real consumer connected - subscription.deactivateCursor(); - subscription.resetCursor(PositionImpl.get(targetMessageId.getLedgerId(), targetMessageId.getEntryId())) - .get(); - } catch (Throwable e) { - Throwable t = e.getCause(); - log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(), topicName, - subscriptionName, targetMessageId, e); - if (t instanceof SubscriptionInvalidCursorPosition) { - asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, - "Unable to find position for position specified: " + t.getMessage())); - return; - } else { - asyncResponse.resume(new RestException(e)); - return; - } - } + private void internalCreateSubscriptionForNonPartitionedTopic(AsyncResponse asyncResponse, String subscriptionName, + MessageIdImpl targetMessageId, boolean authoritative, boolean replicated) { + validateAdminAccessForSubscriber(subscriptionName, authoritative); - log.info("[{}][{}] Successfully created subscription {} at message id {}", clientAppId(), topicName, - subscriptionName, targetMessageId); - asyncResponse.resume(Response.noContent().build()); + PersistentTopic topic = (PersistentTopic) getOrCreateTopic(topicName); + + if (topic.getSubscriptions().containsKey(subscriptionName)) { + asyncResponse.resume(new RestException(Status.CONFLICT, "Subscription already exists for topic")); return; } + + try { + PersistentSubscription subscription = (PersistentSubscription) topic + .createSubscription(subscriptionName, InitialPosition.Latest, replicated).get(); + // Mark the cursor as "inactive" as it was created without a real consumer connected + subscription.deactivateCursor(); + subscription.resetCursor(PositionImpl.get(targetMessageId.getLedgerId(), targetMessageId.getEntryId())) + .get(); + } catch (Throwable e) { + Throwable t = e.getCause(); + log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(), topicName, + subscriptionName, targetMessageId, e); + if (t instanceof SubscriptionInvalidCursorPosition) { + asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, + "Unable to find position for position specified: " + t.getMessage())); + return; + } else { + asyncResponse.resume(new RestException(e)); + return; + } + } + + log.info("[{}][{}] Successfully created subscription {} at message id {}", clientAppId(), topicName, + subscriptionName, targetMessageId); + asyncResponse.resume(Response.noContent().build()); } protected void internalResetCursorOnPosition(String subName, boolean authoritative, MessageIdImpl messageId) { @@ -1418,10 +1461,8 @@ protected void internalResetCursorOnPosition(String subName, boolean authoritati } log.info("[{}][{}] received reset cursor on subscription {} to position {}", clientAppId(), topicName, subName, messageId); - - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - - if (partitionMetadata.partitions > 0) { + // If the topic name is a partition name, no need to get partition topic metadata again + if (!topicName.isPartitioned() && getPartitionedTopicMetadata(topicName, authoritative, false).partitions > 0) { log.warn("[{}] Not supported operation on partitioned-topic {} {}", clientAppId(), topicName, subName); throw new RestException(Status.METHOD_NOT_ALLOWED, @@ -1458,8 +1499,8 @@ protected Response internalPeekNthMessage(String subName, int messagePosition, b if (topicName.isGlobal()) { validateGlobalNamespaceOwnership(namespaceName); } - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - if (partitionMetadata.partitions > 0) { + // If the topic name is a partition name, no need to get partition topic metadata again + if (!topicName.isPartitioned() && getPartitionedTopicMetadata(topicName, authoritative, false).partitions > 0) { throw new RestException(Status.METHOD_NOT_ALLOWED, "Peek messages on a partitioned topic is not allowed"); } validateAdminAccessForSubscriber(subName, authoritative); @@ -1602,43 +1643,8 @@ protected void internalExpireMessages(AsyncResponse asyncResponse, String subNam if (topicName.isGlobal()) { validateGlobalNamespaceOwnership(namespaceName); } - - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - if (partitionMetadata.partitions > 0) { - final List> futures = Lists.newArrayList(); - - // expire messages for each partition topic - for (int i = 0; i < partitionMetadata.partitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - futures.add(pulsar().getAdminClient().topics().expireMessagesAsync(topicNamePartition.toString(), - subName, expireTimeInSeconds)); - } catch (Exception e) { - log.error("[{}] Failed to expire messages up to {} on {}", clientAppId(), expireTimeInSeconds, - topicNamePartition, e); - asyncResponse.resume(new RestException(e)); - return; - } - } - - FutureUtil.waitForAll(futures).handle((result, exception) -> { - if (exception != null) { - Throwable t = exception.getCause(); - if (t instanceof NotFoundException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); - return null; - } else { - log.error("[{}] Failed to expire messages up to {} on {}", clientAppId(), expireTimeInSeconds, - topicName, t); - asyncResponse.resume(new RestException(t)); - return null; - } - } - - asyncResponse.resume(Response.noContent().build()); - return null; - }); - } else { + // If the topic name is a partition name, no need to get partition topic metadata again + if (topicName.isPartitioned()) { try { internalExpireMessagesForSinglePartition(subName, expireTimeInSeconds, authoritative); } catch (WebApplicationException wae) { @@ -1649,7 +1655,54 @@ protected void internalExpireMessages(AsyncResponse asyncResponse, String subNam return; } asyncResponse.resume(Response.noContent().build()); - return; + } else { + PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); + if (partitionMetadata.partitions > 0) { + final List> futures = Lists.newArrayList(); + + // expire messages for each partition topic + for (int i = 0; i < partitionMetadata.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + futures.add(pulsar().getAdminClient().topics().expireMessagesAsync(topicNamePartition.toString(), + subName, expireTimeInSeconds)); + } catch (Exception e) { + log.error("[{}] Failed to expire messages up to {} on {}", clientAppId(), expireTimeInSeconds, + topicNamePartition, e); + asyncResponse.resume(new RestException(e)); + return; + } + } + + FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable t = exception.getCause(); + if (t instanceof NotFoundException) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); + return null; + } else { + log.error("[{}] Failed to expire messages up to {} on {}", clientAppId(), expireTimeInSeconds, + topicName, t); + asyncResponse.resume(new RestException(t)); + return null; + } + } + + asyncResponse.resume(Response.noContent().build()); + return null; + }); + } else { + try { + internalExpireMessagesForSinglePartition(subName, expireTimeInSeconds, authoritative); + } catch (WebApplicationException wae) { + asyncResponse.resume(wae); + return; + } catch (Exception e) { + asyncResponse.resume(new RestException(e)); + return; + } + asyncResponse.resume(Response.noContent().build()); + } } } @@ -1658,9 +1711,8 @@ private void internalExpireMessagesForSinglePartition(String subName, int expire if (topicName.isGlobal()) { validateGlobalNamespaceOwnership(namespaceName); } - - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - if (partitionMetadata.partitions > 0) { + // If the topic name is a partition name, no need to get partition topic metadata again + if (!topicName.isPartitioned() && getPartitionedTopicMetadata(topicName, authoritative, false).partitions > 0) { String msg = "This method should not be called for partitioned topic"; log.error("[{}] {} {} {}", clientAppId(), msg, topicName, subName); throw new IllegalStateException(msg); From 7d4df990798a1f46d26f973120de5896e188eddf Mon Sep 17 00:00:00 2001 From: liyuntao Date: Mon, 17 Feb 2020 18:41:06 +0800 Subject: [PATCH 08/74] explicit statement env 'BOOKIE_MEM' and 'BOOKIE_GC' for values-mini.yaml (#6340) Fixes #6338 ### Motivation This commit started while I was using helm in my local minikube, noticed that there's a mismatch between `values-mini.yaml` and `values.yaml` files. At first I thought it was a copy/paste error. So I created #6338; Then I looked into the details how these env-vars[ were used](https://github.com/apache/pulsar/blob/28875d5abc4cd13a3e9cc4f59524d2566d9f9f05/conf/bkenv.sh#L36), found out its ok to use `PULSAR_MEM` as an alternative. But it introduce problems: 1. Since `BOOKIE_GC` was not defined , the default [BOOKIE_EXTRA_OPTS](https://github.com/apache/pulsar/blob/28875d5abc4cd13a3e9cc4f59524d2566d9f9f05/conf/bkenv.sh#L39) will finally use default value of `BOOKIE_GC`, thus would cover same the JVM parameters defined prior in `PULSAR_MEM`. 2. May cause problems when bootstrap scripts changed in later dev, better to make it explicitly. So I create this pr to solve above problems(hidden trouble). ### Modifications As mentioned above, I've made such modifications below: 1. make `BOOKIE_MEM` and `BOOKIE_GC` explicit in `values-mini.yaml` file. Keep up with the format in`values.yaml` file. 2. remove all print-gc-logs related args. Considering the resource constraints of minikube environment. The removed part's content is `-XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:+PrintGCApplicationStoppedTime -XX:+PrintHeapAtGC -verbosegc -XX:G1LogLevel=finest` 3. leave `PULSAR_PREFIX_dbStorage_rocksDB_blockCacheSize` empty as usual, as [conf/standalone.conf#L576](https://github.com/apache/pulsar/blob/df152109415f2b10dd83e8afe50d9db7ab7cbad5/conf/standalone.conf#L576) says it would to use 10% of the direct memory size by default. --- deployment/kubernetes/helm/pulsar/values-mini.yaml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/deployment/kubernetes/helm/pulsar/values-mini.yaml b/deployment/kubernetes/helm/pulsar/values-mini.yaml index 49f65e047ae7e..796e232b5251c 100644 --- a/deployment/kubernetes/helm/pulsar/values-mini.yaml +++ b/deployment/kubernetes/helm/pulsar/values-mini.yaml @@ -181,7 +181,8 @@ bookkeeper: ## templates/bookkeeper-configmap.yaml ## configData: - PULSAR_MEM: "\"-Xms128m -Xmx256m -XX:MaxDirectMemorySize=128m -Dio.netty.leakDetectionLevel=disabled -Dio.netty.recycler.linkCapacity=1024 -XX:+UseG1GC -XX:MaxGCPauseMillis=10 -XX:+ParallelRefProcEnabled -XX:+UnlockExperimentalVMOptions -XX:+AggressiveOpts -XX:+DoEscapeAnalysis -XX:ParallelGCThreads=32 -XX:ConcGCThreads=32 -XX:G1NewSizePercent=50 -XX:+DisableExplicitGC -XX:-ResizePLAB -XX:+ExitOnOutOfMemoryError -XX:+PerfDisableSharedMem -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:+PrintGCApplicationStoppedTime -XX:+PrintHeapAtGC -verbosegc -XX:G1LogLevel=finest\"" + BOOKIE_MEM: "\"-Xms128m -Xmx256m -XX:MaxDirectMemorySize=128m -Dio.netty.leakDetectionLevel=disabled -Dio.netty.recycler.linkCapacity=1024 -XX:+ParallelRefProcEnabled -XX:+UnlockExperimentalVMOptions -XX:+AggressiveOpts -XX:+DoEscapeAnalysis -XX:ParallelGCThreads=32 -XX:ConcGCThreads=32 -XX:G1NewSizePercent=50 -XX:+DisableExplicitGC -XX:-ResizePLAB -XX:+ExitOnOutOfMemoryError -XX:+PerfDisableSharedMem\"" + BOOKIE_GC: "\"-XX:+UseG1GC -XX:MaxGCPauseMillis=10\"" PULSAR_PREFIX_dbStorage_writeCacheMaxSizeMb: "32" PULSAR_PREFIX_dbStorage_readAheadCacheMaxSizeMb: "32" PULSAR_PREFIX_journalMaxSizeMB: "2048" From 77971e4931924cb1c0b3fd1aada7948be6ecb211 Mon Sep 17 00:00:00 2001 From: lipenghui Date: Tue, 18 Feb 2020 00:51:10 +0800 Subject: [PATCH 09/74] Fix java doc for key shared policy. (#6341) The key shared policy does not support setting the maximum key hash range, so fix the java doc. --- .../main/java/org/apache/pulsar/client/api/ConsumerBuilder.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java index 50328ff04a392..0f26cabe626e5 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java @@ -542,7 +542,7 @@ public interface ConsumerBuilder extends Cloneable { *

Or *

      * client.newConsumer()
-     *          .keySharedPolicy(KeySharedPolicy.autoSplitHashRange().hashRangeTotal(100))
+     *          .keySharedPolicy(KeySharedPolicy.autoSplitHashRange())
      *          .subscribe();
      * 
* Details about auto split hash range policy, please see {@link KeySharedPolicy.KeySharedPolicyAutoSplit}. From 208af7cd7e718d378707df4aba600be3202e23df Mon Sep 17 00:00:00 2001 From: Jia Zhai Date: Tue, 18 Feb 2020 10:32:50 +0800 Subject: [PATCH 10/74] client: make SubscriptionMode a member of ConsumerConfigurationData (#6337) Currently, SubscriptionMode is a parameter to create ConsumerImpl, but it is not exported out, and user could not set this value for consumer. This change tries to make SubscriptionMode a member of ConsumerConfigurationData, so user could set this parameter when create consumer. --- .../pulsar/client/impl/RawReaderImpl.java | 1 - .../pulsar/client/api/ConsumerBuilder.java | 15 +++++++++ .../pulsar/client/api/SubscriptionMode.java | 31 ++++++++++++++++++ .../client/impl/ConsumerBuilderImpl.java | 8 +++++ .../pulsar/client/impl/ConsumerImpl.java | 32 +++++++++---------- .../client/impl/MultiTopicsConsumerImpl.java | 13 ++++---- .../pulsar/client/impl/PulsarClientImpl.java | 3 +- .../apache/pulsar/client/impl/ReaderImpl.java | 4 +-- .../client/impl/ZeroQueueConsumerImpl.java | 4 +-- .../impl/conf/ConsumerConfigurationData.java | 3 ++ .../pulsar/client/impl/ConsumerImplTest.java | 27 ++++++++++------ 11 files changed, 101 insertions(+), 40 deletions(-) create mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/api/SubscriptionMode.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java index 6a4d682d2b8ed..3ca072dda51aa 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java @@ -117,7 +117,6 @@ static class RawConsumerImpl extends ConsumerImpl { TopicName.getPartitionIndex(conf.getSingleTopic()), false, consumerFuture, - SubscriptionMode.Durable, MessageId.earliest, 0 /* startMessageRollbackDurationInSec */, Schema.BYTES, null, diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java index 0f26cabe626e5..ed1ba880e213a 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java @@ -228,6 +228,21 @@ public interface ConsumerBuilder extends Cloneable { */ ConsumerBuilder subscriptionType(SubscriptionType subscriptionType); + /** + * Select the subscription mode to be used when subscribing to the topic. + * + *

Options are: + *

    + *
  • {@link SubscriptionMode#Durable} (Default)
  • + *
  • {@link SubscriptionMode#NonDurable}
  • + *
+ * + * @param subscriptionMode + * the subscription mode value + * @return the consumer builder instance + */ + ConsumerBuilder subscriptionMode(SubscriptionMode subscriptionMode); + /** * Sets a {@link MessageListener} for the consumer * diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/SubscriptionMode.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/SubscriptionMode.java new file mode 100644 index 0000000000000..7e11bbb0dc7eb --- /dev/null +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/SubscriptionMode.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.api; + +/** + * Types of subscription mode supported by Pulsar. + */ +public enum SubscriptionMode { + // Make the subscription to be backed by a durable cursor that will retain messages and persist the current + // position + Durable, + + // Lightweight subscription mode that doesn't have a durable cursor associated + NonDurable +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java index 872abb7b24f43..e31cc84ec50b7 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java @@ -47,6 +47,7 @@ import org.apache.pulsar.client.api.PulsarClientException.InvalidConfigurationException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.api.SubscriptionMode; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.conf.ConfigurationDataUtils; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; @@ -191,6 +192,13 @@ public ConsumerBuilder subscriptionType(@NonNull SubscriptionType subscriptio return this; } + @Override + public ConsumerBuilder subscriptionMode(@NonNull SubscriptionMode subscriptionMode) { + conf.setSubscriptionMode(subscriptionMode); + return this; + } + + @Override public ConsumerBuilder messageListener(@NonNull MessageListener messageListener) { conf.setMessageListener(messageListener); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 919a280238676..5b62248370c54 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -63,6 +63,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.api.SubscriptionMode; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.PulsarClientException.TopicDoesNotExistException; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; @@ -150,39 +151,38 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle private final boolean createTopicIfDoesNotExist; - enum SubscriptionMode { - // Make the subscription to be backed by a durable cursor that will retain messages and persist the current - // position - Durable, - // Lightweight subscription mode that doesn't have a durable cursor associated - NonDurable - } - - static ConsumerImpl newConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurationData conf, - ExecutorService listenerExecutor, int partitionIndex, boolean hasParentConsumer, CompletableFuture> subscribeFuture, - SubscriptionMode subscriptionMode, MessageId startMessageId, Schema schema, ConsumerInterceptors interceptors, - boolean createTopicIfDoesNotExist) { + static ConsumerImpl newConsumerImpl(PulsarClientImpl client, + String topic, + ConsumerConfigurationData conf, + ExecutorService listenerExecutor, + int partitionIndex, + boolean hasParentConsumer, + CompletableFuture> subscribeFuture, + MessageId startMessageId, + Schema schema, + ConsumerInterceptors interceptors, + boolean createTopicIfDoesNotExist) { if (conf.getReceiverQueueSize() == 0) { return new ZeroQueueConsumerImpl<>(client, topic, conf, listenerExecutor, partitionIndex, hasParentConsumer, subscribeFuture, - subscriptionMode, startMessageId, schema, interceptors, + startMessageId, schema, interceptors, createTopicIfDoesNotExist); } else { return new ConsumerImpl<>(client, topic, conf, listenerExecutor, partitionIndex, hasParentConsumer, - subscribeFuture, subscriptionMode, startMessageId, 0 /* rollback time in sec to start msgId */, + subscribeFuture, startMessageId, 0 /* rollback time in sec to start msgId */, schema, interceptors, createTopicIfDoesNotExist); } } protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurationData conf, ExecutorService listenerExecutor, int partitionIndex, boolean hasParentConsumer, - CompletableFuture> subscribeFuture, SubscriptionMode subscriptionMode, MessageId startMessageId, + CompletableFuture> subscribeFuture, MessageId startMessageId, long startMessageRollbackDurationInSec, Schema schema, ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist) { super(client, topic, conf, conf.getReceiverQueueSize(), listenerExecutor, subscribeFuture, schema, interceptors); this.consumerId = client.newConsumerId(); - this.subscriptionMode = subscriptionMode; + this.subscriptionMode = conf.getSubscriptionMode(); this.startMessageId = startMessageId != null ? new BatchMessageIdImpl((MessageIdImpl) startMessageId) : null; this.lastDequeuedMessage = startMessageId == null ? MessageId.earliest : startMessageId; this.initialStartMessageId = this.startMessageId; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 9c95d17d9ea4e..3e9e69ae51c3c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -59,7 +59,6 @@ import org.apache.pulsar.client.api.PulsarClientException.NotSupportedException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; -import org.apache.pulsar.client.impl.ConsumerImpl.SubscriptionMode; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.client.util.ConsumerName; @@ -833,10 +832,10 @@ private void doSubscribeTopicPartitions(CompletableFuture subscribeResult, String partitionName = TopicName.get(topicName).getPartition(partitionIndex).toString(); CompletableFuture> subFuture = new CompletableFuture<>(); ConsumerImpl newConsumer = ConsumerImpl.newConsumerImpl(client, partitionName, - configurationData, client.externalExecutorProvider().getExecutor(), - partitionIndex, true, subFuture, - SubscriptionMode.Durable, null, schema, interceptors, - createIfDoesNotExist); + configurationData, client.externalExecutorProvider().getExecutor(), + partitionIndex, true, subFuture, + null, schema, interceptors, + createIfDoesNotExist); consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); return subFuture; }) @@ -847,7 +846,7 @@ private void doSubscribeTopicPartitions(CompletableFuture subscribeResult, CompletableFuture> subFuture = new CompletableFuture<>(); ConsumerImpl newConsumer = ConsumerImpl.newConsumerImpl(client, topicName, internalConfig, - client.externalExecutorProvider().getExecutor(), -1, true, subFuture, SubscriptionMode.Durable, null, + client.externalExecutorProvider().getExecutor(), -1, true, subFuture, null, schema, interceptors, createIfDoesNotExist); consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); @@ -1118,7 +1117,7 @@ private CompletableFuture subscribeIncreasedTopicPartitions(String topicNa ConsumerImpl newConsumer = ConsumerImpl.newConsumerImpl( client, partitionName, configurationData, client.externalExecutorProvider().getExecutor(), - partitionIndex, true, subFuture, SubscriptionMode.Durable, null, schema, interceptors, + partitionIndex, true, subFuture, null, schema, interceptors, true /* createTopicIfDoesNotExist */); consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); if (log.isDebugEnabled()) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 5a2a6dae09f60..f51fb6b65c3f3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -64,7 +64,6 @@ import org.apache.pulsar.client.api.schema.SchemaInfoProvider; import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.transaction.TransactionBuilder; -import org.apache.pulsar.client.impl.ConsumerImpl.SubscriptionMode; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; @@ -354,7 +353,7 @@ private CompletableFuture> doSingleTopicSubscribeAsync(ConsumerC } else { int partitionIndex = TopicName.getPartitionIndex(topic); consumer = ConsumerImpl.newConsumerImpl(PulsarClientImpl.this, topic, conf, listenerThread, partitionIndex, false, - consumerSubscribedFuture, SubscriptionMode.Durable, null, schema, interceptors, + consumerSubscribedFuture,null, schema, interceptors, true /* createTopicIfDoesNotExist */); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderImpl.java index 5022e11fc0814..4f172e39c8401 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderImpl.java @@ -26,7 +26,6 @@ import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.*; -import org.apache.pulsar.client.impl.ConsumerImpl.SubscriptionMode; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.impl.conf.ReaderConfigurationData; import org.apache.pulsar.common.naming.TopicName; @@ -47,6 +46,7 @@ public ReaderImpl(PulsarClientImpl client, ReaderConfigurationData readerConf consumerConfiguration.getTopicNames().add(readerConfiguration.getTopicName()); consumerConfiguration.setSubscriptionName(subscription); consumerConfiguration.setSubscriptionType(SubscriptionType.Exclusive); + consumerConfiguration.setSubscriptionMode(SubscriptionMode.NonDurable); consumerConfiguration.setReceiverQueueSize(readerConfiguration.getReceiverQueueSize()); consumerConfiguration.setReadCompacted(readerConfiguration.isReadCompacted()); @@ -91,7 +91,7 @@ public void reachedEndOfTopic(Consumer consumer) { final int partitionIdx = TopicName.getPartitionIndex(readerConfiguration.getTopicName()); consumer = new ConsumerImpl<>(client, readerConfiguration.getTopicName(), consumerConfiguration, - listenerExecutor, partitionIdx, false, consumerFuture, SubscriptionMode.NonDurable, + listenerExecutor, partitionIdx, false, consumerFuture, readerConfiguration.getStartMessageId(), readerConfiguration.getStartMessageFromRollbackDurationInSec(), schema, null, true /* createTopicIfDoesNotExist */); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java index b9249828bc1b1..94c8dd3c56ea4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java @@ -49,11 +49,11 @@ public class ZeroQueueConsumerImpl extends ConsumerImpl { public ZeroQueueConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurationData conf, ExecutorService listenerExecutor, int partitionIndex, boolean hasParentConsumer, CompletableFuture> subscribeFuture, - SubscriptionMode subscriptionMode, MessageId startMessageId, Schema schema, + MessageId startMessageId, Schema schema, ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist) { super(client, topic, conf, listenerExecutor, partitionIndex, hasParentConsumer, subscribeFuture, - subscriptionMode, startMessageId, 0 /* startMessageRollbackDurationInSec */, schema, interceptors, + startMessageId, 0 /* startMessageRollbackDurationInSec */, schema, interceptors, createTopicIfDoesNotExist); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java index 6195b5f720844..e64db7287fea8 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java @@ -43,6 +43,7 @@ import org.apache.pulsar.client.api.MessageListener; import org.apache.pulsar.client.api.RegexSubscriptionMode; import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.api.SubscriptionMode; import org.apache.pulsar.client.api.SubscriptionType; @Data @@ -59,6 +60,8 @@ public class ConsumerConfigurationData implements Serializable, Cloneable { private SubscriptionType subscriptionType = SubscriptionType.Exclusive; + private SubscriptionMode subscriptionMode = SubscriptionMode.Durable; + @JsonIgnore private MessageListener messageListener; diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java index a75765164f73c..dcb2e36e2ef22 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ConsumerImplTest.java @@ -18,25 +18,32 @@ */ package org.apache.pulsar.client.impl; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + import io.netty.util.Timer; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.impl.ConsumerImpl.SubscriptionMode; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.testng.Assert; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; - -import static org.mockito.Mockito.*; - public class ConsumerImplTest { @@ -62,7 +69,7 @@ public void setUp() { consumerConf.setSubscriptionName("test-sub"); consumer = ConsumerImpl.newConsumerImpl(client, topic, consumerConf, - executorService, -1, false, subscribeFuture, SubscriptionMode.Durable, null, null, null, + executorService, -1, false, subscribeFuture, null, null, null, true); } From 9b9e79e11acd01c9419af4726ae70ca3af10e074 Mon Sep 17 00:00:00 2001 From: Nick Rivera Date: Tue, 18 Feb 2020 10:08:06 -0800 Subject: [PATCH 11/74] Windows CMake corrections (#6336) * Corrected method of specifying Windows path to LLVM tools * Fixing windows build * Corrected the dll install path * Fixing pulsarShared paths --- pulsar-client-cpp/CMakeLists.txt | 11 ++++++----- pulsar-client-cpp/cmake_modules/FindClangTools.cmake | 8 ++++++-- pulsar-client-cpp/lib/CMakeLists.txt | 3 ++- 3 files changed, 14 insertions(+), 8 deletions(-) diff --git a/pulsar-client-cpp/CMakeLists.txt b/pulsar-client-cpp/CMakeLists.txt index 6357021ba2be3..0a9b15a8ca6d0 100644 --- a/pulsar-client-cpp/CMakeLists.txt +++ b/pulsar-client-cpp/CMakeLists.txt @@ -74,9 +74,9 @@ endif(NOT LOG_CATEGORY_NAME) add_definitions(-DLOG_CATEGORY_NAME=${LOG_CATEGORY_NAME} -DBUILDING_PULSAR -DBOOST_ALL_NO_LIB -DBOOST_ALLOW_DEPRECATED_HEADERS) if (LINK_STATIC) - find_library(ZLIB_LIBRARIES REQUIRED NAMES libz.a z) - find_library(Protobuf_LITE_LIBRARIES NAMES libprotobuf-lite.a) - find_library(CURL_LIBRARIES NAMES libcurl.a curl curl_a) + find_library(ZLIB_LIBRARIES REQUIRED NAMES libz.a z zlib) + find_library(Protobuf_LITE_LIBRARIES NAMES libprotobuf-lite.a libprotobuf-lite) + find_library(CURL_LIBRARIES NAMES libcurl.a curl curl_a libcurl_a) find_library(LIB_ZSTD NAMES libzstd.a) find_library(LIB_SNAPPY NAMES libsnappy.a) @@ -121,8 +121,9 @@ else() endif (NOT PROTOBUF_LIBRARIES) find_library(LIB_ZSTD zstd) - find_library(LIB_SNAPPY snappy libsnappy) - find_library(CURL_LIBRARIES curl) + find_library(LIB_SNAPPY NAMES snappy libsnappy) + find_library(CURL_LIBRARIES NAMES curl libcurl) + if (USE_LOG4CXX) find_library(LOG4CXX_LIBRARY_PATH log4cxx) find_path(LOG4CXX_INCLUDE_PATH log4cxx/logger.h) diff --git a/pulsar-client-cpp/cmake_modules/FindClangTools.cmake b/pulsar-client-cpp/cmake_modules/FindClangTools.cmake index 2d71c6b7cfbde..08d081a89eb7f 100644 --- a/pulsar-client-cpp/cmake_modules/FindClangTools.cmake +++ b/pulsar-client-cpp/cmake_modules/FindClangTools.cmake @@ -30,7 +30,10 @@ # CLANG_FORMAT_BIN, The path to the clang format binary # CLANG_TIDY_FOUND, Whether clang format was found -list(APPEND CLANG_SEARCH_PATHS ${ClangTools_PATH} $ENV{CLANG_TOOLS_PATH} /usr/local/bin /usr/bin $<$:"C:/Program Files/LLVM/bin">) +list(APPEND CLANG_SEARCH_PATHS ${ClangTools_PATH} $ENV{CLANG_TOOLS_PATH} /usr/local/bin /usr/bin) +if (WIN32) + list(APPEND CLANG_SEARCH_PATHS "C:/Program Files/LLVM/bin" "C:/Program Files (x86)/LLVM/bin") +endif() find_program(CLANG_TIDY_BIN NAMES clang-tidy-4.0 @@ -96,4 +99,5 @@ if ( "${CLANG_FORMAT_BIN}" STREQUAL "CLANG_FORMAT_BIN-NOTFOUND" ) else() set(CLANG_FORMAT_FOUND 1) message("clang-format found at ${CLANG_FORMAT_BIN}") -endif() \ No newline at end of file +endif() + diff --git a/pulsar-client-cpp/lib/CMakeLists.txt b/pulsar-client-cpp/lib/CMakeLists.txt index cb77b172e15b8..db514f1d2a505 100644 --- a/pulsar-client-cpp/lib/CMakeLists.txt +++ b/pulsar-client-cpp/lib/CMakeLists.txt @@ -120,5 +120,6 @@ else() install(TARGETS pulsarStatic DESTINATION lib) endif(LINK_STATIC) -install(TARGETS pulsarShared pulsarStatic DESTINATION lib) +install(TARGETS pulsarStatic DESTINATION lib) +install(TARGETS pulsarShared RUNTIME DESTINATION bin LIBRARY DESTINATION lib ARCHIVE DESTINATION lib) install(DIRECTORY "../include/pulsar" DESTINATION include) From 943c9035b77d9a11c77d2ba42c8970dc09d5d70b Mon Sep 17 00:00:00 2001 From: Neng Lu Date: Wed, 19 Feb 2020 11:43:31 -0800 Subject: [PATCH 12/74] remove future.join() from PulsarSinkEffectivelyOnceProcessor (#6361) --- .../main/java/org/apache/pulsar/functions/sink/PulsarSink.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java index 386c621ed1ba3..760845c05118f 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java @@ -234,7 +234,6 @@ public void sendOutputMessage(TypedMessageBuilder msg, Record record) { CompletableFuture future = msg.sendAsync(); future.thenAccept(messageId -> record.ack()).exceptionally(getPublishErrorHandler(record, true)); - future.join(); } } From 5177bfd3eba33534af6eb697909be9196b9ab247 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Fri, 21 Feb 2020 11:53:06 +0800 Subject: [PATCH 13/74] use checkout@v2 to avoid fatal: reference is not a tree (#6386) "fatal: reference is not a tree" is a known issue in actions/checkout#23 and fixed in checkout@v2, update checkout used in GitHub actions. --- .github/workflows/ci-cpp.yaml | 2 +- .github/workflows/ci-go-functions-style.yaml | 2 +- .github/workflows/ci-go-functions-test.yaml | 2 +- .github/workflows/ci-integration-backwards-compatibility.yaml | 2 +- .github/workflows/ci-integration-cli.yaml | 2 +- .github/workflows/ci-integration-function-state.yaml | 2 +- .github/workflows/ci-integration-messaging.yaml | 2 +- .github/workflows/ci-integration-process.yaml | 2 +- .github/workflows/ci-integration-schema.yaml | 2 +- .github/workflows/ci-integration-sql.yaml | 2 +- .github/workflows/ci-integration-standalone.yaml | 2 +- .github/workflows/ci-integration-thread.yaml | 2 +- .github/workflows/ci-integration-tiered-filesystem.yaml | 2 +- .github/workflows/ci-integration-tiered-jcloud.yaml | 2 +- .github/workflows/ci-license.yaml | 2 +- .github/workflows/ci-pulsarbot.yaml | 2 +- .github/workflows/ci-unit-adaptors.yml | 2 +- .github/workflows/ci-unit-broker-sasl.yml | 2 +- .github/workflows/ci-unit-broker.yml | 2 +- .github/workflows/ci-unit-flaky.yaml | 2 +- .github/workflows/ci-unit-proxy.yaml | 2 +- .github/workflows/ci-unit.yaml | 2 +- 22 files changed, 22 insertions(+), 22 deletions(-) diff --git a/.github/workflows/ci-cpp.yaml b/.github/workflows/ci-cpp.yaml index d339b7106e947..b8fc26307ee77 100644 --- a/.github/workflows/ci-cpp.yaml +++ b/.github/workflows/ci-cpp.yaml @@ -33,7 +33,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-go-functions-style.yaml b/.github/workflows/ci-go-functions-style.yaml index 7acb091fc0aa1..bf81d1275dd8b 100644 --- a/.github/workflows/ci-go-functions-style.yaml +++ b/.github/workflows/ci-go-functions-style.yaml @@ -32,7 +32,7 @@ jobs: runs-on: ubuntu-latest steps: - name: Check out code into the Go module directory - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-go-functions-test.yaml b/.github/workflows/ci-go-functions-test.yaml index 5609ad706aee5..4fa93926640a8 100644 --- a/.github/workflows/ci-go-functions-test.yaml +++ b/.github/workflows/ci-go-functions-test.yaml @@ -35,7 +35,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-backwards-compatibility.yaml b/.github/workflows/ci-integration-backwards-compatibility.yaml index aa827897ea07b..04708d876e8c0 100644 --- a/.github/workflows/ci-integration-backwards-compatibility.yaml +++ b/.github/workflows/ci-integration-backwards-compatibility.yaml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-cli.yaml b/.github/workflows/ci-integration-cli.yaml index f12f8f341f092..663fd1825db61 100644 --- a/.github/workflows/ci-integration-cli.yaml +++ b/.github/workflows/ci-integration-cli.yaml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-function-state.yaml b/.github/workflows/ci-integration-function-state.yaml index 548beecf9fbac..93f2e1340db67 100644 --- a/.github/workflows/ci-integration-function-state.yaml +++ b/.github/workflows/ci-integration-function-state.yaml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-messaging.yaml b/.github/workflows/ci-integration-messaging.yaml index 795ce5b8143fd..21faff3c9a999 100644 --- a/.github/workflows/ci-integration-messaging.yaml +++ b/.github/workflows/ci-integration-messaging.yaml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-process.yaml b/.github/workflows/ci-integration-process.yaml index 09ce922b75c24..389da0d9cdf6f 100644 --- a/.github/workflows/ci-integration-process.yaml +++ b/.github/workflows/ci-integration-process.yaml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-schema.yaml b/.github/workflows/ci-integration-schema.yaml index 0b73ddbea4686..b80ca18c3b384 100644 --- a/.github/workflows/ci-integration-schema.yaml +++ b/.github/workflows/ci-integration-schema.yaml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-sql.yaml b/.github/workflows/ci-integration-sql.yaml index c495eb720cea4..44d5d24f11427 100644 --- a/.github/workflows/ci-integration-sql.yaml +++ b/.github/workflows/ci-integration-sql.yaml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-standalone.yaml b/.github/workflows/ci-integration-standalone.yaml index 0347f1e53e049..184e002cce709 100644 --- a/.github/workflows/ci-integration-standalone.yaml +++ b/.github/workflows/ci-integration-standalone.yaml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-thread.yaml b/.github/workflows/ci-integration-thread.yaml index f650c553b54ca..8df3ed42f6554 100644 --- a/.github/workflows/ci-integration-thread.yaml +++ b/.github/workflows/ci-integration-thread.yaml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-tiered-filesystem.yaml b/.github/workflows/ci-integration-tiered-filesystem.yaml index 01b86c4a7d168..3803fb901f783 100644 --- a/.github/workflows/ci-integration-tiered-filesystem.yaml +++ b/.github/workflows/ci-integration-tiered-filesystem.yaml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-tiered-jcloud.yaml b/.github/workflows/ci-integration-tiered-jcloud.yaml index f06f1eec4f26c..78b6e052829a5 100644 --- a/.github/workflows/ci-integration-tiered-jcloud.yaml +++ b/.github/workflows/ci-integration-tiered-jcloud.yaml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-license.yaml b/.github/workflows/ci-license.yaml index 54672a2bcba10..ade35353ad502 100644 --- a/.github/workflows/ci-license.yaml +++ b/.github/workflows/ci-license.yaml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Set up JDK 1.8 uses: actions/setup-java@v1 diff --git a/.github/workflows/ci-pulsarbot.yaml b/.github/workflows/ci-pulsarbot.yaml index c00aaa5ee2492..a2263444eecb9 100644 --- a/.github/workflows/ci-pulsarbot.yaml +++ b/.github/workflows/ci-pulsarbot.yaml @@ -31,7 +31,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Execute pulsarbot command id: pulsarbot env: diff --git a/.github/workflows/ci-unit-adaptors.yml b/.github/workflows/ci-unit-adaptors.yml index 83ebcd218bd8e..2d9015f75b53c 100644 --- a/.github/workflows/ci-unit-adaptors.yml +++ b/.github/workflows/ci-unit-adaptors.yml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-unit-broker-sasl.yml b/.github/workflows/ci-unit-broker-sasl.yml index 57aceb9c8bf37..26c1009f99fd3 100644 --- a/.github/workflows/ci-unit-broker-sasl.yml +++ b/.github/workflows/ci-unit-broker-sasl.yml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-unit-broker.yml b/.github/workflows/ci-unit-broker.yml index 4538af439650f..40afadb4a6e9a 100644 --- a/.github/workflows/ci-unit-broker.yml +++ b/.github/workflows/ci-unit-broker.yml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-unit-flaky.yaml b/.github/workflows/ci-unit-flaky.yaml index 8d3f0fb5e8895..942750d5dfe64 100644 --- a/.github/workflows/ci-unit-flaky.yaml +++ b/.github/workflows/ci-unit-flaky.yaml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-unit-proxy.yaml b/.github/workflows/ci-unit-proxy.yaml index 83e3f3d1112e6..56803d75d6bfb 100644 --- a/.github/workflows/ci-unit-proxy.yaml +++ b/.github/workflows/ci-unit-proxy.yaml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-unit.yaml b/.github/workflows/ci-unit.yaml index b429d1639bcb0..d1be3663d21e0 100644 --- a/.github/workflows/ci-unit.yaml +++ b/.github/workflows/ci-unit.yaml @@ -32,7 +32,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v1 + uses: actions/checkout@v2 - name: Check if this pull request only changes documentation id: docs From 3197dcdb223d4ea8a2bc42bede63cb910eaceadf Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Fri, 21 Feb 2020 14:20:14 +0800 Subject: [PATCH 14/74] [Pulsar-Client] Stop shade snappy-java in pulsar-client-shaded (#6375) Fixes #6260 Snappy, like other compressions (LZ4, ZSTD), depends on native libraries to do the real encode/decode stuff. When we shade them in a fat jar, only the java implementations of snappy class are shaded, however, left the JNI incompatible with the underlying c++ code. We should just remove the shade for snappy, and let maven import its lib as a dependency. I've tested the shaded jar locally generated by this pr, it works for all compression codecs. --- pulsar-broker-shaded/pom.xml | 5 ---- pulsar-client-all/pom.xml | 5 ---- .../pulsar-client-kafka-shaded/pom.xml | 19 +++++--------- .../pulsar-client-kafka-shaded_0_8/pom.xml | 23 ++++++----------- .../pulsar-client-kafka-shaded_0_9/pom.xml | 25 +++++++------------ pulsar-client-shaded/pom.xml | 5 ---- 6 files changed, 23 insertions(+), 59 deletions(-) diff --git a/pulsar-broker-shaded/pom.xml b/pulsar-broker-shaded/pom.xml index 6cb8139187aff..4fcc51ae9e4c7 100644 --- a/pulsar-broker-shaded/pom.xml +++ b/pulsar-broker-shaded/pom.xml @@ -105,7 +105,6 @@ org.codehaus.jackson:jackson-core-asl org.codehaus.jackson:jackson-mapper-asl com.thoughtworks.paranamer:paranamer - org.xerial.snappy:snappy-java org.apache.commons:commons-compress org.tukaani:xz @@ -311,10 +310,6 @@ com.thoughtworks.paranamer org.apache.pulsar.shade.com.thoughtworks.paranamer - - org.xerial.snappy - org.apache.pulsar.shade.org.xerial.snappy - org.tukaani org.apache.pulsar.shade.org.tukaani diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 4d6165af972fe..c220fc0b60f61 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -147,7 +147,6 @@ org.codehaus.jackson:jackson-core-asl org.codehaus.jackson:jackson-mapper-asl com.thoughtworks.paranamer:paranamer - org.xerial.snappy:snappy-java org.apache.commons:commons-compress org.tukaani:xz org.apache.bookkeeper:bookkeeper-common-allocator @@ -312,10 +311,6 @@ com.thoughtworks.paranamer org.apache.pulsar.shade.com.thoughtworks.paranamer - - org.xerial.snappy - org.apache.pulsar.shade.org.xerial.snappy - org.apache.commons org.apache.pulsar.shade.org.apache.commons diff --git a/pulsar-client-kafka-compat/pulsar-client-kafka-shaded/pom.xml b/pulsar-client-kafka-compat/pulsar-client-kafka-shaded/pom.xml index 383203e52ebe3..cbba893521fec 100644 --- a/pulsar-client-kafka-compat/pulsar-client-kafka-shaded/pom.xml +++ b/pulsar-client-kafka-compat/pulsar-client-kafka-shaded/pom.xml @@ -103,7 +103,6 @@ org.codehaus.jackson:jackson-core-asl org.codehaus.jackson:jackson-mapper-asl com.thoughtworks.paranamer:paranamer - org.xerial.snappy:snappy-java org.apache.commons:commons-compress org.tukaani:xz @@ -115,6 +114,12 @@ ** + + org.apache.pulsar:pulsar-client-original + + ** + + @@ -217,10 +222,6 @@ com.thoughtworks.paranamer org.apache.pulsar.shade.com.thoughtworks.paranamer - - org.xerial.snappy - org.apache.pulsar.shade.org.xerial.snappy - org.apache.commons org.apache.pulsar.shade.org.apache.commons @@ -234,14 +235,6 @@ org.apache.pulsar.shade.org.apache.bookkeeper - - - org.apache.pulsar:pulsar-client-original - - ** - - - diff --git a/pulsar-client-kafka-compat/pulsar-client-kafka-shaded_0_8/pom.xml b/pulsar-client-kafka-compat/pulsar-client-kafka-shaded_0_8/pom.xml index 29b8e46c97c08..ffaed560ce884 100644 --- a/pulsar-client-kafka-compat/pulsar-client-kafka-shaded_0_8/pom.xml +++ b/pulsar-client-kafka-compat/pulsar-client-kafka-shaded_0_8/pom.xml @@ -103,18 +103,23 @@ org.codehaus.jackson:jackson-core-asl org.codehaus.jackson:jackson-mapper-asl com.thoughtworks.paranamer:paranamer - org.xerial.snappy:snappy-java org.apache.commons:commons-compress org.tukaani:xz - + commons-logging:commons-logging ** - + + + org.apache.pulsar:pulsar-client-original + + ** + + @@ -375,10 +380,6 @@ com.thoughtworks.paranamer org.apache.pulsar.shade.com.thoughtworks.paranamer - - org.xerial.snappy - org.apache.pulsar.shade.org.xerial.snappy - org.apache.commons org.apache.pulsar.shade.org.apache.commons @@ -392,14 +393,6 @@ org.apache.pulsar.shade.org.apache.bookkeeper - - - org.apache.pulsar:pulsar-client-original - - ** - - - diff --git a/pulsar-client-kafka-compat/pulsar-client-kafka-shaded_0_9/pom.xml b/pulsar-client-kafka-compat/pulsar-client-kafka-shaded_0_9/pom.xml index 00bdb184af9db..954af492f782e 100644 --- a/pulsar-client-kafka-compat/pulsar-client-kafka-shaded_0_9/pom.xml +++ b/pulsar-client-kafka-compat/pulsar-client-kafka-shaded_0_9/pom.xml @@ -99,18 +99,23 @@ org.codehaus.jackson:jackson-core-asl org.codehaus.jackson:jackson-mapper-asl com.thoughtworks.paranamer:paranamer - org.xerial.snappy:snappy-java org.apache.commons:commons-compress org.tukaani:xz - + commons-logging:commons-logging - ** + ** - + + + org.apache.pulsar:pulsar-client-original + + ** + + @@ -213,10 +218,6 @@ com.thoughtworks.paranamer org.apache.pulsar.shade.com.thoughtworks.paranamer - - org.xerial.snappy - org.apache.pulsar.shade.org.xerial.snappy - org.apache.commons org.apache.pulsar.shade.org.apache.commons @@ -230,14 +231,6 @@ org.apache.pulsar.shade.org.apache.bookkeeper - - - org.apache.pulsar:pulsar-client-original - - ** - - - diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index ae45b365950f2..3ef9bf3bb800d 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -137,7 +137,6 @@ org.codehaus.jackson:jackson-core-asl org.codehaus.jackson:jackson-mapper-asl com.thoughtworks.paranamer:paranamer - org.xerial.snappy:snappy-java org.apache.commons:commons-compress org.tukaani:xz @@ -250,10 +249,6 @@ com.thoughtworks.paranamer org.apache.pulsar.shade.com.thoughtworks.paranamer - - org.xerial.snappy - org.apache.pulsar.shade.org.xerial.snappy - org.apache.commons org.apache.pulsar.shade.org.apache.commons From bb5d6fdb0417ec33c0b62c9fafd3622647dbe6bd Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Sat, 22 Feb 2020 13:21:30 +0800 Subject: [PATCH 15/74] Fix CI not triggered (#6397) In #6386 , checkout@v2 is brought in for checkout. However, it's checking out PR merge commit by default, therefore breaks diff-only action which looking for commits that a PR is based on. And make all tests skipped. This PR fixes this issue. and has been proven to work with https://github.com/apache/pulsar/pull/6396 Brokers/unit-tests. --- .github/workflows/ci-cpp.yaml | 3 +++ .github/workflows/ci-go-functions-style.yaml | 3 +++ .github/workflows/ci-go-functions-test.yaml | 3 +++ .github/workflows/ci-integration-backwards-compatibility.yaml | 3 +++ .github/workflows/ci-integration-cli.yaml | 3 +++ .github/workflows/ci-integration-function-state.yaml | 3 +++ .github/workflows/ci-integration-messaging.yaml | 3 +++ .github/workflows/ci-integration-process.yaml | 3 +++ .github/workflows/ci-integration-schema.yaml | 3 +++ .github/workflows/ci-integration-sql.yaml | 3 +++ .github/workflows/ci-integration-standalone.yaml | 3 +++ .github/workflows/ci-integration-thread.yaml | 3 +++ .github/workflows/ci-integration-tiered-filesystem.yaml | 3 +++ .github/workflows/ci-integration-tiered-jcloud.yaml | 3 +++ .github/workflows/ci-license.yaml | 3 +++ .github/workflows/ci-pulsarbot.yaml | 3 +++ .github/workflows/ci-unit-adaptors.yml | 3 +++ .github/workflows/ci-unit-broker-sasl.yml | 3 +++ .github/workflows/ci-unit-broker.yml | 3 +++ .github/workflows/ci-unit-flaky.yaml | 3 +++ .github/workflows/ci-unit-proxy.yaml | 3 +++ .github/workflows/ci-unit.yaml | 3 +++ 22 files changed, 66 insertions(+) diff --git a/.github/workflows/ci-cpp.yaml b/.github/workflows/ci-cpp.yaml index b8fc26307ee77..06fd1471840b5 100644 --- a/.github/workflows/ci-cpp.yaml +++ b/.github/workflows/ci-cpp.yaml @@ -34,6 +34,9 @@ jobs: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-go-functions-style.yaml b/.github/workflows/ci-go-functions-style.yaml index bf81d1275dd8b..549875df72976 100644 --- a/.github/workflows/ci-go-functions-style.yaml +++ b/.github/workflows/ci-go-functions-style.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: Check out code into the Go module directory uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-go-functions-test.yaml b/.github/workflows/ci-go-functions-test.yaml index 4fa93926640a8..a87a81d158deb 100644 --- a/.github/workflows/ci-go-functions-test.yaml +++ b/.github/workflows/ci-go-functions-test.yaml @@ -36,6 +36,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-backwards-compatibility.yaml b/.github/workflows/ci-integration-backwards-compatibility.yaml index 04708d876e8c0..3cdd7c01de0cc 100644 --- a/.github/workflows/ci-integration-backwards-compatibility.yaml +++ b/.github/workflows/ci-integration-backwards-compatibility.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-cli.yaml b/.github/workflows/ci-integration-cli.yaml index 663fd1825db61..7ec6eed59cb8c 100644 --- a/.github/workflows/ci-integration-cli.yaml +++ b/.github/workflows/ci-integration-cli.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-function-state.yaml b/.github/workflows/ci-integration-function-state.yaml index 93f2e1340db67..9af9996ca54c2 100644 --- a/.github/workflows/ci-integration-function-state.yaml +++ b/.github/workflows/ci-integration-function-state.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-messaging.yaml b/.github/workflows/ci-integration-messaging.yaml index 21faff3c9a999..6d71d43ced6fd 100644 --- a/.github/workflows/ci-integration-messaging.yaml +++ b/.github/workflows/ci-integration-messaging.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-process.yaml b/.github/workflows/ci-integration-process.yaml index 389da0d9cdf6f..08ef689a4f002 100644 --- a/.github/workflows/ci-integration-process.yaml +++ b/.github/workflows/ci-integration-process.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-schema.yaml b/.github/workflows/ci-integration-schema.yaml index b80ca18c3b384..4b9be720d49a2 100644 --- a/.github/workflows/ci-integration-schema.yaml +++ b/.github/workflows/ci-integration-schema.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-sql.yaml b/.github/workflows/ci-integration-sql.yaml index 44d5d24f11427..a613fd4f0dc95 100644 --- a/.github/workflows/ci-integration-sql.yaml +++ b/.github/workflows/ci-integration-sql.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-standalone.yaml b/.github/workflows/ci-integration-standalone.yaml index 184e002cce709..b743fae58406d 100644 --- a/.github/workflows/ci-integration-standalone.yaml +++ b/.github/workflows/ci-integration-standalone.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-thread.yaml b/.github/workflows/ci-integration-thread.yaml index 8df3ed42f6554..b01444f9bc379 100644 --- a/.github/workflows/ci-integration-thread.yaml +++ b/.github/workflows/ci-integration-thread.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-tiered-filesystem.yaml b/.github/workflows/ci-integration-tiered-filesystem.yaml index 3803fb901f783..ede7f877e4b7a 100644 --- a/.github/workflows/ci-integration-tiered-filesystem.yaml +++ b/.github/workflows/ci-integration-tiered-filesystem.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-integration-tiered-jcloud.yaml b/.github/workflows/ci-integration-tiered-jcloud.yaml index 78b6e052829a5..d36cc5a07aac1 100644 --- a/.github/workflows/ci-integration-tiered-jcloud.yaml +++ b/.github/workflows/ci-integration-tiered-jcloud.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-license.yaml b/.github/workflows/ci-license.yaml index ade35353ad502..2cc08d14a2f97 100644 --- a/.github/workflows/ci-license.yaml +++ b/.github/workflows/ci-license.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Set up JDK 1.8 uses: actions/setup-java@v1 diff --git a/.github/workflows/ci-pulsarbot.yaml b/.github/workflows/ci-pulsarbot.yaml index a2263444eecb9..d068d5aa444bd 100644 --- a/.github/workflows/ci-pulsarbot.yaml +++ b/.github/workflows/ci-pulsarbot.yaml @@ -32,6 +32,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Execute pulsarbot command id: pulsarbot env: diff --git a/.github/workflows/ci-unit-adaptors.yml b/.github/workflows/ci-unit-adaptors.yml index 2d9015f75b53c..988983b6a6c38 100644 --- a/.github/workflows/ci-unit-adaptors.yml +++ b/.github/workflows/ci-unit-adaptors.yml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-unit-broker-sasl.yml b/.github/workflows/ci-unit-broker-sasl.yml index 26c1009f99fd3..4eecc8af22637 100644 --- a/.github/workflows/ci-unit-broker-sasl.yml +++ b/.github/workflows/ci-unit-broker-sasl.yml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-unit-broker.yml b/.github/workflows/ci-unit-broker.yml index 40afadb4a6e9a..4a7203c9466db 100644 --- a/.github/workflows/ci-unit-broker.yml +++ b/.github/workflows/ci-unit-broker.yml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-unit-flaky.yaml b/.github/workflows/ci-unit-flaky.yaml index 942750d5dfe64..62f096545237a 100644 --- a/.github/workflows/ci-unit-flaky.yaml +++ b/.github/workflows/ci-unit-flaky.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-unit-proxy.yaml b/.github/workflows/ci-unit-proxy.yaml index 56803d75d6bfb..bd2875f5891f9 100644 --- a/.github/workflows/ci-unit-proxy.yaml +++ b/.github/workflows/ci-unit-proxy.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs diff --git a/.github/workflows/ci-unit.yaml b/.github/workflows/ci-unit.yaml index d1be3663d21e0..c793d8fe8790e 100644 --- a/.github/workflows/ci-unit.yaml +++ b/.github/workflows/ci-unit.yaml @@ -33,6 +33,9 @@ jobs: steps: - name: checkout uses: actions/checkout@v2 + with: + fetch-depth: 0 + ref: ${{ github.event.pull_request.head.sha }} - name: Check if this pull request only changes documentation id: docs From af4773bf7bf56a1306ed44a6084cb4491d7d3943 Mon Sep 17 00:00:00 2001 From: John Harris Date: Fri, 21 Feb 2020 22:07:10 -0800 Subject: [PATCH 16/74] [Issue 6355][HELM] autorecovery - could not find or load main class (#6373) This applies the recommended fix from https://github.com/apache/pulsar/issues/6355#issuecomment-587756717 Fixes #6355 ### Motivation This PR corrects the configmap data which was causing the autorecovery pod to crashloop with `could not find or load main class` ### Modifications Updated the configmap var data per [this comment](https://github.com/apache/pulsar/issues/6355#issuecomment-587756717) from @sijie --- deployment/kubernetes/helm/pulsar/values-mini.yaml | 2 +- deployment/kubernetes/helm/pulsar/values.yaml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/deployment/kubernetes/helm/pulsar/values-mini.yaml b/deployment/kubernetes/helm/pulsar/values-mini.yaml index 796e232b5251c..375f8df69223e 100644 --- a/deployment/kubernetes/helm/pulsar/values-mini.yaml +++ b/deployment/kubernetes/helm/pulsar/values-mini.yaml @@ -317,7 +317,7 @@ autoRecovery: ## templates/autorecovery-configmap.yaml ## configData: - PULSAR_MEM: "\" -Xms64m -Xmx128m \"" + BOOKIE_MEM: "\" -Xms64m -Xmx128m \"" ## Pulsar Extra: Dashboard ## templates/dashboard-deployment.yaml diff --git a/deployment/kubernetes/helm/pulsar/values.yaml b/deployment/kubernetes/helm/pulsar/values.yaml index d9fc5025805f9..afe0d5c3a7e30 100644 --- a/deployment/kubernetes/helm/pulsar/values.yaml +++ b/deployment/kubernetes/helm/pulsar/values.yaml @@ -318,7 +318,7 @@ autoRecovery: ## templates/autorecovery-configmap.yaml ## configData: - PULSAR_MEM: "\" -Xms1g -Xmx1g \"" + BOOKIE_MEM: "\" -Xms1g -Xmx1g \"" ## Pulsar Extra: Dashboard ## templates/dashboard-deployment.yaml From 336e971f4d41d6ffb26b3b53a20f36a360c070e8 Mon Sep 17 00:00:00 2001 From: lipenghui Date: Sun, 23 Feb 2020 19:43:48 +0800 Subject: [PATCH 17/74] Creating a topic does not wait for creating cursor of replicators (#6364) ### Motivation Creating a topic does not wait for creating cursor of replicators ## Verifying this change The exists unit test can cover this change --- .../broker/service/AbstractReplicator.java | 38 ++++++---- .../NonPersistentReplicator.java | 5 ++ .../persistent/PersistentReplicator.java | 71 +++++++++++++++++-- .../service/persistent/PersistentTopic.java | 37 ++++------ 4 files changed, 108 insertions(+), 43 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java index d9b2f8eee24fa..13cd0918ab58a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java @@ -122,30 +122,38 @@ public synchronized void startProducer() { log.info("[{}][{} -> {}] Replicator already being started. Replicator state: {}", topicName, localCluster, remoteCluster, state); } - return; } log.info("[{}][{} -> {}] Starting replicator", topicName, localCluster, remoteCluster); - producerBuilder.createAsync().thenAccept(producer -> { - readEntries(producer); - }).exceptionally(ex -> { - if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Stopped)) { - long waitTimeMs = backOff.next(); - log.warn("[{}][{} -> {}] Failed to create remote producer ({}), retrying in {} s", topicName, - localCluster, remoteCluster, ex.getMessage(), waitTimeMs / 1000.0); - - // BackOff before retrying - brokerService.executor().schedule(this::startProducer, waitTimeMs, TimeUnit.MILLISECONDS); - } else { - log.warn("[{}][{} -> {}] Failed to create remote producer. Replicator state: {}", topicName, - localCluster, remoteCluster, STATE_UPDATER.get(this), ex); - } + openCursorAsync().thenAccept(v -> + producerBuilder.createAsync() + .thenAccept(this::readEntries) + .exceptionally(ex -> { + retryCreateProducer(ex); + return null; + })).exceptionally(ex -> { + retryCreateProducer(ex); return null; }); + } + private void retryCreateProducer(Throwable ex) { + if (STATE_UPDATER.compareAndSet(this, State.Starting, State.Stopped)) { + long waitTimeMs = backOff.next(); + log.warn("[{}][{} -> {}] Failed to create remote producer ({}), retrying in {} s", topicName, + localCluster, remoteCluster, ex.getMessage(), waitTimeMs / 1000.0); + + // BackOff before retrying + brokerService.executor().schedule(this::startProducer, waitTimeMs, TimeUnit.MILLISECONDS); + } else { + log.warn("[{}][{} -> {}] Failed to create remote producer. Replicator state: {}", topicName, + localCluster, remoteCluster, STATE_UPDATER.get(this), ex); + } } + protected abstract CompletableFuture openCursorAsync(); + protected synchronized CompletableFuture closeProducerAsync() { if (producer == null) { STATE_UPDATER.set(this, State.Stopped); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java index b6ea53ae9c682..c109560f9f8ec 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentReplicator.java @@ -251,6 +251,11 @@ protected void disableReplicatorRead() { // No-op } + @Override + protected CompletableFuture openCursorAsync() { + return CompletableFuture.completedFuture(null); + } + @Override public boolean isConnected() { ProducerImpl producer = this.producer; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java index c4a19c460b529..2e95eb6654f66 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentReplicator.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service.persistent; import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAGE_RATE_BACKOFF_MS; + import io.netty.buffer.ByteBuf; import io.netty.util.Recycler; import io.netty.util.Recycler.Handle; @@ -33,11 +34,13 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.ClearBacklogCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCallback; +import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenCursorCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedCursor.IndividualDeletedEntries; +import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerException.CursorAlreadyClosedException; import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; @@ -46,6 +49,7 @@ import org.apache.pulsar.broker.service.AbstractReplicator; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerServiceException.NamingException; +import org.apache.pulsar.broker.service.BrokerServiceException.PersistenceException; import org.apache.pulsar.broker.service.BrokerServiceException.TopicBusyException; import org.apache.pulsar.broker.service.Replicator; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; @@ -55,6 +59,7 @@ import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.SendCallback; +import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.api.proto.PulsarMarkers.MarkerType; import org.apache.pulsar.common.policies.data.ReplicatorStats; @@ -65,7 +70,9 @@ public class PersistentReplicator extends AbstractReplicator implements Replicator, ReadEntriesCallback, DeleteCallback { private final PersistentTopic topic; - private final ManagedCursor cursor; + private final String replicatorName; + private final ManagedLedger ledger; + protected ManagedCursor cursor; private Optional dispatchRateLimiter = Optional.empty(); @@ -97,11 +104,14 @@ public class PersistentReplicator extends AbstractReplicator implements Replicat private final ReplicatorStats stats = new ReplicatorStats(); + // Only for test public PersistentReplicator(PersistentTopic topic, ManagedCursor cursor, String localCluster, String remoteCluster, BrokerService brokerService) throws NamingException { super(topic.getName(), topic.getReplicatorPrefix(), localCluster, remoteCluster, brokerService); - this.topic = topic; + this.replicatorName = cursor.getName(); + this.ledger = cursor.getManagedLedger(); this.cursor = cursor; + this.topic = topic; this.expiryMonitor = new PersistentMessageExpiryMonitor(topicName, Codec.decode(cursor.getName()), cursor); HAVE_PENDING_READ_UPDATER.set(this, FALSE); PENDING_MESSAGES_UPDATER.set(this, 0); @@ -116,6 +126,25 @@ public PersistentReplicator(PersistentTopic topic, ManagedCursor cursor, String startProducer(); } + public PersistentReplicator(PersistentTopic topic, String replicatorName, String localCluster, String remoteCluster, + BrokerService brokerService, ManagedLedger ledger) throws NamingException { + super(topic.getName(), topic.getReplicatorPrefix(), localCluster, remoteCluster, brokerService); + this.replicatorName = replicatorName; + this.ledger = ledger; + this.topic = topic; + HAVE_PENDING_READ_UPDATER.set(this, FALSE); + PENDING_MESSAGES_UPDATER.set(this, 0); + + readBatchSize = Math.min( + producerQueueSize, + topic.getBrokerService().pulsar().getConfiguration().getDispatcherMaxReadBatchSize()); + producerQueueThreshold = (int) (producerQueueSize * 0.9); + + this.initializeDispatchRateLimiterIfNeeded(Optional.empty()); + + startProducer(); + } + @Override protected void readEntries(org.apache.pulsar.client.api.Producer producer) { // Rewind the cursor to be sure to read again all non-acked messages sent while restarting @@ -158,6 +187,36 @@ protected void disableReplicatorRead() { this.cursor.setInactive(); } + @Override + protected synchronized CompletableFuture openCursorAsync() { + log.info("[{}][{} -> {}] Starting open cursor for replicator", topicName, localCluster, remoteCluster); + if (cursor != null) { + log.info("[{}][{} -> {}] Using the exists cursor for replicator", topicName, localCluster, remoteCluster); + if (expiryMonitor == null) { + this.expiryMonitor = new PersistentMessageExpiryMonitor(topicName, Codec.decode(cursor.getName()), cursor); + } + return CompletableFuture.completedFuture(null); + } + CompletableFuture res = new CompletableFuture<>(); + ledger.asyncOpenCursor(replicatorName, InitialPosition.Earliest, new OpenCursorCallback() { + @Override + public void openCursorComplete(ManagedCursor cursor, Object ctx) { + log.info("[{}][{} -> {}] Open cursor succeed for replicator", topicName, localCluster, remoteCluster); + PersistentReplicator.this.cursor = cursor; + PersistentReplicator.this.expiryMonitor = new PersistentMessageExpiryMonitor(topicName, Codec.decode(cursor.getName()), cursor); + res.complete(null); + } + + @Override + public void openCursorFailed(ManagedLedgerException exception, Object ctx) { + log.warn("[{}][{} -> {}] Open cursor failed for replicator", topicName, localCluster, remoteCluster, exception); + res.completeExceptionally(new PersistenceException(exception)); + } + + }, null); + return res; + } + /** * Calculate available permits for read entries. @@ -601,7 +660,9 @@ public void updateRates() { msgExpired.calculateRate(); stats.msgRateOut = msgOut.getRate(); stats.msgThroughputOut = msgOut.getValueRate(); - stats.msgRateExpired = msgExpired.getRate() + expiryMonitor.getMessageExpiryRate(); + if (expiryMonitor != null) { + stats.msgRateExpired = msgExpired.getRate() + expiryMonitor.getMessageExpiryRate(); + } } public ReplicatorStats getStats() { @@ -639,7 +700,9 @@ public void expireMessages(int messageTTLInSeconds) { // don't do anything for almost caught-up connected subscriptions return; } - expiryMonitor.expireMessages(messageTTLInSeconds); + if (expiryMonitor != null) { + expiryMonitor.expireMessages(messageTTLInSeconds); + } } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 335be8cf3a847..c950ef267b3a7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -224,7 +224,7 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS if (cursor.getName().startsWith(replicatorPrefix)) { String localCluster = brokerService.pulsar().getConfiguration().getClusterName(); String remoteCluster = PersistentReplicator.getRemoteCluster(cursor.getName()); - boolean isReplicatorStarted = addReplicationCluster(remoteCluster, this, cursor, localCluster); + boolean isReplicatorStarted = addReplicationCluster(remoteCluster, this, cursor.getName(), localCluster); if (!isReplicatorStarted) { throw new NamingException( PersistentTopic.this.getName() + " Failed to start replicator " + remoteCluster); @@ -1189,37 +1189,26 @@ CompletableFuture startReplicator(String remoteCluster) { log.info("[{}] Starting replicator to remote: {}", topic, remoteCluster); final CompletableFuture future = new CompletableFuture<>(); - String name = PersistentReplicator.getReplicatorName(replicatorPrefix, remoteCluster); - ledger.asyncOpenCursor(name, new OpenCursorCallback() { - @Override - public void openCursorComplete(ManagedCursor cursor, Object ctx) { - String localCluster = brokerService.pulsar().getConfiguration().getClusterName(); - boolean isReplicatorStarted = addReplicationCluster(remoteCluster, PersistentTopic.this, cursor, localCluster); - if (isReplicatorStarted) { - future.complete(null); - } else { - future.completeExceptionally(new NamingException( - PersistentTopic.this.getName() + " Failed to start replicator " + remoteCluster)); - } - } - - @Override - public void openCursorFailed(ManagedLedgerException exception, Object ctx) { - future.completeExceptionally(new PersistenceException(exception)); - } - - }, null); + String replicatorName = PersistentReplicator.getReplicatorName(replicatorPrefix, remoteCluster); + String localCluster = brokerService.pulsar().getConfiguration().getClusterName(); + boolean isReplicatorStarted = addReplicationCluster(remoteCluster, PersistentTopic.this, replicatorName, localCluster); + if (isReplicatorStarted) { + future.complete(null); + } else { + future.completeExceptionally(new NamingException( + PersistentTopic.this.getName() + " Failed to start replicator " + remoteCluster)); + } return future; } - protected boolean addReplicationCluster(String remoteCluster, PersistentTopic persistentTopic, ManagedCursor cursor, + protected boolean addReplicationCluster(String remoteCluster, PersistentTopic persistentTopic, String replicatorName, String localCluster) { AtomicBoolean isReplicatorStarted = new AtomicBoolean(true); replicators.computeIfAbsent(remoteCluster, r -> { try { - return new PersistentReplicator(PersistentTopic.this, cursor, localCluster, remoteCluster, - brokerService); + return new PersistentReplicator(PersistentTopic.this, replicatorName, localCluster, remoteCluster, + brokerService, ledger); } catch (NamingException e) { isReplicatorStarted.set(false); log.error("[{}] Replicator startup failed due to partitioned-topic {}", topic, remoteCluster); From f862961cb84c0cc19dff29b8db5a54a6c578fbe4 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Mon, 24 Feb 2020 12:24:58 +0800 Subject: [PATCH 18/74] [Reader] Should set either start message id or start message from roll back duration. (#6392) Currently, when constructing a reader, users can set both start message id and start time. This is strange and the behavior should be forbidden. --- .../apache/pulsar/client/impl/ReaderTest.java | 2 +- .../pulsar/client/impl/ReaderBuilderImpl.java | 10 ++++++-- .../pulsar/client/impl/BuildersTest.java | 25 +++++++++++++++++++ 3 files changed, 34 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java index e61089b81fd21..ba05147ac9f2f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java @@ -242,7 +242,7 @@ public void testReaderWithTimeLong() throws Exception { // (3) Create reader and set position 1 hour back so, it should only read messages which are 2 hours old which // published on step 2 - Reader reader = pulsarClient.newReader().topic(topic).startMessageId(MessageId.earliest) + Reader reader = pulsarClient.newReader().topic(topic) .startMessageFromRollbackDuration(2, TimeUnit.HOURS).create(); List receivedMessageIds = Lists.newArrayList(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java index 7901320cd427e..1c62a472254e4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ReaderBuilderImpl.java @@ -82,9 +82,15 @@ public CompletableFuture> createAsync() { .failedFuture(new IllegalArgumentException("Topic name must be set on the reader builder")); } - if (conf.getStartMessageId() == null) { + if (conf.getStartMessageId() != null && conf.getStartMessageFromRollbackDurationInSec() > 0 || + conf.getStartMessageId() == null && conf.getStartMessageFromRollbackDurationInSec() <= 0) { return FutureUtil - .failedFuture(new IllegalArgumentException("Start message id must be set on the reader builder")); + .failedFuture(new IllegalArgumentException( + "Start message id or start message from roll back must be specified but they cannot be specified at the same time")); + } + + if (conf.getStartMessageFromRollbackDurationInSec() > 0) { + conf.setStartMessageId(MessageId.earliest); } return client.createReaderAsync(conf, schema); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BuildersTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BuildersTest.java index 2560d674a600f..4f7554e7bcbef 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BuildersTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BuildersTest.java @@ -26,8 +26,12 @@ import java.lang.reflect.Field; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.TimeUnit; + import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.impl.conf.ReaderConfigurationData; import org.testng.annotations.Test; @@ -96,5 +100,26 @@ public void readerBuilderLoadConfTest() throws Exception { assertTrue(obj instanceof ReaderConfigurationData); assertEquals(((ReaderConfigurationData) obj).getTopicName(), topicName); assertEquals(((ReaderConfigurationData) obj).getStartMessageId(), messageId); + client.close(); + } + + @Test(expectedExceptions = {PulsarClientException.class}, expectedExceptionsMessageRegExp = ".* must be specified but they cannot be specified at the same time.*") + public void shouldNotSetTwoOptAtTheSameTime() throws Exception { + PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://localhost:6650").build(); + try (Reader reader = client.newReader().topic("abc").startMessageId(MessageId.earliest).startMessageFromRollbackDuration(10, TimeUnit.HOURS).create()) { + // no-op + } finally { + client.close(); + } + } + + @Test(expectedExceptions = {PulsarClientException.class}, expectedExceptionsMessageRegExp = ".* must be specified but they cannot be specified at the same time.*") + public void shouldSetOneStartOpt() throws Exception { + PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://localhost:6650").build(); + try (Reader reader = client.newReader().topic("abc").create()) { + // no-op + } finally { + client.close(); + } } } From 81f8afd18d4f7a097e650997efc3d86e1920fb88 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Mon, 24 Feb 2020 12:25:52 +0800 Subject: [PATCH 19/74] Seek to the first one >= timestamp (#6393) The current logic for `resetCursor` by timestamp is odd. The first message it returns is the last message earlier or equal to the designated timestamp. This "earlier" message should be avoided to emit. --- .../persistent/PersistentMessageFinder.java | 2 +- .../persistent/PersistentSubscription.java | 2 +- .../pulsar/broker/admin/AdminApiTest.java | 22 +++++++++---------- .../broker/admin/v1/V1_AdminApiTest.java | 18 +++++++-------- .../broker/service/SubscriptionSeekTest.java | 4 ++-- .../pulsar/client/api/TopicReaderTest.java | 2 +- .../apache/pulsar/client/impl/ReaderTest.java | 4 ++-- 7 files changed, 27 insertions(+), 27 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java index 9e7514987bf56..c90b01f497058 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java @@ -66,7 +66,7 @@ public void findMessages(final long timestamp, AsyncCallbacks.FindEntryCallback MessageImpl msg = null; try { msg = MessageImpl.deserialize(entry.getDataBuffer()); - return msg.getPublishTime() <= timestamp; + return msg.getPublishTime() < timestamp; } catch (Exception e) { log.error("[{}][{}] Error deserializing message for message position find", topicName, subName, e); } finally { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 7055ac0071e62..9bcd2874dc1bc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -654,7 +654,7 @@ public void findEntryComplete(Position position, Object ctx) { "[{}][{}] Unable to find position for timestamp {}. Resetting cursor to first position {} in ledger", topicName, subName, timestamp, finalPosition); } else { - finalPosition = position; + finalPosition = position.getNext(); } resetCursor(finalPosition, future); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java index 5b9949b4ae745..f0ee4f60e985c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java @@ -1595,14 +1595,14 @@ public void persistentTopicsCursorReset(String topicName) throws Exception { int receivedAfterReset = 0; - for (int i = 4; i < 10; i++) { + for (int i = 5; i < 10; i++) { Message message = consumer.receive(); consumer.acknowledge(message); ++receivedAfterReset; String expected = "message-" + i; assertEquals(message.getData(), expected.getBytes()); } - assertEquals(receivedAfterReset, 6); + assertEquals(receivedAfterReset, 5); consumer.close(); @@ -1654,29 +1654,29 @@ public void persistentTopicsCursorResetAfterReset(String topicName) throws Excep int receivedAfterReset = 0; - // Should received messages from 4-9 - for (int i = 4; i < 10; i++) { + // Should received messages from 5-9 + for (int i = 5; i < 10; i++) { Message message = consumer.receive(); consumer.acknowledge(message); ++receivedAfterReset; String expected = "message-" + i; assertEquals(new String(message.getData()), expected); } - assertEquals(receivedAfterReset, 6); + assertEquals(receivedAfterReset, 5); // Reset at 2nd timestamp receivedAfterReset = 0; admin.topics().resetCursor(topicName, "my-sub", secondTimestamp); - // Should received messages from 7-9 - for (int i = 7; i < 10; i++) { + // Should received messages from 8-9 + for (int i = 8; i < 10; i++) { Message message = consumer.receive(); consumer.acknowledge(message); ++receivedAfterReset; String expected = "message-" + i; assertEquals(new String(message.getData()), expected); } - assertEquals(receivedAfterReset, 3); + assertEquals(receivedAfterReset, 2); consumer.close(); admin.topics().deleteSubscription(topicName, "my-sub"); @@ -1724,14 +1724,14 @@ public void persistentTopicsCursorResetAndFailover() throws Exception { .acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe(); int receivedAfterReset = 0; - for (int i = 4; i < 10; i++) { + for (int i = 5; i < 10; i++) { Message message = consumerA.receive(5, TimeUnit.SECONDS); consumerA.acknowledge(message); ++receivedAfterReset; String expected = "message-" + i; assertEquals(message.getData(), expected.getBytes()); } - assertEquals(receivedAfterReset, 6); + assertEquals(receivedAfterReset, 5); // Closing consumerA activates consumerB consumerA.close(); @@ -1787,7 +1787,7 @@ public void partitionedTopicsCursorReset(String topicName) throws Exception { Set expectedMessages = Sets.newHashSet(); Set receivedMessages = Sets.newHashSet(); - for (int i = 4; i < 10; i++) { + for (int i = 5; i < 10; i++) { Message message = consumer.receive(); consumer.acknowledge(message); expectedMessages.add("message-" + i); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java index 9c7e1d47b94eb..373a6b7c48766 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java @@ -1490,14 +1490,14 @@ public void persistentTopicsCursorReset(String topicName) throws Exception { int receivedAfterReset = 0; - for (int i = 4; i < 10; i++) { + for (int i = 5; i < 10; i++) { Message message = consumer.receive(); consumer.acknowledge(message); ++receivedAfterReset; String expected = "message-" + i; assertEquals(message.getData(), expected.getBytes()); } - assertEquals(receivedAfterReset, 6); + assertEquals(receivedAfterReset, 5); consumer.close(); @@ -1549,29 +1549,29 @@ public void persistentTopicsCursorResetAfterReset(String topicName) throws Excep int receivedAfterReset = 0; - // Should received messages from 4-9 - for (int i = 4; i < 10; i++) { + // Should received messages from 5-9 + for (int i = 5; i < 10; i++) { Message message = consumer.receive(); consumer.acknowledge(message); ++receivedAfterReset; String expected = "message-" + i; assertEquals(new String(message.getData()), expected); } - assertEquals(receivedAfterReset, 6); + assertEquals(receivedAfterReset, 5); // Reset at 2nd timestamp receivedAfterReset = 0; admin.topics().resetCursor(topicName, "my-sub", secondTimestamp); - // Should received messages from 7-9 - for (int i = 7; i < 10; i++) { + // Should received messages from 8-9 + for (int i = 8; i < 10; i++) { Message message = consumer.receive(); consumer.acknowledge(message); ++receivedAfterReset; String expected = "message-" + i; assertEquals(new String(message.getData()), expected); } - assertEquals(receivedAfterReset, 3); + assertEquals(receivedAfterReset, 2); consumer.close(); admin.topics().deleteSubscription(topicName, "my-sub"); @@ -1614,7 +1614,7 @@ public void partitionedTopicsCursorReset(String topicName) throws Exception { Set expectedMessages = Sets.newHashSet(); Set receivedMessages = Sets.newHashSet(); - for (int i = 4; i < 10; i++) { + for (int i = 5; i < 10; i++) { Message message = consumer.receive(); consumer.acknowledge(message); expectedMessages.add("message-" + i); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java index c9cc07ccaed00..9ea1e0cca07a2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java @@ -135,7 +135,7 @@ public void testSeekTime() throws Exception { long currentTimestamp = System.currentTimeMillis(); consumer.seek(currentTimestamp); - assertEquals(sub.getNumberOfEntriesInBacklog(false), 1); + assertEquals(sub.getNumberOfEntriesInBacklog(false), 0); // Wait for consumer to reconnect Thread.sleep(1000); @@ -187,7 +187,7 @@ public void testSeekTimeOnPartitionedTopic() throws Exception { for (PersistentSubscription sub : subs) { backlogs += sub.getNumberOfEntriesInBacklog(false); } - assertEquals(backlogs, 2); + assertEquals(backlogs, 0); // Wait for consumer to reconnect Thread.sleep(1000); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java index 69139b5d04d72..be2097d472149 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java @@ -676,7 +676,7 @@ public void testReaderIsAbleToSeekWithTimeOnMiddleOfTopic() throws Exception { reader.seek(l + plusTime); Set messageSet = Sets.newHashSet(); - for (int i = halfMessages; i < numOfMessage; i++) { + for (int i = halfMessages + 1; i < numOfMessage; i++) { Message message = reader.readNext(); String receivedMessage = new String(message.getData()); String expectedMessage = String.format("msg num %d", i); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java index ba05147ac9f2f..5f97a5888389f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java @@ -256,8 +256,8 @@ public void testReaderWithTimeLong() throws Exception { receivedMessageIds.add(msg.getMessageId()); } - assertEquals(receivedMessageIds.size(), totalMsg + 1); - assertEquals(receivedMessageIds.get(0), lastMsgId); + assertEquals(receivedMessageIds.size(), totalMsg); + assertEquals(receivedMessageIds.get(0), firstMsgId); restartBroker(); From 7fb9affa3615ade8edddabce849437df02f89500 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Mon, 24 Feb 2020 12:26:37 +0800 Subject: [PATCH 20/74] [Minor] Fix java code errors reported by lgtm. (#6398) Four kinds of errors are fixed in this PR: - Array index out of bounds - Inconsistent equals and hashCode - Missing format argument - Reference equality test of boxed types According to https://lgtm.com/projects/g/apache/pulsar/alerts/?mode=tree&severity=error&id=&lang=java --- .../mledger/impl/ManagedLedgerOfflineBacklog.java | 2 +- .../apache/bookkeeper/mledger/util/CallbackMutex.java | 2 +- .../apache/pulsar/broker/admin/impl/BrokersBase.java | 2 +- .../pulsar/client/admin/internal/BaseResource.java | 2 +- .../org/apache/pulsar/client/impl/ConsumerImpl.java | 10 +++++++++- .../pulsar/client/impl/MultiTopicsConsumerImpl.java | 2 +- .../apache/pulsar/common/policies/data/Policies.java | 2 +- .../org/apache/pulsar/common/sasl/KerberosName.java | 2 +- 8 files changed, 16 insertions(+), 8 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java index d8b49ad959b62..88ed6474da407 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java @@ -195,7 +195,7 @@ public void operationComplete(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) @Override public void operationFailed(ManagedLedgerException.MetaStoreException e) { - log.warn("[{}] Unable to obtain managed ledger metadata - {}", e); + log.warn("[{}] Unable to obtain managed ledger metadata - {}", managedLedgerName, e); mlMetaCounter.countDown(); } }); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackMutex.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackMutex.java index 9e934bfa461a6..e5f07c2c7b8d6 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackMutex.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackMutex.java @@ -62,7 +62,7 @@ public void unlock() { if (log.isDebugEnabled()) { owner = null; position = null; - log.debug(">>> Lock {} released token={} at {}", this.hashCode(), + log.debug(">>> Lock {} released at {}", this.hashCode(), Thread.currentThread().getStackTrace()[2]); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java index 1c3b7ca7369e3..3534c0429a55b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokersBase.java @@ -360,7 +360,7 @@ private synchronized void deleteDynamicConfigurationOnZk(String configName) { LOG.info("[{}] Deleted Service configuration {}", clientAppId(), configName); } else { if (LOG.isDebugEnabled()) { - LOG.debug("[{}] Can't update non-dynamic configuration {}/{}", clientAppId(), configName); + LOG.debug("[{}] Can't update non-dynamic configuration {}", clientAppId(), configName); } throw new RestException(Status.PRECONDITION_FAILED, " Can't update non-dynamic configuration"); } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BaseResource.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BaseResource.java index c654717daffd9..2cbe87d3a6490 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BaseResource.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BaseResource.java @@ -87,7 +87,7 @@ public CompletableFuture requestAsync(final WebTarget target) { // auth complete, return a new Builder authFuture.whenComplete((respHeaders, ex) -> { if (ex != null) { - log.warn("[{}] Failed to perform http request at authn stage: {}", + log.warn("[{}] Failed to perform http request at auth stage: {}", target.getUri(), ex.getMessage()); builderFuture.completeExceptionally(new PulsarClientException(ex)); return; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 5b62248370c54..08049a768c3e1 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -1528,7 +1528,7 @@ public CompletableFuture seekAsync(MessageId messageId) { log.error("[{}][{}] Failed to reset subscription: {}", topic, subscription, e.getCause().getMessage()); seekFuture.completeExceptionally( PulsarClientException.wrap(e.getCause(), - String.format("[%s][%s] Failed to seek the subscription %s of the topic %s to the message %s", + String.format("Failed to seek the subscription %s of the topic %s to the message %s", subscription, topicName.toString(), messageId.toString()))); return null; }); @@ -1756,6 +1756,14 @@ public int hashCode() { return Objects.hash(topic, subscription, consumerName); } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof ConsumerImpl)) return false; + ConsumerImpl consumer = (ConsumerImpl) o; + return consumerId == consumer.consumerId; + } + // wrapper for connection methods ClientCnx cnx() { return this.connectionHandler.cnx(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 3e9e69ae51c3c..5c2609573aa78 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -1163,7 +1163,7 @@ public void run(Timeout timeout) throws Exception { } if (log.isDebugEnabled()) { - log.debug("[{}] run partitionsAutoUpdateTimerTask for multiTopicsConsumer: {}", topic); + log.debug("[{}] run partitionsAutoUpdateTimerTask", topic); } // if last auto update not completed yet, do nothing. diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java index 4b2029b36f129..2a9e04314bdd2 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/Policies.java @@ -157,7 +157,7 @@ public boolean equals(Object obj) { && max_unacked_messages_per_subscription == other.max_unacked_messages_per_subscription && compaction_threshold == other.compaction_threshold && offload_threshold == other.offload_threshold - && offload_deletion_lag_ms == other.offload_deletion_lag_ms + && Objects.equals(offload_deletion_lag_ms, other.offload_deletion_lag_ms) && schema_auto_update_compatibility_strategy == other.schema_auto_update_compatibility_strategy && schema_validation_enforced == other.schema_validation_enforced && schema_compatibility_strategy == other.schema_compatibility_strategy diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/sasl/KerberosName.java b/pulsar-common/src/main/java/org/apache/pulsar/common/sasl/KerberosName.java index 6771a23ea8693..c70c7769d4482 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/sasl/KerberosName.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/sasl/KerberosName.java @@ -271,7 +271,7 @@ static String replaceParameters(String format, if (paramNum != null) { try { int num = Integer.parseInt(paramNum); - if (num < 0 || num > params.length) { + if (num < 0 || num >= params.length) { throw new BadFormatString("index " + num + " from " + format + " is outside of the valid range 0 to " + (params.length - 1)); } From 63ccd43e1a3294d696a4af37c76261eed1bb3124 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Mon, 24 Feb 2020 12:27:08 +0800 Subject: [PATCH 21/74] [Java Reader Client] Start reader inside batch result in read first message in batch. (#6345) Fixes #6344 Fixes #6350 The bug was brought in https://github.com/apache/pulsar/pull/5622 by changing the skip logic wrongly. --- .../api/SimpleProducerConsumerTest.java | 38 +++++++--- .../pulsar/client/api/TopicReaderTest.java | 74 +++++++++++++++++-- .../pulsar/client/api/BatchReceivePolicy.java | 2 +- .../pulsar/client/impl/ConsumerBase.java | 2 +- .../pulsar/client/impl/ConsumerImpl.java | 31 ++++++-- 5 files changed, 123 insertions(+), 24 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index 59910a806c3ed..e907197de7b1b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -47,6 +47,8 @@ import java.time.Clock; import java.time.Instant; import java.time.ZoneId; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -69,8 +71,10 @@ import java.util.stream.Collectors; import lombok.Cleanup; +import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.mledger.impl.EntryCacheImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.ConsumerImpl; @@ -1419,7 +1423,6 @@ public void testConsumerBlockingWithUnAckedMessagesMultipleIteration() throws Ex * Verify: Consumer1 which doesn't send ack will not impact Consumer2 which sends ack for consumed message. * * - * @param batchMessageDelayMs * @throws Exception */ @Test @@ -1736,7 +1739,6 @@ public void testUnackedBlockAtBatch(int batchMessageDelayMs) throws Exception { * Verify: Consumer2 sends ack of Consumer1 and consumer1 should be unblock if it is blocked due to unack-messages * * - * @param batchMessageDelayMs * @throws Exception */ @Test @@ -3219,7 +3221,7 @@ public void testPartitionedTopicWithOnePartition() throws Exception { } @Test(dataProvider = "variationsForExpectedPos") - public void testReaderStartMessageIdAtExpectedPos(boolean batching, boolean startInclusive, int numOfMessages) + public void testConsumerStartMessageIdAtExpectedPos(boolean batching, boolean startInclusive, int numOfMessages) throws Exception { final String topicName = "persistent://my-property/my-ns/ConsumerStartMessageIdAtExpectedPos"; final int resetIndex = new Random().nextInt(numOfMessages); // Choose some random index to reset @@ -3230,14 +3232,31 @@ public void testReaderStartMessageIdAtExpectedPos(boolean batching, boolean star .enableBatching(batching) .create(); - MessageId resetPos = null; + CountDownLatch latch = new CountDownLatch(numOfMessages); + + final AtomicReference resetPos = new AtomicReference<>(); + for (int i = 0; i < numOfMessages; i++) { - MessageId msgId = producer.send(String.format("msg num %d", i).getBytes()); - if (resetIndex == i) { - resetPos = msgId; - } + + final int j = i; + + producer.sendAsync(String.format("msg num %d", i).getBytes()) + .thenCompose(messageId -> FutureUtils.value(Pair.of(j, messageId))) + .whenComplete((p, e) -> { + if (e != null) { + fail("send msg failed due to " + e.getMessage()); + } else { + log.info("send msg with id {}", p.getRight()); + if (p.getLeft() == resetIndex) { + resetPos.set(p.getRight()); + } + } + latch.countDown(); + }); } + latch.await(); + ConsumerBuilder consumerBuilder = pulsarClient.newConsumer() .topic(topicName); @@ -3246,7 +3265,8 @@ public void testReaderStartMessageIdAtExpectedPos(boolean batching, boolean star } Consumer consumer = consumerBuilder.subscriptionName("my-subscriber-name").subscribe(); - consumer.seek(resetPos); + consumer.seek(resetPos.get()); + log.info("reset cursor to {}", resetPos.get()); Set messageSet = Sets.newHashSet(); for (int i = firstMessage; i < numOfMessages; i++) { Message message = consumer.receive(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java index be2097d472149..7eda4465f7ff0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java @@ -22,6 +22,7 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -30,12 +31,16 @@ import java.nio.file.Paths; import java.util.*; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.bookkeeper.common.concurrent.FutureUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.ReaderImpl; import org.apache.pulsar.common.policies.data.TopicStats; -import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.RelativeTimeUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -699,17 +704,33 @@ public void testReaderStartMessageIdAtExpectedPos(boolean batching, boolean star .enableBatching(batching) .create(); - MessageId resetPos = null; + CountDownLatch latch = new CountDownLatch(numOfMessages); + + final AtomicReference resetPos = new AtomicReference<>(); + for (int i = 0; i < numOfMessages; i++) { - MessageId msgId = producer.send(String.format("msg num %d", i).getBytes()); - if (resetIndex == i) { - resetPos = msgId; - } + + final int j = i; + + producer.sendAsync(String.format("msg num %d", i).getBytes()) + .thenCompose(messageId -> FutureUtils.value(Pair.of(j, messageId))) + .whenComplete((p, e) -> { + if (e != null) { + fail("send msg failed due to " + e.getMessage()); + } else { + if (p.getLeft() == resetIndex) { + resetPos.set(p.getRight()); + } + } + latch.countDown(); + }); } + latch.await(); + ReaderBuilder readerBuilder = pulsarClient.newReader() .topic(topicName) - .startMessageId(resetPos); + .startMessageId(resetPos.get()); if (startInclusive) { readerBuilder.startMessageIdInclusive(); @@ -761,4 +782,43 @@ public void testReaderBuilderConcurrentCreate() throws Exception { producers.get(i).close(); } } + + @Test + public void testReaderStartInMiddleOfBatch() throws Exception { + final String topicName = "persistent://my-property/my-ns/ReaderStartInMiddleOfBatch"; + final int numOfMessage = 100; + + Producer producer = pulsarClient.newProducer() + .topic(topicName) + .enableBatching(true) + .batchingMaxMessages(10) + .create(); + + CountDownLatch latch = new CountDownLatch(100); + + List allIds = Collections.synchronizedList(new ArrayList<>()); + + for (int i = 0; i < numOfMessage; i++) { + producer.sendAsync(String.format("msg num %d", i).getBytes()).whenComplete((mid, e) -> { + if (e != null) { + fail(); + } else { + allIds.add(mid); + } + latch.countDown(); + }); + } + + latch.await(); + + for (MessageId id : allIds) { + Reader reader = pulsarClient.newReader().topic(topicName) + .startMessageId(id).startMessageIdInclusive().create(); + MessageId idGot = reader.readNext().getMessageId(); + assertEquals(idGot, id); + reader.close(); + } + + producer.close(); + } } diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/BatchReceivePolicy.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/BatchReceivePolicy.java index 2b9be7128c4d8..7a5a1bd19d2cc 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/BatchReceivePolicy.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/BatchReceivePolicy.java @@ -46,7 +46,7 @@ public class BatchReceivePolicy { /** * Default batch receive policy. * - *

Max number of messages: 100 + *

Max number of messages: no limit * Max number of bytes: 10MB * Timeout: 100ms

*/ diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index 86e7fb27da751..0f5219b0a9b95 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -464,7 +464,7 @@ protected boolean enqueueMessageAndCheckBatchReceive(Message message) { } protected boolean hasEnoughMessagesForBatchReceive() { - if (batchReceivePolicy.getMaxNumMessages() <= 0 && batchReceivePolicy.getMaxNumMessages() <= 0) { + if (batchReceivePolicy.getMaxNumMessages() <= 0 && batchReceivePolicy.getMaxNumBytes() <= 0) { return false; } return (batchReceivePolicy.getMaxNumMessages() > 0 && incomingMessages.size() >= batchReceivePolicy.getMaxNumMessages()) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 08049a768c3e1..9431496688944 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -45,6 +45,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReadWriteLock; @@ -123,6 +124,9 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle private final SubscriptionMode subscriptionMode; private volatile BatchMessageIdImpl startMessageId; + private volatile BatchMessageIdImpl seekMessageId; + private final AtomicBoolean duringSeek; + private final BatchMessageIdImpl initialStartMessageId; private final long startMessageRollbackDurationInSec; @@ -205,6 +209,8 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat stats = ConsumerStatsDisabled.INSTANCE; } + duringSeek = new AtomicBoolean(false); + if (conf.getAckTimeoutMillis() != 0) { if (conf.getTickDurationMillis() > 0) { this.unAckedMessageTracker = new UnAckedMessageTracker(client, this, conf.getAckTimeoutMillis(), @@ -667,6 +673,13 @@ private BatchMessageIdImpl clearReceiverQueue() { List> currentMessageQueue = new ArrayList<>(incomingMessages.size()); incomingMessages.drainTo(currentMessageQueue); INCOMING_MESSAGES_SIZE_UPDATER.set(this, 0); + + if (duringSeek.compareAndSet(true, false)) { + return seekMessageId; + } else if (subscriptionMode == SubscriptionMode.Durable) { + return null; + } + if (!currentMessageQueue.isEmpty()) { MessageIdImpl nextMessageInQueue = (MessageIdImpl) currentMessageQueue.get(0).getMessageId(); BatchMessageIdImpl previousMessage; @@ -867,7 +880,7 @@ void messageReceived(MessageIdData messageId, int redeliveryCount, ByteBuf heade // and return undecrypted payload if (isMessageUndecryptable || (numMessages == 1 && !msgMetadata.hasNumMessagesInBatch())) { - if (isResetIncludedAndSameEntryLedger(messageId) && isPriorEntryIndex(messageId.getEntryId())) { + if (isSameEntry(messageId) && isPriorEntryIndex(messageId.getEntryId())) { // We need to discard entries that were prior to startMessageId if (log.isDebugEnabled()) { log.debug("[{}] [{}] Ignoring message from before the startMessageId: {}", subscription, @@ -1018,7 +1031,7 @@ void receiveIndividualMessagesFromBatch(MessageMetadata msgMetadata, int redeliv ByteBuf singleMessagePayload = Commands.deSerializeSingleMessageInBatch(uncompressedPayload, singleMessageMetadataBuilder, i, batchSize); - if (isResetIncludedAndSameEntryLedger(messageId) && isPriorBatchIndex(i)) { + if (isSameEntry(messageId) && isPriorBatchIndex(i)) { // If we are receiving a batch message, we need to discard messages that were prior // to the startMessageId if (log.isDebugEnabled()) { @@ -1091,8 +1104,8 @@ private boolean isPriorBatchIndex(long idx) { return resetIncludeHead ? idx < startMessageId.getBatchIndex() : idx <= startMessageId.getBatchIndex(); } - private boolean isResetIncludedAndSameEntryLedger(MessageIdData messageId) { - return !resetIncludeHead && startMessageId != null + private boolean isSameEntry(MessageIdData messageId) { + return startMessageId != null && messageId.getLedgerId() == startMessageId.getLedgerId() && messageId.getEntryId() == startMessageId.getEntryId(); } @@ -1477,7 +1490,10 @@ public CompletableFuture seekAsync(long timestamp) { cnx.sendRequestWithId(seek, requestId).thenRun(() -> { log.info("[{}][{}] Successfully reset subscription to publish time {}", topic, subscription, timestamp); acknowledgmentsGroupingTracker.flushAndClean(); - lastDequeuedMessage = MessageId.earliest; + + seekMessageId = new BatchMessageIdImpl((MessageIdImpl) MessageId.earliest); + duringSeek.set(true); + incomingMessages.clear(); INCOMING_MESSAGES_SIZE_UPDATER.set(this, 0); seekFuture.complete(null); @@ -1520,7 +1536,10 @@ public CompletableFuture seekAsync(MessageId messageId) { cnx.sendRequestWithId(seek, requestId).thenRun(() -> { log.info("[{}][{}] Successfully reset subscription to message id {}", topic, subscription, messageId); acknowledgmentsGroupingTracker.flushAndClean(); - lastDequeuedMessage = messageId; + + seekMessageId = new BatchMessageIdImpl((MessageIdImpl) messageId); + duringSeek.set(true); + incomingMessages.clear(); INCOMING_MESSAGES_SIZE_UPDATER.set(this, 0); seekFuture.complete(null); From e1f7505cbc2cadcba7cb79d8e3ed5b7855688858 Mon Sep 17 00:00:00 2001 From: Fangbin Sun Date: Mon, 24 Feb 2020 21:46:31 +0800 Subject: [PATCH 22/74] Fix broker to specify a list of bookie groups. (#6349) ### Motivation Fixes #6343 ### Modifications Add a method to cast object value to `String`. --- .../ZkIsolatedBookieEnsemblePlacementPolicy.java | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZkIsolatedBookieEnsemblePlacementPolicy.java b/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZkIsolatedBookieEnsemblePlacementPolicy.java index 656b3a2029101..ef4f72bb14901 100644 --- a/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZkIsolatedBookieEnsemblePlacementPolicy.java +++ b/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZkIsolatedBookieEnsemblePlacementPolicy.java @@ -71,7 +71,7 @@ public RackawareEnsemblePlacementPolicyImpl initialize(ClientConfiguration conf, Optional optionalDnsResolver, HashedWheelTimer timer, FeatureProvider featureProvider, StatsLogger statsLogger) { if (conf.getProperty(ISOLATION_BOOKIE_GROUPS) != null) { - String isolationGroupsString = (String) conf.getProperty(ISOLATION_BOOKIE_GROUPS); + String isolationGroupsString = castToString(conf.getProperty(ISOLATION_BOOKIE_GROUPS)); if (!isolationGroupsString.isEmpty()) { for (String isolationGroup : isolationGroupsString.split(",")) { primaryIsolationGroups.add(isolationGroup); @@ -80,7 +80,7 @@ public RackawareEnsemblePlacementPolicyImpl initialize(ClientConfiguration conf, } } if (conf.getProperty(SECONDARY_ISOLATION_BOOKIE_GROUPS) != null) { - String secondaryIsolationGroupsString = (String) conf.getProperty(SECONDARY_ISOLATION_BOOKIE_GROUPS); + String secondaryIsolationGroupsString = castToString(conf.getProperty(SECONDARY_ISOLATION_BOOKIE_GROUPS)); if (!secondaryIsolationGroupsString.isEmpty()) { for (String isolationGroup : secondaryIsolationGroupsString.split(",")) { secondaryIsolationGroups.add(isolationGroup); @@ -90,6 +90,18 @@ public RackawareEnsemblePlacementPolicyImpl initialize(ClientConfiguration conf, return super.initialize(conf, optionalDnsResolver, timer, featureProvider, statsLogger); } + private String castToString(Object obj) { + if (obj instanceof List) { + List result = new ArrayList<>(); + for (Object o : (List) obj) { + result.add(String.class.cast(o)); + } + return String.join(",", result); + } else { + return obj.toString(); + } + } + private ZooKeeperCache getAndSetZkCache(Configuration conf) { ZooKeeperCache zkCache = null; if (conf.getProperty(ZooKeeperCache.ZK_CACHE_INSTANCE) != null) { From 2f4207753f1f1164b28d24d49c813557c15d2f63 Mon Sep 17 00:00:00 2001 From: guangning Date: Tue, 25 Feb 2020 03:37:04 +0800 Subject: [PATCH 23/74] Fixed enum package not found (#6401) Fixes https://github.com/apache/pulsar/issues/6400 ### Motivation This problem is blocking the current test. 1.1.8 version of `enum34` seems to have some problems, and the problem reproduces: Use pulsar latest code: ``` cd pulsar mvn clean install -DskipTests dokcer pull apachepulsar/pulsar-build:ubuntu-16.04 docker run -it -v $PWD:/pulsar --name pulsar apachepulsar/pulsar-build:ubuntu-16.04 /bin/bash docker exec -it pulsar /bin/bash cmake . make -j4 && make install cd python python setup.py bdist_wheel pip install dist/pulsar_client-*-linux_x86_64.whl ``` `pip show enum34` ``` Name: enum34 Version: 1.1.8 Summary: Python 3.4 Enum backported to 3.3, 3.2, 3.1, 2.7, 2.6, 2.5, and 2.4 Home-page: https://bitbucket.org/stoneleaf/enum34 Author: Ethan Furman Author-email: ethan@stoneleaf.us License: BSD License Location: /usr/local/lib/python2.7/dist-packages Requires: Required-by: pulsar-client, grpcio ``` ``` root@55e06c5c770f:/pulsar/pulsar-client-cpp/python# python Python 2.7.12 (default, Oct 8 2019, 14:14:10) [GCC 5.4.0 20160609] on linux2 Type "help", "copyright", "credits" or "license" for more information. >>> from enum import Enum, EnumMeta Traceback (most recent call last): File "", line 1, in ImportError: No module named enum >>> exit() ``` There is no problem with using 1.1.9 in the test. ### Modifications * Upgrade enum34 from 1.1.8 to 1.1.9 ### Verifying this change local test pass --- pulsar-client-cpp/python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-client-cpp/python/setup.py b/pulsar-client-cpp/python/setup.py index 52617a2cde881..f25113b2c50fe 100644 --- a/pulsar-client-cpp/python/setup.py +++ b/pulsar-client-cpp/python/setup.py @@ -92,7 +92,7 @@ def build_extension(self, ext): if PY2: # Python 2 compat dependencies - dependencies += ['enum34'] + dependencies += ['enum34>=1.1.9'] setup( name=NAME, From 739c0dd6c50261b0725208af517420a83fed030d Mon Sep 17 00:00:00 2001 From: newur Date: Mon, 24 Feb 2020 20:38:37 +0100 Subject: [PATCH 24/74] removed comma from yaml config (#6402) --- site2/docs/io-rabbitmq-source.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site2/docs/io-rabbitmq-source.md b/site2/docs/io-rabbitmq-source.md index e9069cb351a22..890a1eaf7d6ff 100644 --- a/site2/docs/io-rabbitmq-source.md +++ b/site2/docs/io-rabbitmq-source.md @@ -61,7 +61,7 @@ Before using the RabbitMQ source connector, you need to create a configuration f configs: host: "localhost" port: 5672 - virtualHost: "/", + virtualHost: "/" username: "guest" password: "guest" queueName: "test-queue" From 1fcccd62ac9566f3506fad2e2abd3debd54918ee Mon Sep 17 00:00:00 2001 From: Jia Zhai Date: Tue, 25 Feb 2020 04:02:43 +0800 Subject: [PATCH 25/74] Fix broker client tls settings error (#6128) when broker create the inside client, it sets tlsTrustCertsFilePath as "getTlsCertificateFilePath()", but it should be "getBrokerClientTrustCertsFilePath()" --- .../main/java/org/apache/pulsar/broker/PulsarService.java | 8 ++++++++ pulsar-client-cpp/python/pulsar_test.py | 8 ++++---- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index fe848b5e8b466..31b7fa391f245 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -910,6 +910,14 @@ public synchronized PulsarClient getClient() throws PulsarServerException { .enableTls(this.getConfiguration().isTlsEnabled()) .allowTlsInsecureConnection(this.getConfiguration().isTlsAllowInsecureConnection()) .tlsTrustCertsFilePath(this.getConfiguration().getTlsCertificateFilePath()); + + if (this.getConfiguration().isBrokerClientTlsEnabled()) { + builder.tlsTrustCertsFilePath( + isNotBlank(this.getConfiguration().getBrokerClientTrustCertsFilePath()) + ? this.getConfiguration().getBrokerClientTrustCertsFilePath() + : this.getConfiguration().getTlsCertificateFilePath()); + } + if (isNotBlank(this.getConfiguration().getBrokerClientAuthenticationPlugin())) { builder.authentication(this.getConfiguration().getBrokerClientAuthenticationPlugin(), this.getConfiguration().getBrokerClientAuthenticationParameters()); diff --git a/pulsar-client-cpp/python/pulsar_test.py b/pulsar-client-cpp/python/pulsar_test.py index 14cf2938bc554..4109ec79eeaa2 100755 --- a/pulsar-client-cpp/python/pulsar_test.py +++ b/pulsar-client-cpp/python/pulsar_test.py @@ -152,7 +152,7 @@ def test_redelivery_count(self): producer.send(b'hello') redelivery_count = 0 - for i in range(4): + for i in range(4): msg = consumer.receive(TM) print("Received message %s" % msg.data()) consumer.negative_acknowledge(msg) @@ -668,13 +668,13 @@ def test_publish_compact_and_consume(self): while True: s=doHttpGet(url).decode('utf-8') if 'RUNNING' in s: - print("Compact still running") print(s) + print("Compact still running") time.sleep(0.2) else: - self.assertTrue('SUCCESS' in s) - print("Compact Complete now") print(s) + print("Compact Complete now") + self.assertTrue('SUCCESS' in s) break # after compaction completes the compacted ledger is recorded From dab14acdddf4218eb15d6537ec4a5aa2098a5248 Mon Sep 17 00:00:00 2001 From: Sergii Zhevzhyk Date: Tue, 25 Feb 2020 05:00:14 +0100 Subject: [PATCH 26/74] [Issue 3762][Schema] Fix the problem with parsing of an Avro schema related to shading in pulsar-client. (#6406) Motivation Avro schemas are quite important for proper data flow and it is a pity that the #3762 issue stayed untouched for so long. There were some workarounds on how to make Pulsar use an original avro schema, but in the end, it is pretty hard to run an enterprise solution on workarounds. With this PR I would like to find a solution to the problem caused by shading avro in pulsar-client. As it was discussed in the issue, there are two possible solutions for this problem: Unshade the avro library in the pulsar-client library. (IMHO it seems like a proper solution for this problem, but it also brings a risk of unknown side-effects) Use reflection to get original schemas from generated classes. (I went for this solution) Could you please comment if this is a proper solution for the problem? I will add tests when my approach will be confirmed. Modifications First, we try to extract an original avro schema from the "$SCHEMA" field using reflection. If it doesn't work, the process falls back generation of the schema from POJO. --- .../pulsar/client/impl/schema/StructSchema.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/StructSchema.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/StructSchema.java index 1302d7c84985f..f9faa0b1156bc 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/StructSchema.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/StructSchema.java @@ -150,8 +150,7 @@ protected static org.apache.avro.Schema createAvroSchema(SchemaDefinition schema try { // Disable validation of default values for compatibility validateDefaults.set(false); - return schemaDefinition.getAlwaysAllowNull() ? ReflectData.AllowNull.get().getSchema(pojo) - : ReflectData.get().getSchema(pojo); + return extractAvroSchema(schemaDefinition, pojo); } finally { validateDefaults.set(savedValidateDefaults); } @@ -160,6 +159,15 @@ protected static org.apache.avro.Schema createAvroSchema(SchemaDefinition schema } } + protected static Schema extractAvroSchema(SchemaDefinition schemaDefinition, Class pojo) { + try { + return parseAvroSchema(pojo.getDeclaredField("SCHEMA$").get(null).toString()); + } catch (NoSuchFieldException | IllegalAccessException | IllegalArgumentException ignored) { + return schemaDefinition.getAlwaysAllowNull() ? ReflectData.AllowNull.get().getSchema(pojo) + : ReflectData.get().getSchema(pojo); + } + } + protected static org.apache.avro.Schema parseAvroSchema(String schemaJson) { final Parser parser = new Parser(); parser.setValidateDefaults(false); From 49c573c35a52fd8dde159d25b44041c30bcb7610 Mon Sep 17 00:00:00 2001 From: Sergii Zhevzhyk Date: Tue, 25 Feb 2020 05:02:25 +0100 Subject: [PATCH 27/74] Remove duplicated lombok annotations in the tests module (#6412) --- .../tests/integration/io/JdbcSinkTester.java | 2 -- .../tests/integration/schema/Schemas.java | 32 ------------------- .../compat/kafka/KafkaApiTest.java | 4 --- 3 files changed, 38 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/JdbcSinkTester.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/JdbcSinkTester.java index 990470975412e..23c46a443cad1 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/JdbcSinkTester.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/JdbcSinkTester.java @@ -50,8 +50,6 @@ public class JdbcSinkTester extends SinkTester { * */ @Data - @ToString - @EqualsAndHashCode public static class Foo { private String field1; private String field2; diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/schema/Schemas.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/schema/Schemas.java index 9841bcfc9b6bb..1bb9c450f36d5 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/schema/Schemas.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/schema/Schemas.java @@ -56,10 +56,6 @@ public final class Schemas { * A Person Struct. */ @Data - @Getter - @Setter - @ToString - @EqualsAndHashCode public static class Person { private String name; @@ -71,10 +67,6 @@ public static class Person { * A Person Struct. */ @Data - @Getter - @Setter - @ToString - @EqualsAndHashCode public static class PersonConsumeSchema { private String name; @@ -88,10 +80,6 @@ public static class PersonConsumeSchema { * A Student Struct. */ @Data - @Getter - @Setter - @ToString - @EqualsAndHashCode public static class Student { private String name; @@ -102,12 +90,8 @@ public static class Student { } @Data - @Getter - @Setter - @ToString @NoArgsConstructor @AllArgsConstructor - @EqualsAndHashCode @Builder public static class AvroLogicalType{ @org.apache.avro.reflect.AvroSchema("{\n" + @@ -132,10 +116,6 @@ public static class AvroLogicalType{ private Schemas() {} @Data - @Getter - @Setter - @ToString - @EqualsAndHashCode @AllArgsConstructor @NoArgsConstructor public static class PersonOne{ @@ -143,10 +123,6 @@ public static class PersonOne{ } @Data - @Getter - @Setter - @ToString - @EqualsAndHashCode @AllArgsConstructor @NoArgsConstructor public static class PersonTwo{ @@ -157,10 +133,6 @@ public static class PersonTwo{ } @Data - @Getter - @Setter - @ToString - @EqualsAndHashCode public static class PersonThree{ int id; @@ -168,10 +140,6 @@ public static class PersonThree{ } @Data - @Getter - @Setter - @ToString - @EqualsAndHashCode public static class PersonFour{ int id; diff --git a/tests/pulsar-kafka-compat-client-test/src/test/java/org/apache/pulsar/tests/integration/compat/kafka/KafkaApiTest.java b/tests/pulsar-kafka-compat-client-test/src/test/java/org/apache/pulsar/tests/integration/compat/kafka/KafkaApiTest.java index 632cee90e1aeb..fafb1bc66306e 100644 --- a/tests/pulsar-kafka-compat-client-test/src/test/java/org/apache/pulsar/tests/integration/compat/kafka/KafkaApiTest.java +++ b/tests/pulsar-kafka-compat-client-test/src/test/java/org/apache/pulsar/tests/integration/compat/kafka/KafkaApiTest.java @@ -71,8 +71,6 @@ public class KafkaApiTest extends PulsarStandaloneTestSuite { @Data - @ToString - @EqualsAndHashCode public static class Foo { @Nullable private String field1; @@ -82,8 +80,6 @@ public static class Foo { } @Data - @ToString - @EqualsAndHashCode public static class Bar { private boolean field1; } From 848ad30f1cfd4dd731a6c6ab31ffb2e19b1280d5 Mon Sep 17 00:00:00 2001 From: lipenghui Date: Wed, 26 Feb 2020 02:41:31 +0800 Subject: [PATCH 28/74] Add verification for SchemaDefinitionBuilderImpl.java (#6405) ### Motivation Add verification for SchemaDefinitionBuilderImpl.java ### Verifying this change Added a new unit test. --- .../schema/SchemaDefinitionBuilderImpl.java | 9 ++++ .../api/SchemaDefinitionBuilderTest.java | 42 +++++++++++++++++++ 2 files changed, 51 insertions(+) create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/api/SchemaDefinitionBuilderTest.java diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/SchemaDefinitionBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/SchemaDefinitionBuilderImpl.java index ea9ede51f23e9..2a0cab72efb05 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/SchemaDefinitionBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/SchemaDefinitionBuilderImpl.java @@ -18,6 +18,9 @@ */ package org.apache.pulsar.client.impl.schema; +import static com.google.common.base.Preconditions.checkArgument; + +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.schema.SchemaDefinition; import org.apache.pulsar.client.api.schema.SchemaDefinitionBuilder; @@ -98,6 +101,12 @@ public SchemaDefinitionBuilder withProperties(Map properties) @Override public SchemaDefinition build() { + checkArgument(StringUtils.isNotBlank(jsonDef) || clazz != null, + "Must specify one of the pojo or jsonDef for the schema definition."); + + checkArgument(!(StringUtils.isNotBlank(jsonDef) && clazz != null), + "Not allowed to set pojo and jsonDef both for the schema definition."); + properties.put(ALWAYS_ALLOW_NULL, this.alwaysAllowNull ? "true" : "false"); return new SchemaDefinitionImpl(clazz, jsonDef, alwaysAllowNull, properties, supportSchemaVersioning); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/api/SchemaDefinitionBuilderTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/api/SchemaDefinitionBuilderTest.java new file mode 100644 index 0000000000000..90d3a2788f7a1 --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/api/SchemaDefinitionBuilderTest.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.api; + +import org.apache.pulsar.client.api.schema.SchemaDefinition; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class SchemaDefinitionBuilderTest { + + @Test + public void testVerification() { + try { + SchemaDefinition.builder().build(); + Assert.fail("should failed"); + } catch (IllegalArgumentException ignore) { + } + try { + SchemaDefinition.builder().withJsonDef("{}").withPojo(Object.class).build(); + Assert.fail("should failed"); + } catch (IllegalArgumentException ignore) { + } + SchemaDefinition.builder().withJsonDef("{}").build(); + SchemaDefinition.builder().withPojo(Object.class).build(); + } +} From fa4669301af70063f25968b21495f7a2c6a89b44 Mon Sep 17 00:00:00 2001 From: Sergii Zhevzhyk Date: Wed, 26 Feb 2020 10:14:23 +0100 Subject: [PATCH 29/74] Cleanup pom files in the tests module (#6421) ### Modifications - Removed dependencies on test libraries that were already imported in the parent pom file. - Removed groupId tags that are inherited from the parent pom file. --- tests/bc_2_0_0/pom.xml | 6 ------ tests/bc_2_0_1/pom.xml | 6 ------ tests/docker-images/java-test-functions/pom.xml | 1 - tests/docker-images/latest-version-image/pom.xml | 1 - tests/docker-images/pom.xml | 1 - tests/integration/pom.xml | 5 ----- tests/pulsar-spark-test/pom.xml | 6 ------ 7 files changed, 26 deletions(-) diff --git a/tests/bc_2_0_0/pom.xml b/tests/bc_2_0_0/pom.xml index 2619d44b4640b..f691174ec27ca 100644 --- a/tests/bc_2_0_0/pom.xml +++ b/tests/bc_2_0_0/pom.xml @@ -34,12 +34,6 @@ - - org.testng - testng - test - - com.google.code.gson gson diff --git a/tests/bc_2_0_1/pom.xml b/tests/bc_2_0_1/pom.xml index 67e762133e7ff..205b746e674d9 100644 --- a/tests/bc_2_0_1/pom.xml +++ b/tests/bc_2_0_1/pom.xml @@ -34,12 +34,6 @@ - - org.testng - testng - test - - com.google.code.gson gson diff --git a/tests/docker-images/java-test-functions/pom.xml b/tests/docker-images/java-test-functions/pom.xml index aad909b6231de..c801c00b945cc 100644 --- a/tests/docker-images/java-test-functions/pom.xml +++ b/tests/docker-images/java-test-functions/pom.xml @@ -26,7 +26,6 @@ 2.6.0-SNAPSHOT 4.0.0 - org.apache.pulsar.tests java-test-functions Apache Pulsar :: Tests :: Docker Images :: Java Test Functions diff --git a/tests/docker-images/latest-version-image/pom.xml b/tests/docker-images/latest-version-image/pom.xml index 5a961b016f2f7..fabbbc774370c 100644 --- a/tests/docker-images/latest-version-image/pom.xml +++ b/tests/docker-images/latest-version-image/pom.xml @@ -26,7 +26,6 @@ 2.6.0-SNAPSHOT 4.0.0 - org.apache.pulsar.tests latest-version-image Apache Pulsar :: Tests :: Docker Images :: Latest Version Testing pom diff --git a/tests/docker-images/pom.xml b/tests/docker-images/pom.xml index ca052a2f2f8e2..8241abf52454f 100644 --- a/tests/docker-images/pom.xml +++ b/tests/docker-images/pom.xml @@ -28,7 +28,6 @@ tests-parent 2.6.0-SNAPSHOT - org.apache.pulsar.tests docker-images Apache Pulsar :: Tests :: Docker Images diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml index 6f24ed5d7b9cd..4a3016e42e888 100644 --- a/tests/integration/pom.xml +++ b/tests/integration/pom.xml @@ -37,11 +37,6 @@ - - org.testng - testng - test - com.google.code.gson gson diff --git a/tests/pulsar-spark-test/pom.xml b/tests/pulsar-spark-test/pom.xml index 3d286cb672f3b..3d9856f61058e 100644 --- a/tests/pulsar-spark-test/pom.xml +++ b/tests/pulsar-spark-test/pom.xml @@ -56,12 +56,6 @@ test - - org.mockito - mockito-core - test - - org.apache.spark spark-streaming_2.10 From 792ab172ce0419097fdf10eae156ba95b61abc21 Mon Sep 17 00:00:00 2001 From: lipenghui Date: Wed, 26 Feb 2020 17:14:46 +0800 Subject: [PATCH 30/74] Update BatchReceivePolicy.java (#6423) BatchReceivePolicy implements Serializable. --- .../org/apache/pulsar/client/api/BatchReceivePolicy.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/BatchReceivePolicy.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/BatchReceivePolicy.java index 7a5a1bd19d2cc..f331b1b048bbd 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/BatchReceivePolicy.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/BatchReceivePolicy.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import java.io.Serializable; import java.util.concurrent.TimeUnit; /** @@ -41,7 +42,9 @@ * * @since 2.4.1 */ -public class BatchReceivePolicy { +public class BatchReceivePolicy implements Serializable { + + private static final long serialVersionUID = 1L; /** * Default batch receive policy. From e71b9fc4e256f24c9b6c0edd14e40e8af1f24374 Mon Sep 17 00:00:00 2001 From: liudezhi <33149602+liudezhi2098@users.noreply.github.com> Date: Wed, 26 Feb 2020 19:26:15 +0800 Subject: [PATCH 31/74] Consumer received duplicated deplayed messages upon restart Fix when send a delayed message ,there is a case when a consumer restarts and pull duplicate messages. #6403 --- .../bookkeeper/mledger/impl/ManagedCursorImpl.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 9498145de2823..05f6f5c762133 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -1116,7 +1116,14 @@ public synchronized void readEntryFailed(ManagedLedgerException mle, Object ctx) }; positions.stream().filter(position -> !alreadyAcknowledgedPositions.contains(position)) - .forEach(p -> ledger.asyncReadEntry((PositionImpl) p, cb, ctx)); + .forEach(p ->{ + if (((PositionImpl) p).compareTo(this.readPosition) == 0) { + this.setReadPosition(this.readPosition.getNext()); + log.warn("[{}][{}] replayPosition{} equals readPosition{}," + " need set next readPositio", + ledger.getName(), name, (PositionImpl) p, this.readPosition); + } + ledger.asyncReadEntry((PositionImpl) p, cb, ctx); + }); return alreadyAcknowledgedPositions; } From 760bd1abfa5374bde85e4d938937a0410dfb2674 Mon Sep 17 00:00:00 2001 From: Dzmitry Kazimirchyk Date: Wed, 26 Feb 2020 22:43:57 -0800 Subject: [PATCH 32/74] Bump netty version to 4.1.45.Final (#6424) netty 4.1.43 has a bug preventing it from using Linux native Epoll transport This results in pulsar brokers failing over to NioEventLoopGroup even when running on Linux. The bug is fixed in netty releases 4.1.45.Final --- .../server/src/assemble/LICENSE.bin.txt | 32 +++++++++---------- pom.xml | 2 +- pulsar-sql/presto-distribution/LICENSE | 30 ++++++++--------- 3 files changed, 32 insertions(+), 32 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index a817987f0006b..3e17c8723e7ca 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -346,22 +346,22 @@ The Apache Software License, Version 2.0 - org.apache.commons-commons-compress-1.19.jar - org.apache.commons-commons-lang3-3.4.jar * Netty - - io.netty-netty-buffer-4.1.43.Final.jar - - io.netty-netty-codec-4.1.43.Final.jar - - io.netty-netty-codec-dns-4.1.43.Final.jar - - io.netty-netty-codec-http-4.1.43.Final.jar - - io.netty-netty-codec-http2-4.1.43.Final.jar - - io.netty-netty-codec-socks-4.1.43.Final.jar - - io.netty-netty-common-4.1.43.Final.jar - - io.netty-netty-handler-4.1.43.Final.jar - - io.netty-netty-handler-proxy-4.1.43.Final.jar - - io.netty-netty-resolver-4.1.43.Final.jar - - io.netty-netty-resolver-dns-4.1.43.Final.jar - - io.netty-netty-transport-4.1.43.Final.jar - - io.netty-netty-transport-native-epoll-4.1.43.Final-linux-x86_64.jar - - io.netty-netty-transport-native-epoll-4.1.43.Final.jar - - io.netty-netty-transport-native-unix-common-4.1.43.Final.jar - - io.netty-netty-transport-native-unix-common-4.1.43.Final-linux-x86_64.jar + - io.netty-netty-buffer-4.1.45.Final.jar + - io.netty-netty-codec-4.1.45.Final.jar + - io.netty-netty-codec-dns-4.1.45.Final.jar + - io.netty-netty-codec-http-4.1.45.Final.jar + - io.netty-netty-codec-http2-4.1.45.Final.jar + - io.netty-netty-codec-socks-4.1.45.Final.jar + - io.netty-netty-common-4.1.45.Final.jar + - io.netty-netty-handler-4.1.45.Final.jar + - io.netty-netty-handler-proxy-4.1.45.Final.jar + - io.netty-netty-resolver-4.1.45.Final.jar + - io.netty-netty-resolver-dns-4.1.45.Final.jar + - io.netty-netty-transport-4.1.45.Final.jar + - io.netty-netty-transport-native-epoll-4.1.45.Final-linux-x86_64.jar + - io.netty-netty-transport-native-epoll-4.1.45.Final.jar + - io.netty-netty-transport-native-unix-common-4.1.45.Final.jar + - io.netty-netty-transport-native-unix-common-4.1.45.Final-linux-x86_64.jar - io.netty-netty-tcnative-boringssl-static-2.0.26.Final.jar * Prometheus client - io.prometheus-simpleclient-0.5.0.jar diff --git a/pom.xml b/pom.xml index d6c3ea3e13e1c..6c3c1bcf45cce 100644 --- a/pom.xml +++ b/pom.xml @@ -150,7 +150,7 @@ flexible messaging model and an intuitive client API. 4.10.0 3.5.7 - 4.1.43.Final + 4.1.45.Final 2.0.26.Final 2.0.0 9.4.20.v20190813 diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 32b2396743f0b..e1d08d1dc45e3 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -231,23 +231,23 @@ The Apache Software License, Version 2.0 - commons-lang3-3.4.jar * Netty - netty-3.10.6.Final.jar - - netty-buffer-4.1.43.Final.jar - - netty-codec-4.1.43.Final.jar - - netty-codec-dns-4.1.43.Final.jar - - netty-codec-http-4.1.43.Final.jar - - netty-codec-socks-4.1.43.Final.jar - - netty-common-4.1.43.Final.jar - - netty-handler-4.1.43.Final.jar - - netty-handler-proxy-4.1.43.Final.jar + - netty-buffer-4.1.45.Final.jar + - netty-codec-4.1.45.Final.jar + - netty-codec-dns-4.1.45.Final.jar + - netty-codec-http-4.1.45.Final.jar + - netty-codec-socks-4.1.45.Final.jar + - netty-common-4.1.45.Final.jar + - netty-handler-4.1.45.Final.jar + - netty-handler-proxy-4.1.45.Final.jar - netty-reactive-streams-2.0.0.jar - - netty-resolver-4.1.43.Final.jar - - netty-resolver-dns-4.1.43.Final.jar + - netty-resolver-4.1.45.Final.jar + - netty-resolver-dns-4.1.45.Final.jar - netty-tcnative-boringssl-static-2.0.26.Final.jar - - netty-transport-4.1.43.Final.jar - - netty-transport-native-epoll-4.1.43.Final.jar - - netty-transport-native-epoll-4.1.43.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.43.Final.jar - - netty-transport-native-unix-common-4.1.43.Final-linux-x86_64.jar + - netty-transport-4.1.45.Final.jar + - netty-transport-native-epoll-4.1.45.Final.jar + - netty-transport-native-epoll-4.1.45.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.45.Final.jar + - netty-transport-native-unix-common-4.1.45.Final-linux-x86_64.jar * Joda Time - joda-time-2.9.9.jar - joda-time-2.10.1.jar From 75a321dfbe23cbd979fa225ef4a3d2fc4201994b Mon Sep 17 00:00:00 2001 From: ltamber Date: Thu, 27 Feb 2020 20:04:06 +0800 Subject: [PATCH 33/74] Fix publish buffer limit does not take effect Motivation If set up maxMessagePublishBufferSizeInMB > Integer.MAX_VALUE / 1024 / 1024, the publish buffer limit does not take effect. The reason is maxMessagePublishBufferBytes always 0 when use following calculation method : pulsar.getConfiguration().getMaxMessagePublishBufferSizeInMB() * 1024 * 1024; So, changed to pulsar.getConfiguration().getMaxMessagePublishBufferSizeInMB() * 1024L * 1024L; --- .../java/org/apache/pulsar/broker/service/BrokerService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 7d48f76b71e90..498f1ea504b5d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -226,7 +226,7 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener 0 ? - pulsar.getConfiguration().getMaxMessagePublishBufferSizeInMB() * 1024 * 1024 : -1; + pulsar.getConfiguration().getMaxMessagePublishBufferSizeInMB() * 1024L * 1024L : -1; this.resumeProducerReadMessagePublishBufferBytes = this.maxMessagePublishBufferBytes / 2; this.managedLedgerFactory = pulsar.getManagedLedgerFactory(); this.topics = new ConcurrentOpenHashMap<>(); From f98e092ac81a09eafb3f1c4950b30180daeb55f8 Mon Sep 17 00:00:00 2001 From: futeng Date: Fri, 28 Feb 2020 13:35:40 +0800 Subject: [PATCH 34/74] doc: Add on the missing right parenthesis (#6426) * Add on the missing right parenthesis doc: Missing right parenthesis in the `token()` line from Pulsar Client Java Code. * Add on the missing right parenthesis on line L70 --- site2/docs/security-jwt.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/site2/docs/security-jwt.md b/site2/docs/security-jwt.md index 46cf2e6925ec7..ff2e99efe8701 100644 --- a/site2/docs/security-jwt.md +++ b/site2/docs/security-jwt.md @@ -54,7 +54,7 @@ You can use tokens to authenticate the following Pulsar clients. PulsarClient client = PulsarClient.builder() .serviceUrl("pulsar://broker.example.com:6650/") .authentication( - AuthenticationFactory.token("eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPtypruRC4fb1DWtoLL62SY") + AuthenticationFactory.token("eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPtypruRC4fb1DWtoLL62SY")) .build(); ``` @@ -67,7 +67,7 @@ PulsarClient client = PulsarClient.builder() AuthenticationFactory.token(() -> { // Read token from custom source return readToken(); - }) + })) .build(); ``` From b2899edbaeb78ce4eabb44e8a3ebbac29ce32168 Mon Sep 17 00:00:00 2001 From: Sergii Zhevzhyk Date: Fri, 28 Feb 2020 06:38:13 +0100 Subject: [PATCH 35/74] Switch from deprecated MAINTAINER tag to LABEL with maintainer's info in Dockerfile (#6429) Motivation & Modification The MAINTAINER instruction is deprecated in favor of the LABEL instruction with the maintainer's info in docker files. --- dashboard/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dashboard/Dockerfile b/dashboard/Dockerfile index 7388481f7567e..aae74f784f990 100644 --- a/dashboard/Dockerfile +++ b/dashboard/Dockerfile @@ -19,7 +19,7 @@ FROM python:3.7-stretch -MAINTAINER Pulsar +LABEL maintainer="Apache Pulsar " RUN apt-get update RUN apt-get -y install postgresql python sudo nginx supervisor From 39cf6c0b1b4fd3ce6991be12c45b208acdd60a65 Mon Sep 17 00:00:00 2001 From: Fangbin Sun Date: Fri, 28 Feb 2020 15:18:06 +0800 Subject: [PATCH 36/74] Amend the default value of . (#6374) --- conf/broker.conf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conf/broker.conf b/conf/broker.conf index cf01b2d2fddc4..bfb7f7ceee12d 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -720,7 +720,7 @@ loadManagerClassName=org.apache.pulsar.broker.loadbalance.impl.ModularLoadManage # Supported algorithms name for namespace bundle split. # "range_equally_divide" divides the bundle into two parts with the same hash range size. # "topic_count_equally_divide" divides the bundle into two parts with the same topics count. -supportedNamespaceBundleSplitAlgorithms=[range_equally_divide,topic_count_equally_divide] +supportedNamespaceBundleSplitAlgorithms=range_equally_divide,topic_count_equally_divide # Default algorithm name for namespace bundle split defaultNamespaceBundleSplitAlgorithm=range_equally_divide From b8f0ca092b6c6cbd04525fb81ea55ba516a06cf7 Mon Sep 17 00:00:00 2001 From: bilahepan Date: Sat, 29 Feb 2020 08:59:08 +0800 Subject: [PATCH 37/74] fix the bug of authenticationData is't initialized. (#6440) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Motivation fix the bug of authenticationData is't initialized. the method org.apache.pulsar.proxy.server.ProxyConnection#handleConnect can't init the value of authenticationData. cause of the bug that you will get the null value form the method org.apache.pulsar.broker.authorization.AuthorizationProvider#canConsumeAsync when implements org.apache.pulsar.broker.authorization.AuthorizationProvider interface. Modifications init the value of authenticationData from the method org.apache.pulsar.proxy.server.ProxyConnection#handleConnect. Verifying this change implements org.apache.pulsar.broker.authorization.AuthorizationProvider interface, and get the value of authenticationData. --- .../java/org/apache/pulsar/proxy/server/ProxyConnection.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java index d9960c5a1ec31..9b8b7fb0aed14 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java @@ -327,6 +327,7 @@ protected void handleConnect(CommandConnect connect) { } authState = authenticationProvider.newAuthState(clientData, remoteAddress, sslSession); + authenticationData = authState.getAuthDataSource(); doAuthentication(clientData); } catch (Exception e) { LOG.warn("[{}] Unable to authenticate: ", remoteAddress, e); From 7f25add4060f365a92d314baf7df203e0ea05e1f Mon Sep 17 00:00:00 2001 From: Sergii Zhevzhyk Date: Sat, 29 Feb 2020 21:36:25 +0100 Subject: [PATCH 38/74] Remove duplicated test libraries in POM dependencies (#6430) ### Motivation The removed test libraries were already defined in the parent pom ### Modification Removed duplicated test libraries in POM dependencies --- managed-ledger/pom.xml | 12 ------------ pulsar-broker/pom.xml | 7 ------- pulsar-client-tools-test/pom.xml | 5 ----- pulsar-discovery-service/pom.xml | 7 ------- pulsar-io/hdfs2/pom.xml | 5 ----- pulsar-io/hdfs3/pom.xml | 5 ----- pulsar-io/mongo/pom.xml | 6 ------ pulsar-io/redis/pom.xml | 6 ------ pulsar-proxy/pom.xml | 6 ------ pulsar-sql/presto-pulsar/pom.xml | 13 ------------- 10 files changed, 72 deletions(-) diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index b883c8661d9c4..bbd5b27b892c7 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -101,18 +101,6 @@ slf4j-api - - org.mockito - mockito-core - test - - - - ${project.groupId} - buildtools - ${project.version} - test - diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 43244de13b154..5df3fcb5a4ed6 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -279,13 +279,6 @@ hppc - - ${project.groupId} - buildtools - ${project.version} - test - - ${project.groupId} pulsar-functions-api-examples diff --git a/pulsar-client-tools-test/pom.xml b/pulsar-client-tools-test/pom.xml index 71ede709f002d..7eb85a51fd0a0 100644 --- a/pulsar-client-tools-test/pom.xml +++ b/pulsar-client-tools-test/pom.xml @@ -64,11 +64,6 @@ ${project.version} test - - org.mockito - mockito-core - test - diff --git a/pulsar-discovery-service/pom.xml b/pulsar-discovery-service/pom.xml index 1c13fbd3d6fe7..978c81df7f381 100644 --- a/pulsar-discovery-service/pom.xml +++ b/pulsar-discovery-service/pom.xml @@ -149,12 +149,5 @@ test test-jar - - - ${project.groupId} - buildtools - ${project.version} - test - diff --git a/pulsar-io/hdfs2/pom.xml b/pulsar-io/hdfs2/pom.xml index e7eedda9752bb..d6da9b4217342 100644 --- a/pulsar-io/hdfs2/pom.xml +++ b/pulsar-io/hdfs2/pom.xml @@ -45,11 +45,6 @@ jackson-dataformat-yaml - - org.testng - testng - test - org.apache.hadoop hadoop-client diff --git a/pulsar-io/hdfs3/pom.xml b/pulsar-io/hdfs3/pom.xml index 1ceada45f79ba..40c8595318534 100644 --- a/pulsar-io/hdfs3/pom.xml +++ b/pulsar-io/hdfs3/pom.xml @@ -51,11 +51,6 @@ 3.1.1 - - org.testng - testng - test - diff --git a/pulsar-io/mongo/pom.xml b/pulsar-io/mongo/pom.xml index 6f764c282d74a..877a29dec19d9 100644 --- a/pulsar-io/mongo/pom.xml +++ b/pulsar-io/mongo/pom.xml @@ -63,12 +63,6 @@ com.google.guava guava - - ${project.parent.groupId} - buildtools - ${project.parent.version} - test - diff --git a/pulsar-io/redis/pom.xml b/pulsar-io/redis/pom.xml index deadbc3da29b0..4001fc99c8ad8 100644 --- a/pulsar-io/redis/pom.xml +++ b/pulsar-io/redis/pom.xml @@ -75,12 +75,6 @@ commons-collections 3.2.2 - - ${project.parent.groupId} - buildtools - ${project.parent.version} - test - com.github.kstyrc embedded-redis diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index 3320749649ed6..a9e3ae353001a 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -166,11 +166,5 @@ com.beust jcommander - - - org.mockito - mockito-core - test - diff --git a/pulsar-sql/presto-pulsar/pom.xml b/pulsar-sql/presto-pulsar/pom.xml index 4d184d58c37bd..0d35d413c5001 100644 --- a/pulsar-sql/presto-pulsar/pom.xml +++ b/pulsar-sql/presto-pulsar/pom.xml @@ -88,19 +88,6 @@ ${presto.version} provided - - - - org.testng - testng - test - - - - org.mockito - mockito-core - test - From 4f461c3bd36ad832789a6f9186f77da151aba5a5 Mon Sep 17 00:00:00 2001 From: Fangbin Sun Date: Sun, 1 Mar 2020 04:41:55 +0800 Subject: [PATCH 39/74] Add a message on how to make log refresh immediately when starting a component (#6078) ### Motivation Some users may confuse by pulsar/bookie log without flushing immediately. ### Modifications Add a message in `bin/pulsar-daemon` when starting a component. --- bin/pulsar-daemon | 1 + 1 file changed, 1 insertion(+) diff --git a/bin/pulsar-daemon b/bin/pulsar-daemon index 3d555dd09fbbc..196aaa7272fdc 100755 --- a/bin/pulsar-daemon +++ b/bin/pulsar-daemon @@ -151,6 +151,7 @@ case $startStop in rotate_out_log $out echo starting $command, logging to $logfile + echo Note: Set immediateFlush to true in conf/log4j2.yaml will guarantee the logging event is flushing to disk immediately. The default behavior is switched off due to performance considerations. pulsar=$PULSAR_HOME/bin/pulsar nohup $pulsar $command "$@" > "$out" 2>&1 < /dev/null & echo $! > $pid From e6a631d491948119c07a30339c7daf71002c5c0a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Pawe=C5=82=20=C5=81ozi=C5=84ski?= Date: Sat, 29 Feb 2020 21:53:15 +0100 Subject: [PATCH 40/74] Close ZK before canceling future with exception (#6228) (#6399) Fixes #6228 --- .../apache/pulsar/zookeeper/ZookeeperBkClientFactoryImpl.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZookeeperBkClientFactoryImpl.java b/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZookeeperBkClientFactoryImpl.java index aac96ce84c5c1..8cd674d580dfe 100644 --- a/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZookeeperBkClientFactoryImpl.java +++ b/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZookeeperBkClientFactoryImpl.java @@ -55,6 +55,7 @@ public CompletableFuture create(String serverList, SessionType sessio .build(); if (zk.getState() == States.CONNECTEDREADONLY && sessionType != SessionType.AllowReadOnly) { + zk.close(); future.completeExceptionally(new IllegalStateException("Cannot use a read-only session")); } From 2ed2eb86e50d4515bee570c339b2719614a86ecc Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Sun, 1 Mar 2020 17:28:56 +0800 Subject: [PATCH 41/74] [Flink-Connector]Get PulsarClient from cache should always return an open instance (#6436) --- .../pulsar/client/impl/PulsarClientImpl.java | 6 ++++- .../connectors/pulsar/CachedPulsarClient.java | 8 +++++- .../pulsar/CachedPulsarClientTest.java | 25 +++++++++++++++++++ 3 files changed, 37 insertions(+), 2 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index f51fb6b65c3f3..db66c90ca7150 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -94,7 +94,7 @@ public class PulsarClientImpl implements PulsarClient { private final Timer timer; private final ExecutorProvider externalExecutorProvider; - enum State { + public enum State { Open, Closing, Closed } @@ -167,6 +167,10 @@ public Clock getClientClock() { return clientClock; } + public AtomicReference getState() { + return state; + } + @Override public ProducerBuilder newProducer() { return new ProducerBuilderImpl<>(this, Schema.BYTES); diff --git a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClient.java b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClient.java index 5d5715e846d9e..4de514520e19f 100644 --- a/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClient.java +++ b/pulsar-flink/src/main/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClient.java @@ -77,7 +77,13 @@ private static PulsarClientImpl createPulsarClient( } public static PulsarClientImpl getOrCreate(ClientConfigurationData config) throws ExecutionException { - return guavaCache.get(config); + PulsarClientImpl instance = guavaCache.get(config); + if (instance.getState().get() == PulsarClientImpl.State.Open) { + return instance; + } else { + guavaCache.invalidate(config); + return guavaCache.get(config); + } } private static void close(ClientConfigurationData clientConfig, PulsarClientImpl client) { diff --git a/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClientTest.java b/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClientTest.java index a41609f4a2ba2..39cdca19a37cc 100644 --- a/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClientTest.java +++ b/pulsar-flink/src/test/java/org/apache/flink/streaming/connectors/pulsar/CachedPulsarClientTest.java @@ -100,4 +100,29 @@ public void testShouldCloseTheCorrectClient() throws Exception { assertEquals(map2.values().iterator().next(), client1); } + + @Test + public void getClientFromCacheShouldAlwaysReturnAnOpenedInstance() throws Exception { + PulsarClientImpl impl1 = Mockito.mock(PulsarClientImpl.class); + + ClientConfigurationData conf1 = new ClientConfigurationData(); + conf1.setServiceUrl(SERVICE_URL); + + PowerMockito.whenNew(PulsarClientImpl.class) + .withArguments(conf1).thenReturn(impl1); + + PulsarClientImpl client1 = CachedPulsarClient.getOrCreate(conf1); + + ConcurrentMap map1 = CachedPulsarClient.getAsMap(); + assertEquals(map1.size(), 1); + + client1.getState().set(PulsarClientImpl.State.Closed); + + PulsarClientImpl client2 = CachedPulsarClient.getOrCreate(conf1); + + assertNotEquals(client1, client2); + + ConcurrentMap map2 = CachedPulsarClient.getAsMap(); + assertEquals(map2.size(), 1); + } } From c3672a238dd698bfcf0a4831b9bb050e620fad6d Mon Sep 17 00:00:00 2001 From: Ryan Slominski Date: Mon, 2 Mar 2020 02:52:45 -0500 Subject: [PATCH 42/74] Update sidebars.json (#6434) The referenced markdown files do not exist and so the "Next" and "Previous" buttons on the bottom of pages surrounding them result in 404 Not Found errors --- site2/website/sidebars.json | 4 ---- 1 file changed, 4 deletions(-) diff --git a/site2/website/sidebars.json b/site2/website/sidebars.json index 828f5ea155ab5..6ebbea42a1b1b 100644 --- a/site2/website/sidebars.json +++ b/site2/website/sidebars.json @@ -30,10 +30,6 @@ "functions-develop", "functions-debug", "functions-deploy", - "functions-configure", - "functions-monitor", - "functions-secure", - "functions-troubleshoot", "functions-cli" ], "Pulsar IO": [ From 3e1b8f644446e4f63969be621798f14628e53065 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Tue, 3 Mar 2020 03:22:14 +0800 Subject: [PATCH 43/74] [Broker] Create namespace failed when TLS is enabled in PulsarStandalone (#6457) When starting Pulsar in standalone mode with TLS enabled, it will fail to create two namespaces during start. This is because it's using the unencrypted URL/port while constructing the PulsarAdmin client. --- .../org/apache/pulsar/PulsarStandalone.java | 32 ++++++++++++------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java index 7c82cbe7098d0..7c7625770b14f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java @@ -319,16 +319,28 @@ public void start() throws Exception { broker.getTransactionMetadataStoreService().addTransactionMetadataStore(TransactionCoordinatorID.get(0)); - URL webServiceUrl = new URL( - String.format("http://%s:%d", config.getAdvertisedAddress(), config.getWebServicePort().get())); - final String brokerServiceUrl = String.format("pulsar://%s:%d", config.getAdvertisedAddress(), - config.getBrokerServicePort().get()); - admin = PulsarAdmin.builder().serviceHttpUrl(webServiceUrl.toString()).authentication( - config.getBrokerClientAuthenticationPlugin(), config.getBrokerClientAuthenticationParameters()).build(); - final String cluster = config.getClusterName(); - createSampleNameSpace(webServiceUrl, brokerServiceUrl, cluster); + if (!config.isTlsEnabled()) { + URL webServiceUrl = new URL( + String.format("http://%s:%d", config.getAdvertisedAddress(), config.getWebServicePort().get())); + String brokerServiceUrl = String.format("pulsar://%s:%d", config.getAdvertisedAddress(), + config.getBrokerServicePort().get()); + admin = PulsarAdmin.builder().serviceHttpUrl(webServiceUrl.toString()).authentication( + config.getBrokerClientAuthenticationPlugin(), config.getBrokerClientAuthenticationParameters()).build(); + ClusterData clusterData = new ClusterData(webServiceUrl.toString(), null, brokerServiceUrl, null); + createSampleNameSpace(clusterData, cluster); + } else { + URL webServiceUrlTls = new URL( + String.format("http://%s:%d", config.getAdvertisedAddress(), config.getWebServicePortTls().get())); + String brokerServiceUrlTls = String.format("pulsar+ssl://%s:%d", config.getAdvertisedAddress(), + config.getBrokerServicePortTls().get()); + admin = PulsarAdmin.builder().serviceHttpUrl(webServiceUrlTls.toString()).authentication( + config.getBrokerClientAuthenticationPlugin(), config.getBrokerClientAuthenticationParameters()).build(); + ClusterData clusterData = new ClusterData(null, webServiceUrlTls.toString(), null, brokerServiceUrlTls); + createSampleNameSpace(clusterData, cluster); + } + createDefaultNameSpace(cluster); log.debug("--- setup completed ---"); @@ -352,14 +364,12 @@ private void createDefaultNameSpace(String cluster) { } } - private void createSampleNameSpace(URL webServiceUrl, String brokerServiceUrl, String cluster) { + private void createSampleNameSpace(ClusterData clusterData, String cluster) { // Create a sample namespace final String property = "sample"; final String globalCluster = "global"; final String namespace = property + "/" + cluster + "/ns1"; try { - ClusterData clusterData = new ClusterData(webServiceUrl.toString(), null /* serviceUrlTls */, - brokerServiceUrl, null /* brokerServiceUrlTls */); if (!admin.clusters().getClusters().contains(cluster)) { admin.clusters().createCluster(cluster, clusterData); } else { From 330e78225aae48c6b732f4a4e621cf085813293f Mon Sep 17 00:00:00 2001 From: Ryan Slominski Date: Mon, 2 Mar 2020 14:44:52 -0500 Subject: [PATCH 44/74] Update version-2.5.0-sidebars.json (#6455) The referenced markdown files do not exist and so the "Next" and "Previous" buttons on the bottom of pages surrounding them result in 404 Not Found errors --- site2/website/versioned_sidebars/version-2.5.0-sidebars.json | 4 ---- 1 file changed, 4 deletions(-) diff --git a/site2/website/versioned_sidebars/version-2.5.0-sidebars.json b/site2/website/versioned_sidebars/version-2.5.0-sidebars.json index c9850988bef59..50c92fdd95937 100644 --- a/site2/website/versioned_sidebars/version-2.5.0-sidebars.json +++ b/site2/website/versioned_sidebars/version-2.5.0-sidebars.json @@ -31,10 +31,6 @@ "version-2.5.0-functions-develop", "version-2.5.0-functions-debug", "version-2.5.0-functions-deploy", - "version-2.5.0-functions-configure", - "version-2.5.0-functions-monitor", - "version-2.5.0-functions-secure", - "version-2.5.0-functions-troubleshoot", "version-2.5.0-functions-cli" ], "Pulsar IO": [ From 333888ad61c062f9e3d2946918ffc21fafd441af Mon Sep 17 00:00:00 2001 From: k2la Date: Tue, 3 Mar 2020 04:55:33 +0900 Subject: [PATCH 45/74] [Issue 6168] Fix Unacked Message Tracker by Using Time Partition on C++ (#6391) ### Motivation Fix #6168 . >On C++ lib, like the following log, unacked messages are redelivered after about 2 * unAckedMessagesTimeout. ### Modifications As same #3118, by using TimePartition, fixed ` UnackedMessageTracker` . - Add `TickDurationInMs` - Add `redeliverUnacknowledgedMessages` which require `MessageIds` to `ConsumerImpl`, `MultiTopicsConsumerImpl` and `PartitionedConsumerImpl`. --- .../include/pulsar/ConsumerConfiguration.h | 4 + .../include/pulsar/c/producer_configuration.h | 2 +- .../lib/ConsumerConfiguration.cc | 6 + .../lib/ConsumerConfigurationImpl.h | 2 + pulsar-client-cpp/lib/ConsumerImpl.cc | 21 +++- pulsar-client-cpp/lib/ConsumerImpl.h | 1 + pulsar-client-cpp/lib/ConsumerImplBase.h | 1 + pulsar-client-cpp/lib/LogUtils.cc | 2 +- .../lib/MultiTopicsConsumerImpl.cc | 25 ++++- .../lib/MultiTopicsConsumerImpl.h | 1 + .../lib/PartitionedConsumerImpl.cc | 24 +++- .../lib/PartitionedConsumerImpl.h | 1 + .../lib/UnAckedMessageTrackerEnabled.cc | 106 +++++++++++------- .../lib/UnAckedMessageTrackerEnabled.h | 6 +- 14 files changed, 152 insertions(+), 50 deletions(-) diff --git a/pulsar-client-cpp/include/pulsar/ConsumerConfiguration.h b/pulsar-client-cpp/include/pulsar/ConsumerConfiguration.h index 08b7c545eea3c..5468b378b7f92 100644 --- a/pulsar-client-cpp/include/pulsar/ConsumerConfiguration.h +++ b/pulsar-client-cpp/include/pulsar/ConsumerConfiguration.h @@ -155,6 +155,10 @@ class PULSAR_PUBLIC ConsumerConfiguration { */ long getUnAckedMessagesTimeoutMs() const; + void setTickDurationInMs(const uint64_t milliSeconds); + + long getTickDurationInMs() const; + /** * Set the delay to wait before re-delivering messages that have failed to be process. *

diff --git a/pulsar-client-cpp/include/pulsar/c/producer_configuration.h b/pulsar-client-cpp/include/pulsar/c/producer_configuration.h index c846451f9d8a3..1fe44e082fc62 100644 --- a/pulsar-client-cpp/include/pulsar/c/producer_configuration.h +++ b/pulsar-client-cpp/include/pulsar/c/producer_configuration.h @@ -178,4 +178,4 @@ PULSAR_PUBLIC void pulsar_producer_configuration_set_property(pulsar_producer_co #ifdef __cplusplus } -#endif \ No newline at end of file +#endif diff --git a/pulsar-client-cpp/lib/ConsumerConfiguration.cc b/pulsar-client-cpp/lib/ConsumerConfiguration.cc index 38fa1fe8b10b1..546b8b97cb2fb 100644 --- a/pulsar-client-cpp/lib/ConsumerConfiguration.cc +++ b/pulsar-client-cpp/lib/ConsumerConfiguration.cc @@ -98,6 +98,12 @@ void ConsumerConfiguration::setUnAckedMessagesTimeoutMs(const uint64_t milliSeco impl_->unAckedMessagesTimeoutMs = milliSeconds; } +long ConsumerConfiguration::getTickDurationInMs() const { return impl_->tickDurationInMs; } + +void ConsumerConfiguration::setTickDurationInMs(const uint64_t milliSeconds) { + impl_->tickDurationInMs = milliSeconds; +} + void ConsumerConfiguration::setNegativeAckRedeliveryDelayMs(long redeliveryDelayMillis) { impl_->negativeAckRedeliveryDelayMs = redeliveryDelayMillis; } diff --git a/pulsar-client-cpp/lib/ConsumerConfigurationImpl.h b/pulsar-client-cpp/lib/ConsumerConfigurationImpl.h index 55dafd38aaa6b..8dd12633a79fc 100644 --- a/pulsar-client-cpp/lib/ConsumerConfigurationImpl.h +++ b/pulsar-client-cpp/lib/ConsumerConfigurationImpl.h @@ -27,6 +27,7 @@ namespace pulsar { struct ConsumerConfigurationImpl { SchemaInfo schemaInfo; long unAckedMessagesTimeoutMs; + long tickDurationInMs; long negativeAckRedeliveryDelayMs; ConsumerType consumerType; @@ -45,6 +46,7 @@ struct ConsumerConfigurationImpl { ConsumerConfigurationImpl() : schemaInfo(), unAckedMessagesTimeoutMs(0), + tickDurationInMs(1000), negativeAckRedeliveryDelayMs(60000), consumerType(ConsumerExclusive), messageListener(), diff --git a/pulsar-client-cpp/lib/ConsumerImpl.cc b/pulsar-client-cpp/lib/ConsumerImpl.cc index e90aef9b29fcf..6e6b1058b712f 100644 --- a/pulsar-client-cpp/lib/ConsumerImpl.cc +++ b/pulsar-client-cpp/lib/ConsumerImpl.cc @@ -66,8 +66,13 @@ ConsumerImpl::ConsumerImpl(const ClientImplPtr client, const std::string& topic, consumerStrStream << "[" << topic_ << ", " << subscription_ << ", " << consumerId_ << "] "; consumerStr_ = consumerStrStream.str(); if (conf.getUnAckedMessagesTimeoutMs() != 0) { - unAckedMessageTrackerPtr_.reset( - new UnAckedMessageTrackerEnabled(conf.getUnAckedMessagesTimeoutMs(), client, *this)); + if (conf.getTickDurationInMs() > 0) { + unAckedMessageTrackerPtr_.reset(new UnAckedMessageTrackerEnabled( + conf.getUnAckedMessagesTimeoutMs(), conf.getTickDurationInMs(), client, *this)); + } else { + unAckedMessageTrackerPtr_.reset( + new UnAckedMessageTrackerEnabled(conf.getUnAckedMessagesTimeoutMs(), client, *this)); + } } else { unAckedMessageTrackerPtr_.reset(new UnAckedMessageTrackerDisabled()); } @@ -953,6 +958,18 @@ Result ConsumerImpl::resumeMessageListener() { void ConsumerImpl::redeliverUnacknowledgedMessages() { static std::set emptySet; redeliverMessages(emptySet); + unAckedMessageTrackerPtr_->clear(); +} + +void ConsumerImpl::redeliverUnacknowledgedMessages(const std::set& messageIds) { + if (messageIds.empty()) { + return; + } + if (config_.getConsumerType() != ConsumerShared && config_.getConsumerType() != ConsumerKeyShared) { + redeliverUnacknowledgedMessages(); + return; + } + redeliverMessages(messageIds); } void ConsumerImpl::redeliverMessages(const std::set& messageIds) { diff --git a/pulsar-client-cpp/lib/ConsumerImpl.h b/pulsar-client-cpp/lib/ConsumerImpl.h index 8a25b499c1b1d..6d81fd0ab4361 100644 --- a/pulsar-client-cpp/lib/ConsumerImpl.h +++ b/pulsar-client-cpp/lib/ConsumerImpl.h @@ -98,6 +98,7 @@ class ConsumerImpl : public ConsumerImplBase, virtual bool isCumulativeAcknowledgementAllowed(ConsumerType consumerType); virtual void redeliverMessages(const std::set& messageIds); + virtual void redeliverUnacknowledgedMessages(const std::set& messageIds); virtual void negativeAcknowledge(const MessageId& msgId); virtual void closeAsync(ResultCallback callback); diff --git a/pulsar-client-cpp/lib/ConsumerImplBase.h b/pulsar-client-cpp/lib/ConsumerImplBase.h index ab6ed9f14bb83..fc1506643ca1d 100644 --- a/pulsar-client-cpp/lib/ConsumerImplBase.h +++ b/pulsar-client-cpp/lib/ConsumerImplBase.h @@ -48,6 +48,7 @@ class ConsumerImplBase { virtual Result pauseMessageListener() = 0; virtual Result resumeMessageListener() = 0; virtual void redeliverUnacknowledgedMessages() = 0; + virtual void redeliverUnacknowledgedMessages(const std::set& messageIds) = 0; virtual const std::string& getName() const = 0; virtual int getNumOfPrefetchedMessages() const = 0; virtual void getBrokerConsumerStatsAsync(BrokerConsumerStatsCallback callback) = 0; diff --git a/pulsar-client-cpp/lib/LogUtils.cc b/pulsar-client-cpp/lib/LogUtils.cc index e2615a54e6b80..e4f6a1767f873 100644 --- a/pulsar-client-cpp/lib/LogUtils.cc +++ b/pulsar-client-cpp/lib/LogUtils.cc @@ -55,4 +55,4 @@ std::string LogUtils::getLoggerName(const std::string& path) { return path.substr(startIdx + 1, endIdx - startIdx - 1); } -} // namespace pulsar \ No newline at end of file +} // namespace pulsar diff --git a/pulsar-client-cpp/lib/MultiTopicsConsumerImpl.cc b/pulsar-client-cpp/lib/MultiTopicsConsumerImpl.cc index c1ee3e945753b..e7102f7a31685 100644 --- a/pulsar-client-cpp/lib/MultiTopicsConsumerImpl.cc +++ b/pulsar-client-cpp/lib/MultiTopicsConsumerImpl.cc @@ -45,8 +45,13 @@ MultiTopicsConsumerImpl::MultiTopicsConsumerImpl(ClientImplPtr client, const std consumerStr_ = consumerStrStream.str(); if (conf.getUnAckedMessagesTimeoutMs() != 0) { - unAckedMessageTrackerPtr_.reset( - new UnAckedMessageTrackerEnabled(conf.getUnAckedMessagesTimeoutMs(), client, *this)); + if (conf.getTickDurationInMs() > 0) { + unAckedMessageTrackerPtr_.reset(new UnAckedMessageTrackerEnabled( + conf.getUnAckedMessagesTimeoutMs(), conf.getTickDurationInMs(), client, *this)); + } else { + unAckedMessageTrackerPtr_.reset( + new UnAckedMessageTrackerEnabled(conf.getUnAckedMessagesTimeoutMs(), client, *this)); + } } else { unAckedMessageTrackerPtr_.reset(new UnAckedMessageTrackerDisabled()); } @@ -653,6 +658,22 @@ void MultiTopicsConsumerImpl::redeliverUnacknowledgedMessages() { consumer++) { (consumer->second)->redeliverUnacknowledgedMessages(); } + unAckedMessageTrackerPtr_->clear(); +} + +void MultiTopicsConsumerImpl::redeliverUnacknowledgedMessages(const std::set& messageIds) { + if (messageIds.empty()) { + return; + } + if (conf_.getConsumerType() != ConsumerShared && conf_.getConsumerType() != ConsumerKeyShared) { + redeliverUnacknowledgedMessages(); + return; + } + LOG_DEBUG("Sending RedeliverUnacknowledgedMessages command for partitioned consumer."); + for (ConsumerMap::const_iterator consumer = consumers_.begin(); consumer != consumers_.end(); + consumer++) { + (consumer->second)->redeliverUnacknowledgedMessages(messageIds); + } } int MultiTopicsConsumerImpl::getNumOfPrefetchedMessages() const { return messages_.size(); } diff --git a/pulsar-client-cpp/lib/MultiTopicsConsumerImpl.h b/pulsar-client-cpp/lib/MultiTopicsConsumerImpl.h index d190664c33a99..fa271febd4db4 100644 --- a/pulsar-client-cpp/lib/MultiTopicsConsumerImpl.h +++ b/pulsar-client-cpp/lib/MultiTopicsConsumerImpl.h @@ -69,6 +69,7 @@ class MultiTopicsConsumerImpl : public ConsumerImplBase, virtual Result pauseMessageListener(); virtual Result resumeMessageListener(); virtual void redeliverUnacknowledgedMessages(); + virtual void redeliverUnacknowledgedMessages(const std::set& messageIds); virtual int getNumOfPrefetchedMessages() const; virtual void getBrokerConsumerStatsAsync(BrokerConsumerStatsCallback callback); void handleGetConsumerStats(Result, BrokerConsumerStats, LatchPtr, MultiTopicsBrokerConsumerStatsPtr, diff --git a/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc b/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc index 03fd2d2b9ca02..0241a5439bf64 100644 --- a/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc +++ b/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc @@ -43,8 +43,13 @@ PartitionedConsumerImpl::PartitionedConsumerImpl(ClientImplPtr client, const std consumerStrStream << "[Partitioned Consumer: " << topic_ << "," << subscriptionName << "," << numPartitions << "]"; if (conf.getUnAckedMessagesTimeoutMs() != 0) { - unAckedMessageTrackerPtr_.reset( - new UnAckedMessageTrackerEnabled(conf.getUnAckedMessagesTimeoutMs(), client, *this)); + if (conf.getTickDurationInMs() > 0) { + unAckedMessageTrackerPtr_.reset(new UnAckedMessageTrackerEnabled( + conf.getUnAckedMessagesTimeoutMs(), conf.getTickDurationInMs(), client, *this)); + } else { + unAckedMessageTrackerPtr_.reset( + new UnAckedMessageTrackerEnabled(conf.getUnAckedMessagesTimeoutMs(), client, *this)); + } } else { unAckedMessageTrackerPtr_.reset(new UnAckedMessageTrackerDisabled()); } @@ -426,6 +431,21 @@ void PartitionedConsumerImpl::redeliverUnacknowledgedMessages() { for (ConsumerList::const_iterator i = consumers_.begin(); i != consumers_.end(); i++) { (*i)->redeliverUnacknowledgedMessages(); } + unAckedMessageTrackerPtr_->clear(); +} + +void PartitionedConsumerImpl::redeliverUnacknowledgedMessages(const std::set& messageIds) { + if (messageIds.empty()) { + return; + } + if (conf_.getConsumerType() != ConsumerShared && conf_.getConsumerType() != ConsumerKeyShared) { + redeliverUnacknowledgedMessages(); + return; + } + LOG_DEBUG("Sending RedeliverUnacknowledgedMessages command for partitioned consumer."); + for (ConsumerList::const_iterator i = consumers_.begin(); i != consumers_.end(); i++) { + (*i)->redeliverUnacknowledgedMessages(messageIds); + } } const std::string& PartitionedConsumerImpl::getName() const { return partitionStr_; } diff --git a/pulsar-client-cpp/lib/PartitionedConsumerImpl.h b/pulsar-client-cpp/lib/PartitionedConsumerImpl.h index 71b2a30442e63..fb4b04742bbae 100644 --- a/pulsar-client-cpp/lib/PartitionedConsumerImpl.h +++ b/pulsar-client-cpp/lib/PartitionedConsumerImpl.h @@ -64,6 +64,7 @@ class PartitionedConsumerImpl : public ConsumerImplBase, virtual Result pauseMessageListener(); virtual Result resumeMessageListener(); virtual void redeliverUnacknowledgedMessages(); + virtual void redeliverUnacknowledgedMessages(const std::set& messageIds); virtual const std::string& getName() const; virtual int getNumOfPrefetchedMessages() const; virtual void getBrokerConsumerStatsAsync(BrokerConsumerStatsCallback callback); diff --git a/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.cc b/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.cc index 2c768b2e561cd..7894e64a874f6 100644 --- a/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.cc +++ b/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.cc @@ -28,7 +28,7 @@ void UnAckedMessageTrackerEnabled::timeoutHandler() { timeoutHandlerHelper(); ExecutorServicePtr executorService = client_->getIOExecutorProvider()->get(); timer_ = executorService->createDeadlineTimer(); - timer_->expires_from_now(boost::posix_time::milliseconds(timeoutMs_)); + timer_->expires_from_now(boost::posix_time::milliseconds(tickDurationInMs_)); timer_->async_wait([&](const boost::system::error_code& ec) { if (ec) { LOG_DEBUG("Ignoring timer cancelled event, code[" << ec << "]"); @@ -42,86 +42,112 @@ void UnAckedMessageTrackerEnabled::timeoutHandlerHelper() { std::lock_guard acquire(lock_); LOG_DEBUG("UnAckedMessageTrackerEnabled::timeoutHandlerHelper invoked for consumerPtr_ " << consumerReference_.getName().c_str()); - if (!oldSet_.empty()) { + + std::set headPartition = timePartitions.front(); + timePartitions.pop_front(); + + std::set msgIdsToRedeliver; + if (!headPartition.empty()) { LOG_INFO(consumerReference_.getName().c_str() - << ": " << oldSet_.size() << " Messages were not acked within " << timeoutMs_ << " time"); - oldSet_.clear(); - currentSet_.clear(); - consumerReference_.redeliverUnacknowledgedMessages(); + << ": " << headPartition.size() << " Messages were not acked within " + << timePartitions.size() * tickDurationInMs_ << " time"); + for (auto it = headPartition.begin(); it != headPartition.end(); it++) { + msgIdsToRedeliver.insert(*it); + messageIdPartitionMap.erase(*it); + } + } + headPartition.clear(); + timePartitions.push_back(headPartition); + + if (msgIdsToRedeliver.size() > 0) { + consumerReference_.redeliverUnacknowledgedMessages(msgIdsToRedeliver); } - oldSet_.swap(currentSet_); } UnAckedMessageTrackerEnabled::UnAckedMessageTrackerEnabled(long timeoutMs, const ClientImplPtr client, ConsumerImplBase& consumer) : consumerReference_(consumer) { + UnAckedMessageTrackerEnabled(timeoutMs, timeoutMs, client, consumer); +} + +UnAckedMessageTrackerEnabled::UnAckedMessageTrackerEnabled(long timeoutMs, long tickDurationInMs, + const ClientImplPtr client, + ConsumerImplBase& consumer) + : consumerReference_(consumer) { timeoutMs_ = timeoutMs; + tickDurationInMs_ = (timeoutMs >= tickDurationInMs) ? tickDurationInMs : timeoutMs; client_ = client; + + int blankPartitions = (int)std::ceil((double)timeoutMs_ / tickDurationInMs_); + for (int i = 0; i < blankPartitions + 1; i++) { + std::set msgIds; + timePartitions.push_back(msgIds); + } + timeoutHandler(); } bool UnAckedMessageTrackerEnabled::add(const MessageId& m) { std::lock_guard acquire(lock_); - oldSet_.erase(m); - return currentSet_.insert(m).second; + if (messageIdPartitionMap.count(m) == 0) { + bool insert = messageIdPartitionMap.insert(std::make_pair(m, timePartitions.back())).second; + return insert && timePartitions.back().insert(m).second; + } + return false; } bool UnAckedMessageTrackerEnabled::isEmpty() { std::lock_guard acquire(lock_); - return oldSet_.empty() && currentSet_.empty(); + return messageIdPartitionMap.empty(); } bool UnAckedMessageTrackerEnabled::remove(const MessageId& m) { std::lock_guard acquire(lock_); - return oldSet_.erase(m) || currentSet_.erase(m); + bool removed = false; + std::map>::iterator exist = messageIdPartitionMap.find(m); + if (exist != messageIdPartitionMap.end()) { + removed = exist->second.erase(m); + } + return removed; } long UnAckedMessageTrackerEnabled::size() { std::lock_guard acquire(lock_); - return oldSet_.size() + currentSet_.size(); + return messageIdPartitionMap.size(); } void UnAckedMessageTrackerEnabled::removeMessagesTill(const MessageId& msgId) { std::lock_guard acquire(lock_); - for (std::set::iterator it = oldSet_.begin(); it != oldSet_.end();) { - if (*it < msgId && it->partition() == msgId.partition()) { - oldSet_.erase(it++); - } else { - it++; - } - } - for (std::set::iterator it = currentSet_.begin(); it != currentSet_.end();) { - if (*it < msgId && it->partition() == msgId.partition()) { - currentSet_.erase(it++); - } else { - it++; + for (auto it = messageIdPartitionMap.begin(); it != messageIdPartitionMap.end(); it++) { + MessageId msgIdInMap = it->first; + if (msgIdInMap < msgId) { + std::map>::iterator exist = messageIdPartitionMap.find(msgId); + if (exist != messageIdPartitionMap.end()) { + exist->second.erase(msgId); + } } } } // this is only for MultiTopicsConsumerImpl, when un-subscribe a single topic, should remove all it's message. void UnAckedMessageTrackerEnabled::removeTopicMessage(const std::string& topic) { - for (std::set::iterator it = oldSet_.begin(); it != oldSet_.end();) { - const std::string& topicPartitionName = it->getTopicName(); - if (topicPartitionName.find(topic) != std::string::npos) { - oldSet_.erase(it++); - } else { - it++; - } - } - for (std::set::iterator it = currentSet_.begin(); it != currentSet_.end();) { - const std::string& topicPartitionName = it->getTopicName(); - if (topicPartitionName.find(topic) != std::string::npos) { - currentSet_.erase(it++); - } else { - it++; + std::lock_guard acquire(lock_); + for (auto it = messageIdPartitionMap.begin(); it != messageIdPartitionMap.end(); it++) { + MessageId msgIdInMap = it->first; + if (msgIdInMap.getTopicName().compare(topic) == 0) { + std::map>::iterator exist = messageIdPartitionMap.find(msgIdInMap); + if (exist != messageIdPartitionMap.end()) { + exist->second.erase(msgIdInMap); + } } } } void UnAckedMessageTrackerEnabled::clear() { - currentSet_.clear(); - oldSet_.clear(); + messageIdPartitionMap.clear(); + for (auto it = timePartitions.begin(); it != timePartitions.end(); it++) { + it->clear(); + } } UnAckedMessageTrackerEnabled::~UnAckedMessageTrackerEnabled() { diff --git a/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.h b/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.h index 921e74725d0ed..c2b4012adb184 100644 --- a/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.h +++ b/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.h @@ -28,6 +28,7 @@ class UnAckedMessageTrackerEnabled : public UnAckedMessageTrackerInterface { public: ~UnAckedMessageTrackerEnabled(); UnAckedMessageTrackerEnabled(long timeoutMs, const ClientImplPtr, ConsumerImplBase&); + UnAckedMessageTrackerEnabled(long timeoutMs, long tickDuration, const ClientImplPtr, ConsumerImplBase&); bool add(const MessageId& m); bool remove(const MessageId& m); void removeMessagesTill(const MessageId& msgId); @@ -40,13 +41,14 @@ class UnAckedMessageTrackerEnabled : public UnAckedMessageTrackerInterface { void timeoutHandlerHelper(); bool isEmpty(); long size(); - std::set currentSet_; - std::set oldSet_; + std::map> messageIdPartitionMap; + std::deque> timePartitions; std::mutex lock_; DeadlineTimerPtr timer_; ConsumerImplBase& consumerReference_; ClientImplPtr client_; long timeoutMs_; + long tickDurationInMs_; }; } // namespace pulsar From baf155f4a41de959145841e4cd922e89363e9292 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Tue, 3 Mar 2020 15:04:49 +0800 Subject: [PATCH 46/74] [ClientAPI]Fix hasMessageAvailable() (#6362) Fixes #6333 Previously, `hasMoreMessages` is test against: ``` return lastMessageIdInBroker.compareTo(lastDequeuedMessage) == 0 && incomingMessages.size() > 0; ``` However, the `incomingMessages` could be 0 when the consumer/reader has just started and hasn't received any messages yet. In this PR, the last entry is retrieved and decoded to get message metadata. for the batchIndex field population. --- .../pulsar/broker/service/ServerCnx.java | 86 ++++++++++++++-- .../pulsar/compaction/TwoPhaseCompactor.java | 6 +- .../pulsar/client/api/TopicReaderTest.java | 75 +++++++++++++- .../apache/pulsar/client/impl/ReaderTest.java | 4 +- .../pulsar/client/impl/ConsumerImpl.java | 98 ++++++++++++++----- .../client/impl/ZeroQueueConsumerImpl.java | 2 +- .../pulsar/common/api/raw/RawMessageImpl.java | 3 + 7 files changed, 233 insertions(+), 41 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index bef63fc7f5b0a..4497c977036fd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -46,7 +46,11 @@ import javax.naming.AuthenticationException; import javax.net.ssl.SSLSession; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.util.SafeRun; import org.apache.commons.lang3.StringUtils; @@ -59,6 +63,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException.ServerMetadataException; import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException; import org.apache.pulsar.broker.service.BrokerServiceException.TopicNotFoundException; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.web.RestException; @@ -1396,22 +1401,83 @@ protected void handleGetLastMessageId(CommandGetLastMessageId getLastMessageId) Topic topic = consumer.getSubscription().getTopic(); Position position = topic.getLastMessageId(); int partitionIndex = TopicName.getPartitionIndex(topic.getName()); - if (log.isDebugEnabled()) { - log.debug("[{}] [{}][{}] Get LastMessageId {} partitionIndex {}", remoteAddress, - topic.getName(), consumer.getSubscription().getName(), position, partitionIndex); - } - MessageIdData messageId = MessageIdData.newBuilder() - .setLedgerId(((PositionImpl)position).getLedgerId()) - .setEntryId(((PositionImpl)position).getEntryId()) - .setPartition(partitionIndex) - .build(); - ctx.writeAndFlush(Commands.newGetLastMessageIdResponse(requestId, messageId)); + getLargestBatchIndexWhenPossible( + topic, + (PositionImpl) position, + partitionIndex, + requestId, + consumer.getSubscription().getName()); + } else { ctx.writeAndFlush(Commands.newError(getLastMessageId.getRequestId(), ServerError.MetadataError, "Consumer not found")); } } + private void getLargestBatchIndexWhenPossible( + Topic topic, + PositionImpl position, + int partitionIndex, + long requestId, + String subscriptionName) { + + PersistentTopic persistentTopic = (PersistentTopic) topic; + ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + + // If it's not pointing to a valid entry, respond messageId of the current position. + if (position.getEntryId() == -1) { + MessageIdData messageId = MessageIdData.newBuilder() + .setLedgerId(position.getLedgerId()) + .setEntryId(position.getEntryId()) + .setPartition(partitionIndex).build(); + + ctx.writeAndFlush(Commands.newGetLastMessageIdResponse(requestId, messageId)); + } + + // For a valid position, we read the entry out and parse the batch size from its metadata. + CompletableFuture entryFuture = new CompletableFuture<>(); + ml.asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback() { + @Override + public void readEntryComplete(Entry entry, Object ctx) { + entryFuture.complete(entry); + } + + @Override + public void readEntryFailed(ManagedLedgerException exception, Object ctx) { + entryFuture.completeExceptionally(exception); + } + }, null); + + CompletableFuture batchSizeFuture = entryFuture.thenApply(entry -> { + MessageMetadata metadata = Commands.parseMessageMetadata(entry.getDataBuffer()); + int batchSize = metadata.getNumMessagesInBatch(); + entry.release(); + return batchSize; + }); + + batchSizeFuture.whenComplete((batchSize, e) -> { + if (e != null) { + ctx.writeAndFlush(Commands.newError( + requestId, ServerError.MetadataError, "Failed to get batch size for entry " + e.getMessage())); + } else { + int largestBatchIndex = batchSize > 1 ? batchSize - 1 : -1; + + if (log.isDebugEnabled()) { + log.debug("[{}] [{}][{}] Get LastMessageId {} partitionIndex {}", remoteAddress, + topic.getName(), subscriptionName, position, partitionIndex); + } + + MessageIdData messageId = MessageIdData.newBuilder() + .setLedgerId(position.getLedgerId()) + .setEntryId(position.getEntryId()) + .setPartition(partitionIndex) + .setBatchIndex(largestBatchIndex).build(); + + ctx.writeAndFlush(Commands.newGetLastMessageIdResponse(requestId, messageId)); + } + }); + } + @Override protected void handleGetTopicsOfNamespace(CommandGetTopicsOfNamespace commandGetTopicsOfNamespace) { final long requestId = commandGetTopicsOfNamespace.getRequestId(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java index 06afe9310433f..95f6f1ad7f5ad 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java @@ -43,6 +43,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.RawMessage; import org.apache.pulsar.client.api.RawReader; +import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.RawBatchConverter; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; @@ -95,7 +96,10 @@ private CompletableFuture phaseOne(RawReader reader) { } else { log.info("Commencing phase one of compaction for {}, reading to {}", reader.getTopic(), lastMessageId); - phaseOneLoop(reader, Optional.empty(), Optional.empty(), lastMessageId, latestForKey, + // Each entry is processed as a whole, discard the batchIndex part deliberately. + MessageIdImpl lastImpl = (MessageIdImpl) lastMessageId; + MessageIdImpl lastEntryMessageId = new MessageIdImpl(lastImpl.getLedgerId(), lastImpl.getEntryId(), lastImpl.getPartitionIndex()); + phaseOneLoop(reader, Optional.empty(), Optional.empty(), lastEntryMessageId, latestForKey, loopPromise); } }); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java index 7eda4465f7ff0..552f69db57609 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TopicReaderTest.java @@ -91,6 +91,17 @@ public static Object[][] variationsForResetOnLatestMsg() { }; } + @DataProvider + public static Object[][] variationsForHasMessageAvailable() { + return new Object[][] { + // batching / start-inclusive + {true, true}, + {true, false}, + {false, true}, + {false, false}, + }; + } + @Test public void testSimpleReader() throws Exception { Reader reader = pulsarClient.newReader().topic("persistent://my-property/my-ns/testSimpleReader") @@ -531,6 +542,68 @@ public void testMessageAvailableAfterRestart() throws Exception { } + @Test(dataProvider = "variationsForHasMessageAvailable") + public void testHasMessageAvailable(boolean enableBatch, boolean startInclusive) throws Exception { + final String topicName = "persistent://my-property/my-ns/HasMessageAvailable"; + final int numOfMessage = 100; + + ProducerBuilder producerBuilder = pulsarClient.newProducer() + .topic(topicName); + + if (enableBatch) { + producerBuilder + .enableBatching(true) + .batchingMaxMessages(10); + } else { + producerBuilder + .enableBatching(false); + } + + Producer producer = producerBuilder.create(); + + CountDownLatch latch = new CountDownLatch(numOfMessage); + + List allIds = Collections.synchronizedList(new ArrayList<>()); + + for (int i = 0; i < numOfMessage; i++) { + producer.sendAsync(String.format("msg num %d", i).getBytes()).whenComplete((mid, e) -> { + if (e != null) { + Assert.fail(); + } else { + allIds.add(mid); + } + latch.countDown(); + }); + } + + latch.await(); + + allIds.sort(null); // make sure the largest mid appears at last. + + for (MessageId id : allIds) { + Reader reader; + + if (startInclusive) { + reader = pulsarClient.newReader().topic(topicName) + .startMessageId(id).startMessageIdInclusive().create(); + } else { + reader = pulsarClient.newReader().topic(topicName) + .startMessageId(id).create(); + } + + if (startInclusive) { + assertTrue(reader.hasMessageAvailable()); + } else if (id != allIds.get(allIds.size() - 1)) { + assertTrue(reader.hasMessageAvailable()); + } else { + assertFalse(reader.hasMessageAvailable()); + } + reader.close(); + } + + producer.close(); + } + @Test public void testReaderNonDurableIsAbleToSeekRelativeTime() throws Exception { final int numOfMessage = 10; @@ -794,7 +867,7 @@ public void testReaderStartInMiddleOfBatch() throws Exception { .batchingMaxMessages(10) .create(); - CountDownLatch latch = new CountDownLatch(100); + CountDownLatch latch = new CountDownLatch(numOfMessage); List allIds = Collections.synchronizedList(new ArrayList<>()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java index 5f97a5888389f..b75cfcee0b9d2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java @@ -145,7 +145,9 @@ public void testReadMessageWithBatchingWithMessageInclusive() throws Exception { while (reader.hasMessageAvailable()) { Assert.assertTrue(keys.remove(reader.readNext().getKey())); } - Assert.assertTrue(keys.isEmpty()); + // start from latest with start message inclusive should only read the last message in batch + Assert.assertTrue(keys.size() == 9); + Assert.assertFalse(keys.contains("key9")); Assert.assertFalse(reader.hasMessageAvailable()); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 9431496688944..ce4ee1ebc8ea8 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -104,7 +104,7 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle @SuppressWarnings("unused") private volatile int availablePermits = 0; - protected volatile MessageId lastDequeuedMessage = MessageId.earliest; + protected volatile MessageId lastDequeuedMessageId = MessageId.earliest; private volatile MessageId lastMessageIdInBroker = MessageId.earliest; private long subscribeTimeout; @@ -188,7 +188,6 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat this.consumerId = client.newConsumerId(); this.subscriptionMode = conf.getSubscriptionMode(); this.startMessageId = startMessageId != null ? new BatchMessageIdImpl((MessageIdImpl) startMessageId) : null; - this.lastDequeuedMessage = startMessageId == null ? MessageId.earliest : startMessageId; this.initialStartMessageId = this.startMessageId; this.startMessageRollbackDurationInSec = startMessageRollbackDurationInSec; AVAILABLE_PERMITS_UPDATER.set(this, 0); @@ -677,7 +676,7 @@ private BatchMessageIdImpl clearReceiverQueue() { if (duringSeek.compareAndSet(true, false)) { return seekMessageId; } else if (subscriptionMode == SubscriptionMode.Durable) { - return null; + return startMessageId; } if (!currentMessageQueue.isEmpty()) { @@ -695,10 +694,10 @@ private BatchMessageIdImpl clearReceiverQueue() { } return previousMessage; - } else if (!lastDequeuedMessage.equals(MessageId.earliest)) { + } else if (!lastDequeuedMessageId.equals(MessageId.earliest)) { // If the queue was empty we need to restart from the message just after the last one that has been dequeued // in the past - return new BatchMessageIdImpl((MessageIdImpl) lastDequeuedMessage); + return new BatchMessageIdImpl((MessageIdImpl) lastDequeuedMessageId); } else { // No message was received or dequeued by this consumer. Next message would still be the startMessageId return startMessageId; @@ -1118,7 +1117,7 @@ private boolean isSameEntry(MessageIdData messageId) { protected synchronized void messageProcessed(Message msg) { ClientCnx currentCnx = cnx(); ClientCnx msgCnx = ((MessageImpl) msg).getCnx(); - lastDequeuedMessage = msg.getMessageId(); + lastDequeuedMessageId = msg.getMessageId(); if (msgCnx != currentCnx) { // The processed message did belong to the old queue that was cleared after reconnection. @@ -1493,6 +1492,7 @@ public CompletableFuture seekAsync(long timestamp) { seekMessageId = new BatchMessageIdImpl((MessageIdImpl) MessageId.earliest); duringSeek.set(true); + lastDequeuedMessageId = MessageId.earliest; incomingMessages.clear(); INCOMING_MESSAGES_SIZE_UPDATER.set(this, 0); @@ -1539,6 +1539,7 @@ public CompletableFuture seekAsync(MessageId messageId) { seekMessageId = new BatchMessageIdImpl((MessageIdImpl) messageId); duringSeek.set(true); + lastDequeuedMessageId = MessageId.earliest; incomingMessages.clear(); INCOMING_MESSAGES_SIZE_UPDATER.set(this, 0); @@ -1555,17 +1556,7 @@ public CompletableFuture seekAsync(MessageId messageId) { } public boolean hasMessageAvailable() throws PulsarClientException { - // we need to seek to the last position then the last message can be received when the resetIncludeHead - // specified. - if (lastDequeuedMessage == MessageId.latest && resetIncludeHead) { - lastDequeuedMessage = getLastMessageId(); - seek(lastDequeuedMessage); - } try { - if (hasMoreMessages(lastMessageIdInBroker, lastDequeuedMessage)) { - return true; - } - return hasMessageAvailableAsync().get(); } catch (Exception e) { throw PulsarClientException.unwrap(e); @@ -1575,12 +1566,56 @@ public boolean hasMessageAvailable() throws PulsarClientException { public CompletableFuture hasMessageAvailableAsync() { final CompletableFuture booleanFuture = new CompletableFuture<>(); - if (hasMoreMessages(lastMessageIdInBroker, lastDequeuedMessage)) { - booleanFuture.complete(true); + // we haven't read yet. use startMessageId for comparison + if (lastDequeuedMessageId == MessageId.earliest) { + // if we are starting from latest, we should seek to the actual last message first. + // allow the last one to be read when read head inclusively. + if (startMessageId.getLedgerId() == Long.MAX_VALUE && + startMessageId.getEntryId() == Long.MAX_VALUE && + startMessageId.partitionIndex == -1) { + + getLastMessageIdAsync() + .thenCompose(this::seekAsync) + .whenComplete((ignore, e) -> { + if (e != null) { + log.error("[{}][{}] Failed getLastMessageId command", topic, subscription); + booleanFuture.completeExceptionally(e.getCause()); + } else { + booleanFuture.complete(resetIncludeHead); + } + }); + + return booleanFuture; + } + + if (hasMoreMessages(lastMessageIdInBroker, startMessageId, resetIncludeHead)) { + booleanFuture.complete(true); + return booleanFuture; + } + + getLastMessageIdAsync().thenAccept(messageId -> { + lastMessageIdInBroker = messageId; + if (hasMoreMessages(lastMessageIdInBroker, startMessageId, resetIncludeHead)) { + booleanFuture.complete(true); + } else { + booleanFuture.complete(false); + } + }).exceptionally(e -> { + log.error("[{}][{}] Failed getLastMessageId command", topic, subscription); + booleanFuture.completeExceptionally(e.getCause()); + return null; + }); + } else { + // read before, use lastDequeueMessage for comparison + if (hasMoreMessages(lastMessageIdInBroker, lastDequeuedMessageId, false)) { + booleanFuture.complete(true); + return booleanFuture; + } + getLastMessageIdAsync().thenAccept(messageId -> { lastMessageIdInBroker = messageId; - if (hasMoreMessages(lastMessageIdInBroker, lastDequeuedMessage)) { + if (hasMoreMessages(lastMessageIdInBroker, lastDequeuedMessageId, false)) { booleanFuture.complete(true); } else { booleanFuture.complete(false); @@ -1591,18 +1626,22 @@ public CompletableFuture hasMessageAvailableAsync() { return null; }); } + return booleanFuture; } - private boolean hasMoreMessages(MessageId lastMessageIdInBroker, MessageId lastDequeuedMessage) { - if (lastMessageIdInBroker.compareTo(lastDequeuedMessage) > 0 && + private boolean hasMoreMessages(MessageId lastMessageIdInBroker, MessageId messageId, boolean inclusive) { + if (inclusive && lastMessageIdInBroker.compareTo(messageId) >= 0 && ((MessageIdImpl)lastMessageIdInBroker).getEntryId() != -1) { return true; - } else { - // Make sure batching message can be read completely. - return lastMessageIdInBroker.compareTo(lastDequeuedMessage) == 0 - && incomingMessages.size() > 0; } + + if (!inclusive && lastMessageIdInBroker.compareTo(messageId) > 0 && + ((MessageIdImpl)lastMessageIdInBroker).getEntryId() != -1) { + return true; + } + + return false; } @Override @@ -1647,8 +1686,13 @@ private void internalGetLastMessageIdAsync(final Backoff backoff, cnx.sendGetLastMessageId(getLastIdCmd, requestId).thenAccept((result) -> { log.info("[{}][{}] Successfully getLastMessageId {}:{}", topic, subscription, result.getLedgerId(), result.getEntryId()); - future.complete(new MessageIdImpl(result.getLedgerId(), - result.getEntryId(), result.getPartition())); + if (result.getBatchIndex() < 0) { + future.complete(new MessageIdImpl(result.getLedgerId(), + result.getEntryId(), result.getPartition())); + } else { + future.complete(new BatchMessageIdImpl(result.getLedgerId(), + result.getEntryId(), result.getPartition(), result.getBatchIndex())); + } }).exceptionally(e -> { log.error("[{}][{}] Failed getLastMessageId command", topic, subscription); future.completeExceptionally( diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java index 94c8dd3c56ea4..a0de0704c2d9a 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java @@ -98,7 +98,7 @@ private Message fetchSingleMessageFromBroker() throws PulsarClientException { } do { message = incomingMessages.take(); - lastDequeuedMessage = message.getMessageId(); + lastDequeuedMessageId = message.getMessageId(); ClientCnx msgCnx = ((MessageImpl) message).getCnx(); // synchronized need to prevent race between connectionOpened and the check "msgCnx == cnx()" synchronized (this) { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessageImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessageImpl.java index 335bf967bf099..371f47f4054fd 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessageImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/RawMessageImpl.java @@ -162,4 +162,7 @@ public boolean hasBase64EncodedKey() { return msgMetadata.get().getPartitionKeyB64Encoded(); } + public int getBatchSize() { + return msgMetadata.get().getNumMessagesInBatch(); + } } From 459ec6e8386f6d67e966321dcd0558d15c96205b Mon Sep 17 00:00:00 2001 From: Rolf Arne Corneliussen Date: Tue, 3 Mar 2020 08:21:58 +0100 Subject: [PATCH 47/74] Use System.nanoTime() instead of System.currentTimeMillis() (#6454) Fixes #6453 ### Motivation `ConsumerBase` and `ProducerImpl` use `System.currentTimeMillis()` to measure the elapsed time in the 'operations' inner classes (`ConsumerBase$OpBatchReceive` and `ProducerImpl$OpSendMsg`). An instance variable `createdAt` is initialized with `System.currentTimeMills()`, but it is not used for reading wall clock time, the variable is only used for computing elapsed time (e.g. timeout for a batch). When the variable is used to compute elapsed time, it would more sense to use `System.nanoTime()`. ### Modifications The instance variable `createdAt` in `ConsumerBase$OpBatchReceive` and `ProducerImpl$OpSendMsg` is initialized with `System.nanoTime()`. Usage of the variable is updated to reflect that the variable holds nano time; computations of elapsed time takes the difference between the current system nano time and the `createdAt` variable. The `createdAt` field is package protected, and is currently only used in the declaring class and outer class, limiting the chances for unwanted side effects. --- .../apache/pulsar/client/impl/ConsumerBase.java | 9 +++++---- .../apache/pulsar/client/impl/ProducerImpl.java | 15 ++++++++------- 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index 0f5219b0a9b95..de9ec85449df7 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -287,7 +287,7 @@ public CompletableFuture acknowledgeCumulativeAsync(MessageId messageId, T public void negativeAcknowledge(Message message) { negativeAcknowledge(message.getMessageId()); } - + protected CompletableFuture doAcknowledgeWithTxn(MessageId messageId, AckType ackType, Map properties, TransactionImpl txn) { @@ -507,7 +507,7 @@ protected static final class OpBatchReceive { private OpBatchReceive(CompletableFuture> future) { this.future = future; - this.createdAt = System.currentTimeMillis(); + this.createdAt = System.nanoTime(); } static OpBatchReceive of(CompletableFuture> future) { @@ -566,8 +566,9 @@ public void run(Timeout timeout) throws Exception { while (firstOpBatchReceive != null) { // If there is at least one batch receive, calculate the diff between the batch receive timeout - // and the current time. - long diff = (firstOpBatchReceive.createdAt + batchReceivePolicy.getTimeoutMs()) - System.currentTimeMillis(); + // and the elapsed time since the operation was created. + long diff = batchReceivePolicy.getTimeoutMs() + - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - firstOpBatchReceive.createdAt); if (diff <= 0) { // The diff is less than or equal to zero, meaning that the batch receive has been timed out. // complete the OpBatchReceive and continue to check the next OpBatchReceive in pendingBatchReceives. diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index 776300deaec0a..eb059096e254e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -987,7 +987,7 @@ static OpSendMsg create(MessageImpl msg, ByteBufPair cmd, long sequenceId, Se op.cmd = cmd; op.callback = callback; op.sequenceId = sequenceId; - op.createdAt = System.currentTimeMillis(); + op.createdAt = System.nanoTime(); return op; } @@ -997,7 +997,7 @@ static OpSendMsg create(List> msgs, ByteBufPair cmd, long sequenc op.cmd = cmd; op.callback = callback; op.sequenceId = sequenceId; - op.createdAt = System.currentTimeMillis(); + op.createdAt = System.nanoTime(); return op; } @@ -1009,7 +1009,7 @@ static OpSendMsg create(List> msgs, ByteBufPair cmd, long lowestS op.callback = callback; op.sequenceId = lowestSequenceId; op.highestSequenceId = highestSequenceId; - op.createdAt = System.currentTimeMillis(); + op.createdAt = System.nanoTime(); return op; } @@ -1307,9 +1307,10 @@ public void run(Timeout timeout) throws Exception { // If there are no pending messages, reset the timeout to the configured value. timeToWaitMs = conf.getSendTimeoutMs(); } else { - // If there is at least one message, calculate the diff between the message timeout and the current - // time. - long diff = (firstMsg.createdAt + conf.getSendTimeoutMs()) - System.currentTimeMillis(); + // If there is at least one message, calculate the diff between the message timeout and the elapsed + // time since first message was created. + long diff = conf.getSendTimeoutMs() + - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - firstMsg.createdAt); if (diff <= 0) { // The diff is less than or equal to zero, meaning that the message has been timed out. // Set the callback to timeout on every message, then clear the pending queue. @@ -1560,7 +1561,7 @@ private void recoverProcessOpSendMsgFrom(ClientCnx cnx, MessageImpl from) { public long getDelayInMillis() { OpSendMsg firstMsg = pendingMessages.peek(); if (firstMsg != null) { - return System.currentTimeMillis() - firstMsg.createdAt; + return TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - firstMsg.createdAt); } return 0L; } From 6ff87eed0df2eea424f95363f086589c13c502d8 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 3 Mar 2020 13:35:27 -0800 Subject: [PATCH 48/74] Fixed the max backoff configuration for lookups (#6444) * Fixed the max backoff configuration for lookups * Fixed test expectation * More test fixes --- .../apache/pulsar/client/impl/BinaryProtoLookupService.java | 2 +- .../java/org/apache/pulsar/client/impl/PulsarClientImpl.java | 2 +- .../pulsar/proxy/server/ProxyConnectionThrottlingTest.java | 2 +- .../apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 3419a6397fbd9..102f394091d8e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -213,7 +213,7 @@ public CompletableFuture> getTopicsUnderNamespace(NamespaceName nam Backoff backoff = new BackoffBuilder() .setInitialTime(100, TimeUnit.MILLISECONDS) .setMandatoryStop(opTimeoutMs.get() * 2, TimeUnit.MILLISECONDS) - .setMax(0, TimeUnit.MILLISECONDS) + .setMax(1, TimeUnit.MINUTES) .create(); getTopicsUnderNamespace(serviceNameResolver.resolveHost(), namespace, backoff, opTimeoutMs, topicsFuture, mode); return topicsFuture; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index db66c90ca7150..2fe8201aeb7d5 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -663,7 +663,7 @@ public CompletableFuture getPartitionedTopicMetadata(S Backoff backoff = new BackoffBuilder() .setInitialTime(100, TimeUnit.MILLISECONDS) .setMandatoryStop(opTimeoutMs.get() * 2, TimeUnit.MILLISECONDS) - .setMax(0, TimeUnit.MILLISECONDS) + .setMax(1, TimeUnit.MINUTES) .create(); getPartitionedTopicMetadata(topicName, backoff, opTimeoutMs, metadataFuture); } catch (IllegalArgumentException e) { diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java index 3e1084f4efb87..c30ae4daed456 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java @@ -100,7 +100,7 @@ public void testInboundConnection() throws Exception { // OK } // should add retry count since retry every 100ms and operation timeout is set to 1000ms - Assert.assertEquals(ProxyService.rejectedConnections.get(), 11.0d); + Assert.assertEquals(ProxyService.rejectedConnections.get(), 5.0d); } private static final Logger LOG = LoggerFactory.getLogger(ProxyConnectionThrottlingTest.class); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java index 4d38c5e5958be..8949f8df34995 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java @@ -96,7 +96,7 @@ public void testLookup() throws Exception { } catch (Exception ex) { // Ignore } - Assert.assertEquals(LookupProxyHandler.rejectedPartitionsMetadataRequests.get(), 11.0d); + Assert.assertEquals(LookupProxyHandler.rejectedPartitionsMetadataRequests.get(), 5.0d); proxyService.getLookupRequestSemaphore().release(); try { @Cleanup @@ -106,6 +106,6 @@ public void testLookup() throws Exception { Assert.fail("Should not have failed since can acquire LookupRequestSemaphore"); } - Assert.assertEquals(LookupProxyHandler.rejectedPartitionsMetadataRequests.get(), 11.0d); + Assert.assertEquals(LookupProxyHandler.rejectedPartitionsMetadataRequests.get(), 5.0d); } } From 4284e393c8ec9502e35d73eafaa1c73efc9fa986 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 4 Mar 2020 09:56:54 -0800 Subject: [PATCH 49/74] upgrade scala-maven-plugin to 4.1.0 (#6469) ### Motivation The Pulsar examples include some third-party libraries with security vulnerabilities. - log4j-core-2.8.1 https://www.cvedetails.com/cve/CVE-2017-5645 ### Modifications - Upgraded the version of scala-maven-plugin from 4.0.1 to 4.1.0. log4j-core-2.8.1 were installed because scala-maven-plugin depends on it. --- examples/flink/pom.xml | 2 +- examples/spark/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/flink/pom.xml b/examples/flink/pom.xml index 186c975c137f6..9615aae04a727 100644 --- a/examples/flink/pom.xml +++ b/examples/flink/pom.xml @@ -133,7 +133,7 @@ net.alchim31.maven scala-maven-plugin - 4.0.1 + 4.1.0 diff --git a/examples/spark/pom.xml b/examples/spark/pom.xml index 4b3446d02be61..c4c21b9a189c2 100644 --- a/examples/spark/pom.xml +++ b/examples/spark/pom.xml @@ -116,7 +116,7 @@ net.alchim31.maven scala-maven-plugin - 4.0.1 + 4.1.0 From 65cc30317e46e05ac88866944e2a7e9a68c8afbb Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 4 Mar 2020 17:20:36 -0800 Subject: [PATCH 50/74] [pulsar-proxy] fix logging for published messages (#6474) ### Motivation Proxy-logging fetches incorrect producerId for `Send` command because of that logging always gets producerId as 0 and it fetches invalid topic name for the logging. ### Modification Fixed topic logging by fetching correct producerId for `Send` command. --- .../java/org/apache/pulsar/proxy/server/ParserProxyHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ParserProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ParserProxyHandler.java index cd04b8e62012e..fc455f7778d9f 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ParserProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ParserProxyHandler.java @@ -116,7 +116,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) { logging(ctx.channel() , cmd.getType() , "", null); break; } - topicName = TopicName.get(ParserProxyHandler.producerHashMap.get(String.valueOf(cmd.getProducer().getProducerId()) + "," + String.valueOf(ctx.channel().id()))); + topicName = TopicName.get(ParserProxyHandler.producerHashMap.get(String.valueOf(cmd.getSend().getProducerId()) + "," + String.valueOf(ctx.channel().id()))); MessageParser.parseMessage(topicName, -1L, -1L,buffer,(message) -> { messages.add(message); From c3292a611c9cbf2b17c96c5317d8f20247eb1f41 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 4 Mar 2020 17:21:53 -0800 Subject: [PATCH 51/74] [Issue 6394] Add configuration to disable auto creation of subscriptions (#6456) ### Motivation Fixes #6394 ### Modifications - provide a flag `allowAutoSubscriptionCreation` in `ServiceConfiguration`, defaults to `true` - when `allowAutoSubscriptionCreation` is disabled and the specified subscription (`Durable`) on the topic does not exist when trying to subscribe via a consumer, the server should reject the request directly by `handleSubscribe` in `ServerCnx` - create the subscription on the coordination topic if it does not exist when init `WorkerService` --- conf/broker.conf | 3 ++ conf/standalone.conf | 3 ++ .../pulsar/broker/ServiceConfiguration.java | 5 +++ .../service/BrokerServiceException.java | 6 +++ .../pulsar/broker/service/ServerCnx.java | 13 +++++- .../BrokerServiceAutoTopicCreationTest.java | 43 +++++++++++++++++++ .../functions/worker/WorkerService.java | 5 +++ site2/docs/reference-configuration.md | 1 + 8 files changed, 77 insertions(+), 2 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index bfb7f7ceee12d..59898800773c9 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -91,6 +91,9 @@ allowAutoTopicCreation=true # The type of topic that is allowed to be automatically created.(partitioned/non-partitioned) allowAutoTopicCreationType=non-partitioned +# Enable subscription auto creation if new consumer connected (disable auto creation with value false) +allowAutoSubscriptionCreation=true + # The number of partitioned topics that is allowed to be automatically created if allowAutoTopicCreationType is partitioned. defaultNumPartitions=1 diff --git a/conf/standalone.conf b/conf/standalone.conf index 7dff0c3c913ab..f351d8d2bff5f 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -632,6 +632,9 @@ allowAutoTopicCreation=true # The type of topic that is allowed to be automatically created.(partitioned/non-partitioned) allowAutoTopicCreationType=non-partitioned +# Enable subscription auto creation if new consumer connected (disable auto creation with value false) +allowAutoSubscriptionCreation=true + # The number of partitioned topics that is allowed to be automatically created if allowAutoTopicCreationType is partitioned. defaultNumPartitions=1 diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 86da1027f34d6..0d554f0c5dc74 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -998,6 +998,11 @@ public class ServiceConfiguration implements PulsarConfiguration { doc = "The type of topic that is allowed to be automatically created.(partitioned/non-partitioned)" ) private String allowAutoTopicCreationType = "non-partitioned"; + @FieldContext( + category = CATEGORY_STORAGE_ML, + doc = "Allow automated creation of subscriptions if set to true (default value)." + ) + private boolean allowAutoSubscriptionCreation = true; @FieldContext( category = CATEGORY_STORAGE_ML, doc = "The number of partitioned topics that is allowed to be automatically created" diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java index e794718c5fb1b..b4bfed518ad3c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerServiceException.java @@ -123,6 +123,12 @@ public TopicNotFoundException(String msg) { } } + public static class SubscriptionNotFoundException extends BrokerServiceException { + public SubscriptionNotFoundException(String msg) { + super(msg); + } + } + public static class SubscriptionBusyException extends BrokerServiceException { public SubscriptionBusyException(String msg) { super(msg); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index 4497c977036fd..9294330c87f42 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -62,6 +62,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; import org.apache.pulsar.broker.service.BrokerServiceException.ServerMetadataException; import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException; +import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionNotFoundException; import org.apache.pulsar.broker.service.BrokerServiceException.TopicNotFoundException; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; @@ -725,7 +726,6 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { subscribe.getStartMessageId().getLedgerId(), subscribe.getStartMessageId().getEntryId(), subscribe.getStartMessageId().getPartition(), subscribe.getStartMessageId().getBatchIndex()) : null; - final String subscription = subscribe.getSubscription(); final int priorityLevel = subscribe.hasPriorityLevel() ? subscribe.getPriorityLevel() : 0; final boolean readCompacted = subscribe.getReadCompacted(); final Map metadata = CommandUtils.metadataFromCommand(subscribe); @@ -751,7 +751,7 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { if (service.isAuthorizationEnabled()) { authorizationFuture = service.getAuthorizationService().canConsumeAsync(topicName, originalPrincipal != null ? originalPrincipal : authRole, authenticationData, - subscription); + subscriptionName); } else { authorizationFuture = CompletableFuture.completedFuture(true); } @@ -814,6 +814,15 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { Topic topic = optTopic.get(); + boolean rejectSubscriptionIfDoesNotExist = isDurable + && !service.pulsar().getConfig().isAllowAutoSubscriptionCreation() + && !topic.getSubscriptions().containsKey(subscriptionName); + + if (rejectSubscriptionIfDoesNotExist) { + return FutureUtil + .failedFuture(new SubscriptionNotFoundException("Subscription does not exist")); + } + if (schema != null) { return topic.addSchemaIfIdleOrCheckCompatible(schema) .thenCompose(v -> topic.subscribe(ServerCnx.this, subscriptionName, consumerId, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java index 929f9c24133fc..d595a64648cd6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceAutoTopicCreationTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClientException; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; @@ -104,6 +105,48 @@ public void testAutoTopicCreationDisableIfNonPartitionedTopicAlreadyExist() thro assertTrue(admin.namespaces().getTopics("prop/ns-abc").contains(topicName)); } + @Test + public void testAutoSubscriptionCreationDisable() throws Exception{ + pulsar.getConfiguration().setAllowAutoSubscriptionCreation(false); + + final String topicName = "persistent://prop/ns-abc/test-subtopic"; + final String subscriptionName = "test-subtopic-sub"; + + admin.topics().createNonPartitionedTopic(topicName); + + try { + pulsarClient.newConsumer().topic(topicName).subscriptionName(subscriptionName).subscribe(); + fail("Subscribe operation should have failed"); + } catch (Exception e) { + assertTrue(e instanceof PulsarClientException); + } + assertFalse(admin.topics().getSubscriptions(topicName).contains(subscriptionName)); + + // Reset to default + pulsar.getConfiguration().setAllowAutoSubscriptionCreation(true); + } + + @Test + public void testSubscriptionCreationWithAutoCreationDisable() throws Exception{ + pulsar.getConfiguration().setAllowAutoSubscriptionCreation(false); + + final String topicName = "persistent://prop/ns-abc/test-subtopic"; + final String subscriptionName = "test-subtopic-sub"; + + admin.topics().createNonPartitionedTopic(topicName); + assertFalse(admin.topics().getSubscriptions(topicName).contains(subscriptionName)); + + // Create the subscription by PulsarAdmin + admin.topics().createSubscription(topicName, subscriptionName, MessageId.earliest); + assertTrue(admin.topics().getSubscriptions(topicName).contains(subscriptionName)); + + // Subscribe operation should be successful + pulsarClient.newConsumer().topic(topicName).subscriptionName(subscriptionName).subscribe(); + + // Reset to default + pulsar.getConfiguration().setAllowAutoSubscriptionCreation(true); + } + /** * CheckAllowAutoCreation's default value is false. * So using getPartitionedTopicMetadata() directly will not produce partitioned topic diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerService.java index fdfb7dec9b626..b23c707cc5c3a 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerService.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerService.java @@ -34,6 +34,7 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.authorization.AuthorizationService; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -162,6 +163,10 @@ public void start(URI dlogUri, this.connectorsManager = new ConnectorsManager(workerConfig); //create membership manager + String coordinationTopic = workerConfig.getClusterCoordinationTopic(); + if (!brokerAdmin.topics().getSubscriptions(coordinationTopic).contains(MembershipManager.COORDINATION_TOPIC_SUBSCRIPTION)) { + brokerAdmin.topics().createSubscription(coordinationTopic, MembershipManager.COORDINATION_TOPIC_SUBSCRIPTION, MessageId.earliest); + } this.membershipManager = new MembershipManager(this, this.client, this.brokerAdmin); // create function runtime manager diff --git a/site2/docs/reference-configuration.md b/site2/docs/reference-configuration.md index cdfe0d419a70d..edc3f996451b3 100644 --- a/site2/docs/reference-configuration.md +++ b/site2/docs/reference-configuration.md @@ -131,6 +131,7 @@ Pulsar brokers are responsible for handling incoming messages from producers, di |backlogQuotaDefaultLimitGB| The default per-topic backlog quota limit | -1 | |allowAutoTopicCreation| Enable topic auto creation if a new producer or consumer connected |true| |allowAutoTopicCreationType| The topic type (partitioned or non-partitioned) that is allowed to be automatically created. |Partitioned| +|allowAutoSubscriptionCreation| Enable subscription auto creation if a new consumer connected |true| |defaultNumPartitions| The number of partitioned topics that is allowed to be automatically created if `allowAutoTopicCreationType` is partitioned |1| |brokerDeleteInactiveTopicsEnabled| Enable the deletion of inactive topics |true| |brokerDeleteInactiveTopicsFrequencySeconds| How often to check for inactive topics |60| From 17f71d3f73fe1bcb6723c8b0718d9a6e05574146 Mon Sep 17 00:00:00 2001 From: Sijie Guo Date: Wed, 4 Mar 2020 17:34:41 -0800 Subject: [PATCH 52/74] Make tests more stable by using JSONAssert equals (#6435) Similar to the change you already merged for AvroSchemaTest.java(#6247): `jsonSchema.getSchemaInfo().getSchema()` in `pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/JSONSchemaTest.java` returns a JSON object. `schemaJson` compares with hard-coded JSON String. However, the order of entries in `schemaJson` is not guaranteed. Similarly, test `testKeyValueSchemaInfoToString` in `pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/KeyValueSchemaInfoTest.java` returns a JSON object. `havePrimitiveType` compares with hard-coded JSON String, and the order of entries in `havePrimitiveType` is not guaranteed. This PR proposes to use JSONAssert and modify the corresponding JSON test assertions so that the test is more stable. ### Motivation Using JSONAssert and modifying the corresponding JSON test assertions so that the test is more stable. ### Modifications Adding `assertJSONEqual` method and replacing `assertEquals` with it in tests `testAllowNullSchema`, `testNotAllowNullSchema` and `testKeyValueSchemaInfoToString`. --- .../pulsar/client/impl/schema/JSONSchemaTest.java | 13 +++++++++---- .../client/impl/schema/KeyValueSchemaInfoTest.java | 7 ++++--- 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/JSONSchemaTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/JSONSchemaTest.java index c303ce9293329..d17bbf0b8a0f0 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/JSONSchemaTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/JSONSchemaTest.java @@ -33,8 +33,10 @@ import org.apache.pulsar.client.impl.schema.SchemaTestUtils.NestedBar; import org.apache.pulsar.client.impl.schema.SchemaTestUtils.NestedBarList; import org.apache.pulsar.common.schema.SchemaType; +import org.skyscreamer.jsonassert.JSONAssert; import org.testng.Assert; import org.testng.annotations.Test; +import org.json.JSONException; import static org.apache.pulsar.client.impl.schema.SchemaTestUtils.FOO_FIELDS; import static org.apache.pulsar.client.impl.schema.SchemaTestUtils.SCHEMA_JSON_NOT_ALLOW_NULL; @@ -44,13 +46,16 @@ @Slf4j public class JSONSchemaTest { + public static void assertJSONEqual(String s1, String s2) throws JSONException{ + JSONAssert.assertEquals(s1, s2, false); + } @Test - public void testNotAllowNullSchema() { + public void testNotAllowNullSchema() throws JSONException { JSONSchema jsonSchema = JSONSchema.of(SchemaDefinition.builder().withPojo(Foo.class).withAlwaysAllowNull(false).build()); Assert.assertEquals(jsonSchema.getSchemaInfo().getType(), SchemaType.JSON); Schema.Parser parser = new Schema.Parser(); String schemaJson = new String(jsonSchema.getSchemaInfo().getSchema()); - Assert.assertEquals(schemaJson, SCHEMA_JSON_NOT_ALLOW_NULL); + assertJSONEqual(schemaJson, SCHEMA_JSON_NOT_ALLOW_NULL); Schema schema = parser.parse(schemaJson); for (String fieldName : FOO_FIELDS) { @@ -67,13 +72,13 @@ public void testNotAllowNullSchema() { } @Test - public void testAllowNullSchema() { + public void testAllowNullSchema() throws JSONException { JSONSchema jsonSchema = JSONSchema.of(SchemaDefinition.builder().withPojo(Foo.class).build()); Assert.assertEquals(jsonSchema.getSchemaInfo().getType(), SchemaType.JSON); Schema.Parser parser = new Schema.Parser(); parser.setValidateDefaults(false); String schemaJson = new String(jsonSchema.getSchemaInfo().getSchema()); - Assert.assertEquals(schemaJson, SCHEMA_JSON_ALLOW_NULL); + assertJSONEqual(schemaJson, SCHEMA_JSON_ALLOW_NULL); Schema schema = parser.parse(schemaJson); for (String fieldName : FOO_FIELDS) { diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/KeyValueSchemaInfoTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/KeyValueSchemaInfoTest.java index 99a1724a83ab2..994f01386c4bf 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/KeyValueSchemaInfoTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/KeyValueSchemaInfoTest.java @@ -38,6 +38,7 @@ import org.apache.pulsar.common.schema.KeyValueEncodingType; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; +import org.json.JSONException; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -209,18 +210,18 @@ public void testKeyValueSchemaInfoBackwardCompatibility() { } @Test - public void testKeyValueSchemaInfoToString() { + public void testKeyValueSchemaInfoToString() throws JSONException { String havePrimitiveType = DefaultImplementation .convertKeyValueSchemaInfoDataToString(KeyValueSchemaInfo .decodeKeyValueSchemaInfo(Schema.KeyValue(Schema.AVRO(Foo.class), Schema.STRING) .getSchemaInfo())); - assertEquals(havePrimitiveType, KEY_VALUE_SCHEMA_INFO_INCLUDE_PRIMITIVE); + JSONSchemaTest.assertJSONEqual(havePrimitiveType, KEY_VALUE_SCHEMA_INFO_INCLUDE_PRIMITIVE); String notHavePrimitiveType = DefaultImplementation .convertKeyValueSchemaInfoDataToString(KeyValueSchemaInfo .decodeKeyValueSchemaInfo(Schema.KeyValue(Schema.AVRO(Foo.class), Schema.AVRO(Foo.class)).getSchemaInfo())); - assertEquals(notHavePrimitiveType, KEY_VALUE_SCHEMA_INFO_NOT_INCLUDE_PRIMITIVE); + JSONSchemaTest.assertJSONEqual(notHavePrimitiveType, KEY_VALUE_SCHEMA_INFO_NOT_INCLUDE_PRIMITIVE); } } From 6604f540995d8bb674bfe2eb50cdb2802a0b7cec Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 5 Mar 2020 15:13:38 -0800 Subject: [PATCH 53/74] Avoid calling ConsumerImpl::redeliverMessages() when message list is empty (#6480) --- pulsar-client-cpp/lib/NegativeAcksTracker.cc | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pulsar-client-cpp/lib/NegativeAcksTracker.cc b/pulsar-client-cpp/lib/NegativeAcksTracker.cc index 492e379459393..202c5e48825aa 100644 --- a/pulsar-client-cpp/lib/NegativeAcksTracker.cc +++ b/pulsar-client-cpp/lib/NegativeAcksTracker.cc @@ -74,7 +74,9 @@ void NegativeAcksTracker::handleTimer(const boost::system::error_code &ec) { } } - consumer_.redeliverMessages(messagesToRedeliver); + if (!messagesToRedeliver.empty()) { + consumer_.redeliverMessages(messagesToRedeliver); + } scheduleTimer(); } From ad5415ab90fac123d00ed1ec55b696914645edb1 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Thu, 5 Mar 2020 18:55:40 -0800 Subject: [PATCH 54/74] [pulsar-client] fix deadlock on send failure (#6488) --- .../api/SimpleProducerConsumerTest.java | 33 +++++++++++++++++++ .../pulsar/client/impl/ProducerImpl.java | 20 ++++++----- 2 files changed, 45 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java index e907197de7b1b..612b610754361 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java @@ -77,6 +77,7 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.impl.ClientCnx; import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.client.impl.MessageCrypto; import org.apache.pulsar.client.impl.MessageIdImpl; @@ -3283,4 +3284,36 @@ public void testConsumerStartMessageIdAtExpectedPos(boolean batching, boolean st consumer.close(); producer.close(); } + + /** + * It verifies that message failure successfully releases semaphore and client successfully receives + * InvalidMessageException. + * + * @throws Exception + */ + @Test + public void testReleaseSemaphoreOnFailMessages() throws Exception { + log.info("-- Starting {} test --", methodName); + + int maxPendingMessages = 10; + ProducerBuilder producerBuilder = pulsarClient.newProducer().enableBatching(false) + .blockIfQueueFull(true).maxPendingMessages(maxPendingMessages) + .topic("persistent://my-property/my-ns/my-topic2"); + + Producer producer = producerBuilder.create(); + List> futures = Lists.newArrayList(); + + // Asynchronously produce messages + byte[] message = new byte[ClientCnx.getMaxMessageSize() + 1]; + for (int i = 0; i < maxPendingMessages + 10; i++) { + Future future = producer.sendAsync(message); + try { + future.get(); + fail("should fail with InvalidMessageException"); + } catch (Exception e) { + assertTrue(e.getCause() instanceof PulsarClientException.InvalidMessageException); + } + } + log.info("-- Exiting {} test --", methodName); + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index eb059096e254e..7e5cb3ba79036 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -351,7 +351,7 @@ public void sendAsync(Message message, SendCallback callback) { PulsarClientException.InvalidMessageException invalidMessageException = new PulsarClientException.InvalidMessageException( format("The producer %s of the topic %s sends a %s message with %d bytes that exceeds %d bytes", producerName, topic, compressedStr, compressedSize, ClientCnx.getMaxMessageSize())); - callback.sendComplete(invalidMessageException); + completeCallbackAndReleaseSemaphore(callback, invalidMessageException); return; } } @@ -360,7 +360,7 @@ public void sendAsync(Message message, SendCallback callback) { PulsarClientException.InvalidMessageException invalidMessageException = new PulsarClientException.InvalidMessageException( format("The producer %s of the topic %s can not reuse the same message", producerName, topic)); - callback.sendComplete(invalidMessageException); + completeCallbackAndReleaseSemaphore(callback, invalidMessageException); compressedPayload.release(); return; } @@ -455,11 +455,9 @@ public void sendAsync(Message message, SendCallback callback) { } } } catch (PulsarClientException e) { - semaphore.release(); - callback.sendComplete(e); + completeCallbackAndReleaseSemaphore(callback, e); } catch (Throwable t) { - semaphore.release(); - callback.sendComplete(new PulsarClientException(t)); + completeCallbackAndReleaseSemaphore(callback, new PulsarClientException(t)); } } @@ -471,8 +469,9 @@ private boolean populateMessageSchema(MessageImpl msg, SendCallback callback) { return true; } if (!isMultiSchemaEnabled(true)) { - callback.sendComplete(new PulsarClientException.InvalidMessageException( - format("The producer %s of the topic %s is disabled the `MultiSchema`", producerName, topic))); + PulsarClientException.InvalidMessageException e = new PulsarClientException.InvalidMessageException( + format("The producer %s of the topic %s is disabled the `MultiSchema`", producerName, topic)); + completeCallbackAndReleaseSemaphore(callback, e); return false; } SchemaHash schemaHash = SchemaHash.of(msg.getSchema()); @@ -872,6 +871,11 @@ private void releaseSemaphoreForSendOp(OpSendMsg op) { semaphore.release(isBatchMessagingEnabled() ? op.numMessagesInBatch : 1); } + private void completeCallbackAndReleaseSemaphore(SendCallback callback, Exception exception) { + semaphore.release(); + callback.sendComplete(exception); + } + /** * Checks message checksum to retry if message was corrupted while sending to broker. Recomputes checksum of the * message header-payload again. From f33567ec6b805a7ae704520ab93ff87b0abb02b6 Mon Sep 17 00:00:00 2001 From: Sanjeev Kulkarni Date: Thu, 5 Mar 2020 21:19:26 -0800 Subject: [PATCH 55/74] Enhance Authorization by adding TenantAdmin interface (#6487) * Enhance Authorization by adding TenantAdmin interface * Remove debugging comment Co-authored-by: Sanjeev Kulkarni --- .../authorization/AuthorizationProvider.java | 13 ++++++++++ .../authorization/AuthorizationService.java | 9 +++++++ .../admin/impl/PersistentTopicsBase.java | 2 +- .../pulsar/broker/web/PulsarWebResource.java | 26 +++++++++---------- .../pulsar/broker/admin/NamespacesTest.java | 2 ++ .../pulsar/io/PulsarFunctionTlsTest.java | 5 ++++ .../service/BrokerDiscoveryProvider.java | 2 +- .../worker/rest/api/ComponentImpl.java | 4 +-- .../worker/rest/api/FunctionsImplTest.java | 19 +++++++++----- .../proxy/server/BrokerDiscoveryProvider.java | 2 +- 10 files changed, 59 insertions(+), 25 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java index 9787eaebc7ebc..cb6541623eb7f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationProvider.java @@ -29,6 +29,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.TenantInfo; /** * Provider of authorization mechanism @@ -46,6 +47,18 @@ default CompletableFuture isSuperUser(String role, ServiceConfiguration return CompletableFuture.completedFuture(role != null && superUserRoles.contains(role) ? true : false); } + /** + * Check if specified role is an admin of the tenant + * @param tenant the tenant to check + * @param role the role to check + * @return a CompletableFuture containing a boolean in which true means the role is an admin user + * and false if it is not + */ + default CompletableFuture isTenantAdmin(String tenant, String role, TenantInfo tenantInfo, + AuthenticationDataSource authenticationData) { + return CompletableFuture.completedFuture(role != null && tenantInfo.getAdminRoles() != null && tenantInfo.getAdminRoles().contains(role) ? true : false); + } + /** * Perform initialization for the authorization provider * diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java index 95ff7643f98cb..381e8cf780e3e 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/AuthorizationService.java @@ -26,6 +26,7 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.util.FutureUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -76,6 +77,14 @@ public CompletableFuture isSuperUser(String user) { return FutureUtil.failedFuture(new IllegalStateException("No authorization provider configured")); } + public CompletableFuture isTenantAdmin(String tenant, String role, TenantInfo tenantInfo, + AuthenticationDataSource authenticationData) { + if (provider != null) { + return provider.isTenantAdmin(tenant, role, tenantInfo, authenticationData); + } + return FutureUtil.failedFuture(new IllegalStateException("No authorization provider configured")); + } + /** * * Grant authorization-action permission on a namespace to the given client diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 503b21b430d58..e37f09da31ff5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -1797,7 +1797,7 @@ public static CompletableFuture getPartitionedTopicMet checkAuthorization(pulsar, topicName, clientAppId, authenticationData); } catch (RestException e) { try { - validateAdminAccessForTenant(pulsar, clientAppId, originalPrincipal, topicName.getTenant()); + validateAdminAccessForTenant(pulsar, clientAppId, originalPrincipal, topicName.getTenant(), authenticationData); } catch (RestException authException) { log.warn("Failed to authorize {} on cluster {}", clientAppId, topicName.toString()); throw new PulsarClientException(String.format("Authorization failed %s on topic %s with error %s", diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java index ca9ec8567b006..497883de73563 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/web/PulsarWebResource.java @@ -52,6 +52,8 @@ import org.apache.pulsar.broker.admin.AdminResource; import org.apache.pulsar.broker.authentication.AuthenticationDataHttps; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; +import org.apache.pulsar.broker.authorization.AuthorizationProvider; +import org.apache.pulsar.broker.authorization.AuthorizationService; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.common.naming.Constants; import org.apache.pulsar.common.naming.NamespaceBundle; @@ -224,7 +226,7 @@ protected void validateSuperUserAccess() { */ protected void validateAdminAccessForTenant(String tenant) { try { - validateAdminAccessForTenant(pulsar(), clientAppId(), originalPrincipal(), tenant); + validateAdminAccessForTenant(pulsar(), clientAppId(), originalPrincipal(), tenant, clientAuthData()); } catch (RestException e) { throw e; } catch (Exception e) { @@ -234,7 +236,8 @@ protected void validateAdminAccessForTenant(String tenant) { } protected static void validateAdminAccessForTenant(PulsarService pulsar, String clientAppId, - String originalPrincipal, String tenant) + String originalPrincipal, String tenant, + AuthenticationDataSource authenticationData) throws RestException, Exception { if (log.isDebugEnabled()) { log.debug("check admin access on tenant: {} - Authenticated: {} -- role: {}", tenant, @@ -259,22 +262,17 @@ protected static void validateAdminAccessForTenant(PulsarService pulsar, String validateOriginalPrincipal(pulsar.getConfiguration().getProxyRoles(), clientAppId, originalPrincipal); if (pulsar.getConfiguration().getProxyRoles().contains(clientAppId)) { - CompletableFuture isProxySuperUserFuture; CompletableFuture isOriginalPrincipalSuperUserFuture; try { - isProxySuperUserFuture = pulsar.getBrokerService() - .getAuthorizationService() - .isSuperUser(clientAppId); + AuthorizationService authorizationService = pulsar.getBrokerService().getAuthorizationService(); + isProxySuperUserFuture = authorizationService.isSuperUser(clientAppId); - isOriginalPrincipalSuperUserFuture = pulsar.getBrokerService() - .getAuthorizationService() - .isSuperUser(originalPrincipal); + isOriginalPrincipalSuperUserFuture = authorizationService.isSuperUser(originalPrincipal); - Set adminRoles = tenantInfo.getAdminRoles(); - boolean proxyAuthorized = isProxySuperUserFuture.get() || adminRoles.contains(clientAppId); - boolean originalPrincipalAuthorized - = isOriginalPrincipalSuperUserFuture.get() || adminRoles.contains(originalPrincipal); + boolean proxyAuthorized = isProxySuperUserFuture.get() || authorizationService.isTenantAdmin(tenant, clientAppId, tenantInfo, authenticationData).get(); + boolean originalPrincipalAuthorized + = isOriginalPrincipalSuperUserFuture.get() || authorizationService.isTenantAdmin(tenant, originalPrincipal, tenantInfo, authenticationData).get(); if (!proxyAuthorized || !originalPrincipalAuthorized) { throw new RestException(Status.UNAUTHORIZED, String.format("Proxy not authorized to access resource (proxy:%s,original:%s)", @@ -290,7 +288,7 @@ protected static void validateAdminAccessForTenant(PulsarService pulsar, String .getAuthorizationService() .isSuperUser(clientAppId) .join()) { - if (!tenantInfo.getAdminRoles().contains(clientAppId)) { + if (!pulsar.getBrokerService().getAuthorizationService().isTenantAdmin(tenant, clientAppId, tenantInfo, authenticationData).get()) { throw new RestException(Status.UNAUTHORIZED, "Don't have permission to administrate resources on this tenant"); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java index a607921952667..0f7d100e4d122 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/NamespacesTest.java @@ -145,6 +145,7 @@ public void setup() throws Exception { doReturn(false).when(namespaces).isRequestHttps(); doReturn("test").when(namespaces).clientAppId(); doReturn(null).when(namespaces).originalPrincipal(); + doReturn(null).when(namespaces).clientAuthData(); doReturn(Sets.newTreeSet(Lists.newArrayList("use", "usw", "usc", "global"))).when(namespaces).clusters(); doNothing().when(namespaces).validateAdminAccessForTenant(this.testTenant); doNothing().when(namespaces).validateAdminAccessForTenant("non-existing-tenant"); @@ -987,6 +988,7 @@ public void testValidateTopicOwnership() throws Exception { doReturn(false).when(topics).isRequestHttps(); doReturn("test").when(topics).clientAppId(); doReturn(null).when(topics).originalPrincipal(); + doReturn(null).when(topics).clientAuthData(); mockWebUrl(localWebServiceUrl, testNs); doReturn("persistent").when(topics).domain(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java index 8b7e92e6f51d0..ed7a0ddd9660a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionTlsTest.java @@ -42,6 +42,8 @@ import org.apache.pulsar.broker.ServiceConfigurationUtils; import org.apache.pulsar.broker.authentication.AuthenticationProviderTls; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.broker.authorization.AuthorizationService; +import org.apache.pulsar.broker.cache.ConfigurationCacheService; import org.apache.pulsar.client.admin.Namespaces; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -109,13 +111,16 @@ void setup(Method method) throws Exception { Set providers = new HashSet<>(); providers.add(AuthenticationProviderTls.class.getName()); config.setAuthenticationEnabled(true); + config.setAuthorizationEnabled(true); config.setAuthenticationProviders(providers); config.setTlsCertificateFilePath(TLS_SERVER_CERT_FILE_PATH); config.setTlsKeyFilePath(TLS_SERVER_KEY_FILE_PATH); config.setTlsAllowInsecureConnection(true); functionsWorkerService = spy(createPulsarFunctionWorker(config)); AuthenticationService authenticationService = new AuthenticationService(config); + AuthorizationService authorizationService = new AuthorizationService(config, mock(ConfigurationCacheService.class)); when(functionsWorkerService.getAuthenticationService()).thenReturn(authenticationService); + when(functionsWorkerService.getAuthorizationService()).thenReturn(authorizationService); when(functionsWorkerService.isInitialized()).thenReturn(true); PulsarAdmin admin = mock(PulsarAdmin.class); diff --git a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/BrokerDiscoveryProvider.java b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/BrokerDiscoveryProvider.java index 8b3a012120fcb..de3fef1026d92 100644 --- a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/BrokerDiscoveryProvider.java +++ b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/BrokerDiscoveryProvider.java @@ -155,7 +155,7 @@ protected static void checkAuthorization(DiscoveryService service, TopicName top throw new IllegalAccessException(String.format("Failed to get property %s admin data due to %s", topicName.getTenant(), e.getMessage())); } - if (!tenantInfo.getAdminRoles().contains(role)) { + if (!service.getAuthorizationService().isTenantAdmin(topicName.getTenant(), role, tenantInfo, authenticationData).get()) { throw new IllegalAccessException("Don't have permission to administrate resources on this property"); } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java index 98cc3a5eb83bc..cab2b07e4022b 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java @@ -1449,10 +1449,10 @@ public boolean isAuthorizedRole(String tenant, String namespace, String clientRo if (clientRole != null) { try { TenantInfo tenantInfo = worker().getBrokerAdmin().tenants().getTenantInfo(tenant); - if (tenantInfo != null && tenantInfo.getAdminRoles() != null && tenantInfo.getAdminRoles().contains(clientRole)) { + if (tenantInfo != null && worker().getAuthorizationService().isTenantAdmin(tenant, clientRole, tenantInfo, authenticationData).get()) { return true; } - } catch (PulsarAdminException.NotFoundException e) { + } catch (PulsarAdminException.NotFoundException | InterruptedException | ExecutionException e) { } } diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java index 258c123797745..cd55b2aee0396 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java @@ -20,6 +20,7 @@ import org.apache.distributedlog.api.namespace.Namespace; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; +import org.apache.pulsar.broker.authorization.AuthorizationService; import org.apache.pulsar.client.admin.Namespaces; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -60,6 +61,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyInt; @@ -230,51 +232,56 @@ public void testMetricsEmpty() { } @Test - public void testIsAuthorizedRole() throws PulsarAdminException { - + public void testIsAuthorizedRole() throws PulsarAdminException, InterruptedException, ExecutionException { + TenantInfo tenantInfo = new TenantInfo(); + AuthenticationDataSource authenticationDataSource = mock(AuthenticationDataSource.class); FunctionsImpl functionImpl = spy(new FunctionsImpl(() -> mockedWorkerService)); + AuthorizationService authorizationService = mock(AuthorizationService.class); + doReturn(authorizationService).when(mockedWorkerService).getAuthorizationService(); WorkerConfig workerConfig = new WorkerConfig(); workerConfig.setAuthorizationEnabled(true); workerConfig.setSuperUserRoles(Collections.singleton(superUser)); doReturn(workerConfig).when(mockedWorkerService).getWorkerConfig(); // test super user - AuthenticationDataSource authenticationDataSource = mock(AuthenticationDataSource.class); assertTrue(functionImpl.isAuthorizedRole("test-tenant", "test-ns", superUser, authenticationDataSource)); // test normal user functionImpl = spy(new FunctionsImpl(() -> mockedWorkerService)); doReturn(false).when(functionImpl).allowFunctionOps(any(), any(), any()); Tenants tenants = mock(Tenants.class); - when(tenants.getTenantInfo(any())).thenReturn(new TenantInfo()); + when(tenants.getTenantInfo(any())).thenReturn(tenantInfo); PulsarAdmin admin = mock(PulsarAdmin.class); when(admin.tenants()).thenReturn(tenants); when(this.mockedWorkerService.getBrokerAdmin()).thenReturn(admin); + when(authorizationService.isTenantAdmin("test-tenant", "test-user", tenantInfo, authenticationDataSource)).thenReturn(CompletableFuture.completedFuture(false)); assertFalse(functionImpl.isAuthorizedRole("test-tenant", "test-ns", "test-user", authenticationDataSource)); // if user is tenant admin functionImpl = spy(new FunctionsImpl(() -> mockedWorkerService)); doReturn(false).when(functionImpl).allowFunctionOps(any(), any(), any()); tenants = mock(Tenants.class); - TenantInfo tenantInfo = new TenantInfo(); tenantInfo.setAdminRoles(Collections.singleton("test-user")); when(tenants.getTenantInfo(any())).thenReturn(tenantInfo); admin = mock(PulsarAdmin.class); when(admin.tenants()).thenReturn(tenants); when(this.mockedWorkerService.getBrokerAdmin()).thenReturn(admin); + when(authorizationService.isTenantAdmin("test-tenant", "test-user", tenantInfo, authenticationDataSource)).thenReturn(CompletableFuture.completedFuture(true)); assertTrue(functionImpl.isAuthorizedRole("test-tenant", "test-ns", "test-user", authenticationDataSource)); // test user allow function action functionImpl = spy(new FunctionsImpl(() -> mockedWorkerService)); doReturn(true).when(functionImpl).allowFunctionOps(any(), any(), any()); tenants = mock(Tenants.class); - when(tenants.getTenantInfo(any())).thenReturn(new TenantInfo()); + tenantInfo.setAdminRoles(Collections.emptySet()); + when(tenants.getTenantInfo(any())).thenReturn(tenantInfo); admin = mock(PulsarAdmin.class); when(admin.tenants()).thenReturn(tenants); when(this.mockedWorkerService.getBrokerAdmin()).thenReturn(admin); + when(authorizationService.isTenantAdmin("test-tenant", "test-user", tenantInfo, authenticationDataSource)).thenReturn(CompletableFuture.completedFuture(true)); assertTrue(functionImpl.isAuthorizedRole("test-tenant", "test-ns", "test-user", authenticationDataSource)); // test role is null diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/BrokerDiscoveryProvider.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/BrokerDiscoveryProvider.java index 28f28cdcfac24..e055872795cc1 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/BrokerDiscoveryProvider.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/BrokerDiscoveryProvider.java @@ -153,7 +153,7 @@ protected static void checkAuthorization(ProxyService service, TopicName topicNa throw new IllegalAccessException(String.format("Failed to get property %s admin data due to %s", topicName.getTenant(), e.getMessage())); } - if (!tenantInfo.getAdminRoles().contains(role)) { + if (!service.getAuthorizationService().isTenantAdmin(topicName.getTenant(), role, tenantInfo, authenticationData).get()) { throw new IllegalAccessException("Don't have permission to administrate resources on this tenant"); } } From 8003d08e5ca325867d2e825921f18ddda8d4e1d4 Mon Sep 17 00:00:00 2001 From: congbo <39078850+congbobo184@users.noreply.github.com> Date: Fri, 6 Mar 2020 14:28:30 +0800 Subject: [PATCH 56/74] Independent schema is set for each consumer generated by topic (#6356) ### Motivation Master Issue: #5454 When one Consumer subscribe multi topic, setSchemaInfoPorvider() will be covered by the consumer generated by the last topic. ### Modification clone schema for each consumer generated by topic. ### Verifying this change Add the schemaTest for it. --- .../JsonSchemaCompatibilityCheckTest.java | 5 + .../org/apache/pulsar/schema/SchemaTest.java | 136 ++++++++++++++++++ .../schema/{compatibility => }/Schemas.java | 2 +- .../SchemaCompatibilityCheckTest.java | 22 +-- .../org/apache/pulsar/client/api/Schema.java | 9 +- .../kafka/compat/PulsarKafkaSchema.java | 5 + .../kafka/compat/PulsarKafkaSchema.java | 5 + .../client/impl/MultiTopicsConsumerImpl.java | 7 +- .../pulsar/client/impl/PulsarClientImpl.java | 21 +-- .../client/impl/schema/AbstractSchema.java | 5 + .../client/impl/schema/AutoConsumeSchema.java | 14 ++ .../impl/schema/AutoProduceBytesSchema.java | 5 + .../pulsar/client/impl/schema/AvroSchema.java | 10 ++ .../client/impl/schema/KeyValueSchema.java | 5 + .../impl/schema/KeyValueSchemaInfo.java | 5 + .../schema/generic/GenericAvroSchema.java | 10 ++ .../schema/generic/GenericSchemaImpl.java | 1 - .../pulsar/functions/source/SerDeSchema.java | 4 + 18 files changed, 244 insertions(+), 27 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java rename pulsar-broker/src/test/java/org/apache/pulsar/schema/{compatibility => }/Schemas.java (96%) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/JsonSchemaCompatibilityCheckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/JsonSchemaCompatibilityCheckTest.java index 09a0c44fbf312..7befab0942f26 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/JsonSchemaCompatibilityCheckTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/JsonSchemaCompatibilityCheckTest.java @@ -124,5 +124,10 @@ public static OldJSONSchema of(Class pojo, Map propert info.setSchema(mapper.writeValueAsBytes(schema)); return new OldJSONSchema<>(info, pojo, mapper); } + + @Override + public Schema clone() { + return this; + } } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java new file mode 100644 index 0000000000000..eba8ec074411c --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java @@ -0,0 +1,136 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.schema; + +import com.google.common.collect.Sets; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.schema.SchemaDefinition; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.util.Collections; + +import static org.apache.pulsar.common.naming.TopicName.PUBLIC_TENANT; +import static org.apache.pulsar.schema.compatibility.SchemaCompatibilityCheckTest.randomName; +import static org.junit.Assert.assertEquals; + +public class SchemaTest extends MockedPulsarServiceBaseTest { + + private final static String CLUSTER_NAME = "test"; + + @BeforeMethod + @Override + public void setup() throws Exception { + super.internalSetup(); + + // Setup namespaces + admin.clusters().createCluster(CLUSTER_NAME, new ClusterData(pulsar.getBrokerServiceUrl())); + TenantInfo tenantInfo = new TenantInfo(); + tenantInfo.setAllowedClusters(Collections.singleton(CLUSTER_NAME)); + admin.tenants().createTenant(PUBLIC_TENANT, tenantInfo); + } + + @AfterMethod + @Override + public void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testMultiTopicSetSchemaProvider() throws Exception { + final String tenant = PUBLIC_TENANT; + final String namespace = "test-namespace-" + randomName(16); + final String topicOne = "test-multi-version-schema-one"; + final String topicTwo = "test-multi-version-schema-two"; + final String fqtnOne = TopicName.get( + TopicDomain.persistent.value(), + tenant, + namespace, + topicOne + ).toString(); + + final String fqtnTwo = TopicName.get( + TopicDomain.persistent.value(), + tenant, + namespace, + topicTwo + ).toString(); + + + admin.namespaces().createNamespace( + tenant + "/" + namespace, + Sets.newHashSet(CLUSTER_NAME) + ); + + admin.topics().createPartitionedTopic(fqtnOne, 3); + admin.topics().createPartitionedTopic(fqtnTwo, 3); + + admin.schemas().createSchema(fqtnOne, Schema.AVRO( + SchemaDefinition.builder().withAlwaysAllowNull + (false).withSupportSchemaVersioning(true). + withPojo(Schemas.PersonOne.class).build()).getSchemaInfo()); + + admin.schemas().createSchema(fqtnOne, Schema.AVRO( + SchemaDefinition.builder().withAlwaysAllowNull + (false).withSupportSchemaVersioning(true). + withPojo(Schemas.PersonTwo.class).build()).getSchemaInfo()); + + admin.schemas().createSchema(fqtnTwo, Schema.AVRO( + SchemaDefinition.builder().withAlwaysAllowNull + (false).withSupportSchemaVersioning(true). + withPojo(Schemas.PersonTwo.class).build()).getSchemaInfo()); + + Producer producer = pulsarClient.newProducer(Schema.AVRO( + SchemaDefinition.builder().withAlwaysAllowNull + (false).withSupportSchemaVersioning(true). + withPojo(Schemas.PersonTwo.class).build())) + .topic(fqtnOne) + .create(); + + Schemas.PersonTwo personTwo = new Schemas.PersonTwo(); + personTwo.setId(1); + personTwo.setName("Tom"); + + + Consumer consumer = pulsarClient.newConsumer(Schema.AVRO( + SchemaDefinition.builder().withAlwaysAllowNull + (false).withSupportSchemaVersioning(true). + withPojo(Schemas.PersonTwo.class).build())) + .subscriptionName("test") + .topic(fqtnOne, fqtnTwo) + .subscribe(); + + producer.send(personTwo); + + Schemas.PersonTwo personConsume = consumer.receive().getValue(); + assertEquals("Tom", personConsume.getName()); + assertEquals(1, personConsume.getId()); + + producer.close(); + consumer.close(); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/Schemas.java b/pulsar-broker/src/test/java/org/apache/pulsar/schema/Schemas.java similarity index 96% rename from pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/Schemas.java rename to pulsar-broker/src/test/java/org/apache/pulsar/schema/Schemas.java index 0978547b9d1e9..df02574bf8afc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/Schemas.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/schema/Schemas.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.schema.compatibility; +package org.apache.pulsar.schema; import lombok.AllArgsConstructor; import lombok.Data; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java index 4903a68cbb496..f94d28b92854c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java @@ -35,6 +35,7 @@ import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.schema.Schemas; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -46,7 +47,6 @@ import static org.apache.pulsar.common.naming.TopicName.PUBLIC_TENANT; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; @Slf4j @@ -141,7 +141,7 @@ public void testConsumerCompatibilityCheckCanReadLastTest(SchemaCompatibilityStr Schemas.PersonOne personOne = new Schemas.PersonOne(); - personOne.id = 1; + personOne.setId(1); producerOne.send(personOne); Message message = null; @@ -162,16 +162,16 @@ public void testConsumerCompatibilityCheckCanReadLastTest(SchemaCompatibilityStr .create(); Schemas.PersonTwo personTwo = new Schemas.PersonTwo(); - personTwo.id = 1; - personTwo.name = "Jerry"; + personTwo.setId(1); + personTwo.setName("Jerry"); producerTwo.send(personTwo); message = consumerThree.receive(); Schemas.PersonThree personThree = message.getValue(); consumerThree.acknowledge(message); - assertEquals(personThree.id, 1); - assertEquals(personThree.name, "Jerry"); + assertEquals(personThree.getId(), 1); + assertEquals(personThree.getName(), "Jerry"); consumerThree.close(); producerOne.close(); @@ -270,8 +270,8 @@ public void testIsAutoUpdateSchema(SchemaCompatibilityStrategy schemaCompatibili Schemas.PersonTwo personTwo = message.getValue(); consumerTwo.acknowledge(message); - assertEquals(personTwo.id, 2); - assertEquals(personTwo.name, "Lucy"); + assertEquals(personTwo.getId(), 2); + assertEquals(personTwo.getName(), "Lucy"); producer.close(); consumerTwo.close(); @@ -287,8 +287,8 @@ public void testIsAutoUpdateSchema(SchemaCompatibilityStrategy schemaCompatibili personTwo = message.getValue(); consumerTwo.acknowledge(message); - assertEquals(personTwo.id, 2); - assertEquals(personTwo.name, "Lucy"); + assertEquals(personTwo.getId(), 2); + assertEquals(personTwo.getName(), "Lucy"); consumerTwo.close(); producer.close(); @@ -338,7 +338,7 @@ public void testProducerSendWithOldSchemaAndConsumerCanRead(SchemaCompatibilityS Message message = consumerOne.receive(); personOne = message.getValue(); - assertEquals(10, personOne.id); + assertEquals(10, personOne.getId()); consumerOne.close(); producerOne.close(); diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Schema.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Schema.java index bb0e85a33c754..0c871b35b03af 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Schema.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Schema.java @@ -35,7 +35,7 @@ /** * Message schema definition. */ -public interface Schema { +public interface Schema extends Cloneable{ /** * Check if the message is a valid object for this schema. @@ -136,6 +136,13 @@ default void configureSchemaInfo(String topic, String componentName, // no-op } + /** + * Duplicates the schema. + * + * @return The duplicated schema. + */ + Schema clone(); + /** * Schema that doesn't perform any encoding on the message payloads. Accepts a byte array and it passes it through. */ diff --git a/pulsar-client-kafka-compat/pulsar-client-kafka/src/main/java/org/apache/pulsar/client/kafka/compat/PulsarKafkaSchema.java b/pulsar-client-kafka-compat/pulsar-client-kafka/src/main/java/org/apache/pulsar/client/kafka/compat/PulsarKafkaSchema.java index 807f482dd6c56..aef6dd16d7d09 100644 --- a/pulsar-client-kafka-compat/pulsar-client-kafka/src/main/java/org/apache/pulsar/client/kafka/compat/PulsarKafkaSchema.java +++ b/pulsar-client-kafka-compat/pulsar-client-kafka/src/main/java/org/apache/pulsar/client/kafka/compat/PulsarKafkaSchema.java @@ -74,4 +74,9 @@ public T decode(byte[] message) { public SchemaInfo getSchemaInfo() { return Schema.BYTES.getSchemaInfo(); } + + @Override + public Schema clone() { + return this; + } } diff --git a/pulsar-client-kafka-compat/pulsar-client-kafka_0_9/src/main/java/org/apache/pulsar/client/kafka/compat/PulsarKafkaSchema.java b/pulsar-client-kafka-compat/pulsar-client-kafka_0_9/src/main/java/org/apache/pulsar/client/kafka/compat/PulsarKafkaSchema.java index 807f482dd6c56..aef6dd16d7d09 100644 --- a/pulsar-client-kafka-compat/pulsar-client-kafka_0_9/src/main/java/org/apache/pulsar/client/kafka/compat/PulsarKafkaSchema.java +++ b/pulsar-client-kafka-compat/pulsar-client-kafka_0_9/src/main/java/org/apache/pulsar/client/kafka/compat/PulsarKafkaSchema.java @@ -74,4 +74,9 @@ public T decode(byte[] message) { public SchemaInfo getSchemaInfo() { return Schema.BYTES.getSchemaInfo(); } + + @Override + public Schema clone() { + return this; + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 5c2609573aa78..970e1343c3adb 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -800,16 +800,17 @@ private CompletableFuture subscribeAsync(String topicName, int numberParti private void subscribeTopicPartitions(CompletableFuture subscribeResult, String topicName, int numPartitions, boolean createIfDoesNotExist) { - client.preProcessSchemaBeforeSubscribe(client, schema, topicName).whenComplete((ignored, cause) -> { + client.preProcessSchemaBeforeSubscribe(client, schema, topicName).whenComplete((schema, cause) -> { if (null == cause) { - doSubscribeTopicPartitions(subscribeResult, topicName, numPartitions, createIfDoesNotExist); + doSubscribeTopicPartitions(schema, subscribeResult, topicName, numPartitions, createIfDoesNotExist); } else { subscribeResult.completeExceptionally(cause); } }); } - private void doSubscribeTopicPartitions(CompletableFuture subscribeResult, String topicName, int numPartitions, + private void doSubscribeTopicPartitions(Schema schema, + CompletableFuture subscribeResult, String topicName, int numPartitions, boolean createIfDoesNotExist) { if (log.isDebugEnabled()) { log.debug("Subscribe to topic {} metadata.partitions: {}", topicName, numPartitions); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 2fe8201aeb7d5..27a158d4813fd 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -335,7 +335,7 @@ public CompletableFuture> subscribeAsync(ConsumerConfigurationDa private CompletableFuture> singleTopicSubscribeAsync(ConsumerConfigurationData conf, Schema schema, ConsumerInterceptors interceptors) { return preProcessSchemaBeforeSubscribe(this, schema, conf.getSingleTopic()) - .thenCompose(ignored -> doSingleTopicSubscribeAsync(conf, schema, interceptors)); + .thenCompose(schemaClone -> doSingleTopicSubscribeAsync(conf, schemaClone, interceptors)); } private CompletableFuture> doSingleTopicSubscribeAsync(ConsumerConfigurationData conf, Schema schema, ConsumerInterceptors interceptors) { @@ -448,7 +448,7 @@ public CompletableFuture> createReaderAsync(ReaderConfigurationDa public CompletableFuture> createReaderAsync(ReaderConfigurationData conf, Schema schema) { return preProcessSchemaBeforeSubscribe(this, schema, conf.getTopicName()) - .thenCompose(ignored -> doCreateReaderAsync(conf, schema)); + .thenCompose(schemaClone -> doCreateReaderAsync(conf, schemaClone)); } CompletableFuture> doCreateReaderAsync(ReaderConfigurationData conf, Schema schema) { @@ -768,8 +768,8 @@ private LoadingCache getSchemaProviderLoadingCache() } @SuppressWarnings("unchecked") - protected CompletableFuture preProcessSchemaBeforeSubscribe(PulsarClientImpl pulsarClientImpl, - Schema schema, + protected CompletableFuture> preProcessSchemaBeforeSubscribe(PulsarClientImpl pulsarClientImpl, + Schema schema, String topicName) { if (schema != null && schema.supportSchemaVersioning()) { final SchemaInfoProvider schemaInfoProvider; @@ -779,11 +779,12 @@ protected CompletableFuture preProcessSchemaBeforeSubscribe(PulsarClientIm log.error("Failed to load schema info provider for topic {}", topicName, e); return FutureUtil.failedFuture(e.getCause()); } - + schema = schema.clone(); if (schema.requireFetchingSchemaInfo()) { + Schema finalSchema = schema; return schemaInfoProvider.getLatestSchema().thenCompose(schemaInfo -> { if (null == schemaInfo) { - if (!(schema instanceof AutoConsumeSchema)) { + if (!(finalSchema instanceof AutoConsumeSchema)) { // no schema info is found return FutureUtil.failedFuture( new PulsarClientException.NotFoundException( @@ -792,18 +793,18 @@ protected CompletableFuture preProcessSchemaBeforeSubscribe(PulsarClientIm } try { log.info("Configuring schema for topic {} : {}", topicName, schemaInfo); - schema.configureSchemaInfo(topicName, "topic", schemaInfo); + finalSchema.configureSchemaInfo(topicName, "topic", schemaInfo); } catch (RuntimeException re) { return FutureUtil.failedFuture(re); } - schema.setSchemaInfoProvider(schemaInfoProvider); - return CompletableFuture.completedFuture(null); + finalSchema.setSchemaInfoProvider(schemaInfoProvider); + return CompletableFuture.completedFuture(finalSchema); }); } else { schema.setSchemaInfoProvider(schemaInfoProvider); } } - return CompletableFuture.completedFuture(null); + return CompletableFuture.completedFuture(schema); } // diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AbstractSchema.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AbstractSchema.java index f459d5cdc4dd7..10843281ef8c0 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AbstractSchema.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AbstractSchema.java @@ -61,4 +61,9 @@ T decode(ByteBuf byteBuf, byte[] schemaVersion) { // ignore version by default (most of the primitive schema implementations ignore schema version) return decode(byteBuf); } + + @Override + public Schema clone() { + return this; + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AutoConsumeSchema.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AutoConsumeSchema.java index 27e8e6e7aa085..049b0f55e5b98 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AutoConsumeSchema.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AutoConsumeSchema.java @@ -132,6 +132,20 @@ public void configureSchemaInfo(String topicName, } } + @Override + public Schema clone() { + Schema schema = Schema.AUTO_CONSUME(); + if (this.schema != null) { + schema.configureSchemaInfo(topicName, componentName, this.schema.getSchemaInfo()); + } else { + schema.configureSchemaInfo(topicName, componentName, null); + } + if (schemaInfoProvider != null) { + schema.setSchemaInfoProvider(schemaInfoProvider); + } + return schema; + } + private GenericSchema generateSchema(SchemaInfo schemaInfo) { if (schemaInfo.getType() != SchemaType.AVRO && schemaInfo.getType() != SchemaType.JSON) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AutoProduceBytesSchema.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AutoProduceBytesSchema.java index dd5193f7a3519..7578ffafa4483 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AutoProduceBytesSchema.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AutoProduceBytesSchema.java @@ -95,4 +95,9 @@ public SchemaInfo getSchemaInfo() { return schema.getSchemaInfo(); } + + @Override + public Schema clone() { + return new AutoProduceBytesSchema<>(schema.clone()); + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AvroSchema.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AvroSchema.java index f4d130f5ba03b..fe801dd1ea48e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AvroSchema.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/AvroSchema.java @@ -22,6 +22,7 @@ import org.apache.avro.Conversions; import org.apache.avro.data.TimeConversions; import org.apache.avro.reflect.ReflectData; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.schema.SchemaDefinition; import org.apache.pulsar.client.api.schema.SchemaReader; import org.apache.pulsar.client.impl.schema.reader.AvroReader; @@ -75,6 +76,15 @@ public boolean supportSchemaVersioning() { return true; } + @Override + public Schema clone() { + Schema schema = new AvroSchema<>(schemaInfo); + if (schemaInfoProvider != null) { + schema.setSchemaInfoProvider(schemaInfoProvider); + } + return schema; + } + public static AvroSchema of(SchemaDefinition schemaDefinition) { return new AvroSchema<>(parseSchemaInfo(schemaDefinition, SchemaType.AVRO)); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/KeyValueSchema.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/KeyValueSchema.java index 0e173d21d158f..b81a94706bbc8 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/KeyValueSchema.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/KeyValueSchema.java @@ -193,6 +193,11 @@ public void configureSchemaInfo(String topicName, } } + @Override + public Schema> clone() { + return KeyValueSchema.of(keySchema.clone(), valueSchema.clone(), keyValueEncodingType); + } + private void configureKeyValueSchemaInfo() { this.schemaInfo = KeyValueSchemaInfo.encodeKeyValueSchemaInfo( keySchema, valueSchema, keyValueEncodingType diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/KeyValueSchemaInfo.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/KeyValueSchemaInfo.java index 120f5a75bb661..95735261f494b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/KeyValueSchemaInfo.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/KeyValueSchemaInfo.java @@ -48,6 +48,11 @@ public byte[] encode(SchemaInfo si) { public SchemaInfo getSchemaInfo() { return BytesSchema.BYTES.getSchemaInfo(); } + + @Override + public Schema clone() { + return this; + } }; private static final String KEY_SCHEMA_NAME = "key.schema.name"; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericAvroSchema.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericAvroSchema.java index 91220b4221ca7..98e646ea2ba50 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericAvroSchema.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericAvroSchema.java @@ -54,6 +54,16 @@ public boolean supportSchemaVersioning() { return true; } + @Override + public org.apache.pulsar.client.api.Schema clone() { + org.apache.pulsar.client.api.Schema schema = + GenericAvroSchema.of(schemaInfo, useProvidedSchemaAsReaderSchema); + if (schemaInfoProvider != null) { + schema.setSchemaInfoProvider(schemaInfoProvider); + } + return schema; + } + @Override protected SchemaReader loadReader(BytesSchemaVersion schemaVersion) { SchemaInfo schemaInfo = getSchemaInfoByVersion(schemaVersion.get()); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericSchemaImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericSchemaImpl.java index f22c449bfe80c..7d18e52979a30 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericSchemaImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/generic/GenericSchemaImpl.java @@ -76,5 +76,4 @@ public static GenericSchemaImpl of(SchemaInfo schemaInfo, + schemaInfo.getType() + "'"); } } - } diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/source/SerDeSchema.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/source/SerDeSchema.java index 8d4bf1ff39a85..c1adfcc4b80c5 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/source/SerDeSchema.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/source/SerDeSchema.java @@ -49,4 +49,8 @@ public SchemaInfo getSchemaInfo() { return null; } + @Override + public Schema clone() { + return this; + } } From f2ec1b4e2836859b0a6beb9b5a12656e4bcaf8f9 Mon Sep 17 00:00:00 2001 From: Rolf Arne Corneliussen Date: Fri, 6 Mar 2020 07:32:28 +0100 Subject: [PATCH 57/74] Fix memory leak when running topic compaction. (#6485) Fixes #6482 ### Motivation Prevent topic compaction from leaking direct memory ### Modifications Several leaks were discovered using Netty leak detection and code review. * `CompactedTopicImpl.readOneMessageId` would get an `Enumeration` of `LedgerEntry`, but did not release the underlying buffers. Fix: iterate though the `Enumeration` and release underlying buffer. Instead of logging the case where the `Enumeration` did not contain any elements, complete the future exceptionally with the message (will be logged by Caffeine). * Two main sources of leak in `TwoPhaseCompactor`. The `RawBacthConverter.rebatchMessage` method failed to close/release a `ByteBuf` (uncompressedPayload). Also, the return ByteBuf of `RawBacthConverter.rebatchMessage` was not closed. The first one was easy to fix (release buffer), to fix the second one and make the code easier to read, I decided to not let `RawBacthConverter.rebatchMessage` close the message read from the topic, instead the message read from the topic can be closed in a try/finally clause surrounding most of the method body handing a message from a topic (in phase two loop). Then if a new message was produced by `RawBacthConverter.rebatchMessage` we check that after we have added the message to the compact ledger and release the message. ### Verifying this change Modified `RawReaderTest.testBatchingRebatch` to show new contract. One can run the test described to reproduce the issue, to verify no leak is detected. --- .../pulsar/client/impl/RawBatchConverter.java | 5 +- .../pulsar/compaction/CompactedTopicImpl.java | 19 ++- .../pulsar/compaction/TwoPhaseCompactor.java | 119 ++++++++++-------- .../pulsar/client/impl/RawReaderTest.java | 4 +- 4 files changed, 82 insertions(+), 65 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java index e252426acbbfa..8c21a737e65be 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java @@ -91,8 +91,7 @@ public static List> extractIdsAndKeys(RawMessage * Take a batched message and a filter, and returns a message with the only the sub-messages * which match the filter. Returns an empty optional if no messages match. * - * This takes ownership of the passes in message, and if the returned optional is not empty, - * the ownership of that message is returned also. + * NOTE: this message does not alter the reference count of the RawMessage argument. */ public static Optional rebatchMessage(RawMessage msg, BiPredicate filter) @@ -161,9 +160,9 @@ public static Optional rebatchMessage(RawMessage msg, return Optional.empty(); } } finally { + uncompressedPayload.release(); batchBuffer.release(); metadata.recycle(); - msg.close(); } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java index b1378b648bf81..22efe8e5c5719 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java @@ -164,12 +164,19 @@ private static CompletableFuture readOneMessageId(LedgerHandle lh if (rc != BKException.Code.OK) { promise.completeExceptionally(BKException.create(rc)); } else { - try (RawMessage m = RawMessageImpl.deserializeFrom( - seq.nextElement().getEntryBuffer())) { - promise.complete(m.getMessageIdData()); - } catch (NoSuchElementException e) { - log.error("No such entry {} in ledger {}", entryId, lh.getId()); - promise.completeExceptionally(e); + // Need to release buffers for all entries in the sequence + if (seq.hasMoreElements()) { + LedgerEntry entry = seq.nextElement(); + try (RawMessage m = RawMessageImpl.deserializeFrom(entry.getEntryBuffer())) { + entry.getEntryBuffer().release(); + while (seq.hasMoreElements()) { + seq.nextElement().getEntryBuffer().release(); + } + promise.complete(m.getMessageIdData()); + } + } else { + promise.completeExceptionally(new NoSuchElementException( + String.format("No such entry %d in ledger %d", entryId, lh.getId()))); } } }, null); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java index 95f6f1ad7f5ad..a275bb5fb107e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java @@ -212,77 +212,88 @@ private CompletableFuture phaseTwoSeekThenLoop(RawReader reader, MessageId private void phaseTwoLoop(RawReader reader, MessageId to, Map latestForKey, LedgerHandle lh, Semaphore outstanding, CompletableFuture promise) { + if (promise.isDone()) { + return; + } reader.readNextAsync().whenCompleteAsync( (m, exception) -> { if (exception != null) { promise.completeExceptionally(exception); return; } else if (promise.isDone()) { + m.close(); return; } - MessageId id = m.getMessageId(); - Optional messageToAdd = Optional.empty(); - if (RawBatchConverter.isReadableBatch(m)) { - try { - messageToAdd = RawBatchConverter.rebatchMessage( - m, (key, subid) -> latestForKey.get(key).equals(subid)); - } catch (IOException ioe) { - log.info("Error decoding batch for message {}. Whole batch will be included in output", - id, ioe); - messageToAdd = Optional.of(m); - } - } else { - Pair keyAndSize = extractKeyAndSize(m); - MessageId msg; - if (keyAndSize == null) { // pass through messages without a key - messageToAdd = Optional.of(m); - } else if ((msg = latestForKey.get(keyAndSize.getLeft())) != null - && msg.equals(id)) { // consider message only if present into latestForKey map - if (keyAndSize.getRight() <= 0) { - promise.completeExceptionally(new IllegalArgumentException( - "Compaction phase found empty record from sorted key-map")); + try { + MessageId id = m.getMessageId(); + Optional messageToAdd = Optional.empty(); + if (RawBatchConverter.isReadableBatch(m)) { + try { + messageToAdd = RawBatchConverter.rebatchMessage( + m, (key, subid) -> latestForKey.get(key).equals(subid)); + } catch (IOException ioe) { + log.info("Error decoding batch for message {}. Whole batch will be included in output", + id, ioe); + messageToAdd = Optional.of(m); } - messageToAdd = Optional.of(m); } else { - m.close(); + Pair keyAndSize = extractKeyAndSize(m); + MessageId msg; + if (keyAndSize == null) { // pass through messages without a key + messageToAdd = Optional.of(m); + } else if ((msg = latestForKey.get(keyAndSize.getLeft())) != null + && msg.equals(id)) { // consider message only if present into latestForKey map + if (keyAndSize.getRight() <= 0) { + promise.completeExceptionally(new IllegalArgumentException( + "Compaction phase found empty record from sorted key-map")); + } + messageToAdd = Optional.of(m); + } } - } - if (messageToAdd.isPresent()) { - try { - outstanding.acquire(); - CompletableFuture addFuture = addToCompactedLedger(lh, messageToAdd.get()) - .whenComplete((res, exception2) -> { - outstanding.release(); - if (exception2 != null) { - promise.completeExceptionally(exception2); + if (messageToAdd.isPresent()) { + RawMessage message = messageToAdd.get(); + try { + outstanding.acquire(); + CompletableFuture addFuture = addToCompactedLedger(lh, message) + .whenComplete((res, exception2) -> { + outstanding.release(); + if (exception2 != null) { + promise.completeExceptionally(exception2); + } + }); + if (to.equals(id)) { + addFuture.whenComplete((res, exception2) -> { + if (exception2 == null) { + promise.complete(null); } }); - if (to.equals(id)) { - addFuture.whenComplete((res, exception2) -> { - if (exception2 == null) { - promise.complete(null); - } - }); + } + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + promise.completeExceptionally(ie); + } finally { + if (message != m) { + message.close(); + } } - } catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - promise.completeExceptionally(ie); - } - } else if (to.equals(id)) { - // Reached to last-id and phase-one found it deleted-message while iterating on ledger so, not - // present under latestForKey. Complete the compaction. - try { - // make sure all inflight writes have finished - outstanding.acquire(MAX_OUTSTANDING); - promise.complete(null); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - promise.completeExceptionally(e); + } else if (to.equals(id)) { + // Reached to last-id and phase-one found it deleted-message while iterating on ledger so, + // not present under latestForKey. Complete the compaction. + try { + // make sure all inflight writes have finished + outstanding.acquire(MAX_OUTSTANDING); + promise.complete(null); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + promise.completeExceptionally(e); + } + return; } - return; + phaseTwoLoop(reader, to, latestForKey, lh, outstanding, promise); + } finally { + m.close(); } - phaseTwoLoop(reader, to, latestForKey, lh, outstanding, promise); }, scheduler); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java index b0c7cd1830055..5ae46185c4cb0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawReaderTest.java @@ -319,13 +319,13 @@ public void testBatchingRebatch() throws Exception { } RawReader reader = RawReader.create(pulsarClient, topic, subscription).get(); - try { - RawMessage m1 = reader.readNextAsync().get(); + try (RawMessage m1 = reader.readNextAsync().get()) { RawMessage m2 = RawBatchConverter.rebatchMessage(m1, (key, id) -> key.equals("key2")).get(); List> idsAndKeys = RawBatchConverter.extractIdsAndKeys(m2); Assert.assertEquals(idsAndKeys.size(), 1); Assert.assertEquals(idsAndKeys.get(0).getRight(), "key2"); m2.close(); + Assert.assertEquals(m1.getHeadersAndPayload().refCnt(), 1); } finally { reader.closeAsync().get(); } From 19ccfd5c60020a32bceeca128a9846ca006f0dc7 Mon Sep 17 00:00:00 2001 From: lipenghui Date: Fri, 6 Mar 2020 16:50:10 +0800 Subject: [PATCH 58/74] Fix create partitioned topic with a substring of an existing topic name. (#6478) Fixes #6468 Fix create a partitioned topic with a substring of an existing topic name. And make create partitioned topic async. --- .../pulsar/broker/admin/AdminResource.java | 114 +++++++++++++++++- .../admin/impl/PersistentTopicsBase.java | 97 +++------------ .../broker/admin/v1/NonPersistentTopics.java | 34 +----- .../broker/admin/v1/PersistentTopics.java | 16 ++- .../broker/admin/v2/NonPersistentTopics.java | 37 ++---- .../broker/admin/v2/PersistentTopics.java | 29 ++++- .../pulsar/broker/admin/AdminApiTest.java | 8 +- .../apache/pulsar/broker/admin/AdminTest.java | 6 +- .../broker/admin/PersistentTopicsTest.java | 57 +++++++-- .../broker/admin/v1/V1_AdminApiTest.java | 4 +- 10 files changed, 227 insertions(+), 175 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 722da4f49214c..a21698298488b 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 @@ -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,16 +258,19 @@ protected List getListOfNamespaces(String property) throws Exception { return namespaces; } - protected void tryCreatePartitionsAsync(int numPartitions) { + protected CompletableFuture tryCreatePartitionsAsync(int numPartitions) { if (!topicName.isPersistent()) { - return; + return CompletableFuture.completedFuture(null); } + List> 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 tryCreatePartitionAsync(final int partition, CompletableFuture reuseFuture) { + CompletableFuture 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) { @@ -272,18 +278,22 @@ private void tryCreatePartitionAsync(final int partition) { 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 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) -> { + 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 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)); + } + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index e37f09da31ff5..f2a95d1879a84 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -20,7 +20,7 @@ import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; -import org.apache.pulsar.common.api.proto.PulsarApi; + import static org.apache.pulsar.common.util.Codec.decode; import com.github.zafarkhaja.semver.Version; @@ -390,46 +390,6 @@ protected void internalRevokePermissionsOnTopic(String role) { revokePermissions(topicName.toString(), role); } - protected void internalCreatePartitionedTopic(int numPartitions) { - validateAdminAccessForTenant(topicName.getTenant()); - if (numPartitions <= 0) { - throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 0"); - } - validatePartitionTopicName(topicName.getLocalName()); - try { - boolean topicExist = pulsar().getNamespaceService() - .getListOfTopics(topicName.getNamespaceObject(), PulsarApi.CommandGetTopicsOfNamespace.Mode.ALL) - .join() - .contains(topicName.toString()); - if (topicExist) { - log.warn("[{}] Failed to create already existing topic {}", clientAppId(), topicName); - throw new RestException(Status.CONFLICT, "This topic already exists"); - } - } catch (Exception e) { - log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e); - throw new RestException(e); - } - try { - String path = ZkAdminPaths.partitionedTopicPath(topicName); - byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions)); - zkCreateOptimistic(path, data); - tryCreatePartitionsAsync(numPartitions); - // Sync data to all quorums and the observers - zkSync(path); - log.info("[{}] Successfully created partitioned topic {}", clientAppId(), topicName); - } catch (KeeperException.NodeExistsException e) { - log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), topicName); - throw new RestException(Status.CONFLICT, "Partitioned topic already exists"); - } catch (KeeperException.BadVersionException e) { - log.warn("[{}] Failed to create partitioned topic {}: concurrent modification", clientAppId(), - topicName); - throw new RestException(Status.CONFLICT, "Concurrent modification"); - } catch (Exception e) { - log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e); - throw new RestException(e); - } - } - protected void internalCreateNonPartitionedTopic(boolean authoritative) { validateAdminAccessForTenant(topicName.getTenant()); validateNonPartitionTopicName(topicName.getLocalName()); @@ -540,11 +500,22 @@ protected void internalUpdatePartitionedTopic(int numPartitions, boolean updateL } } - protected void internalCreateMissedPartitions() { - PartitionedTopicMetadata metadata = getPartitionedTopicMetadata(topicName, false, false); - if (metadata != null) { - tryCreatePartitionsAsync(metadata.partitions); - } + protected void internalCreateMissedPartitions(AsyncResponse asyncResponse) { + getPartitionedTopicMetadataAsync(topicName, false, false).thenAccept(metadata -> { + if (metadata != null) { + tryCreatePartitionsAsync(metadata.partitions).thenAccept(v -> { + asyncResponse.resume(Response.noContent().build()); + }).exceptionally(e -> { + log.error("[{}] Failed to create partitions for topic {}", clientAppId(), topicName); + resumeAsyncResponseExceptionally(asyncResponse, e); + return null; + }); + } + }).exceptionally(e -> { + log.error("[{}] Failed to create partitions for topic {}", clientAppId(), topicName); + resumeAsyncResponseExceptionally(asyncResponse, e); + return null; + }); } private CompletableFuture updatePartitionInOtherCluster(int numPartitions, Set clusters) { @@ -2071,40 +2042,6 @@ private void validatePartitionTopicUpdate(String topicName, int numberOfPartitio } } - /** - * Validate partitioned topic name. - * Validation will fail and throw RestException if - * 1) There's already a partitioned topic with same topic name and have some of its partition created. - * 2) There's already non partition topic with same name and contains partition suffix "-partition-" - * followed by numeric value. In this case internal created partition of partitioned topic could override - * the existing non partition topic. - * - * @param topicName - */ - private void validatePartitionTopicName(String topicName) { - List existingTopicList = internalGetList(); - String prefix = topicName + TopicName.PARTITIONED_TOPIC_SUFFIX; - for (String existingTopicName : existingTopicList) { - if (existingTopicName.contains(prefix)) { - try { - Long.parseLong(existingTopicName.substring( - existingTopicName.indexOf(TopicName.PARTITIONED_TOPIC_SUFFIX) - + TopicName.PARTITIONED_TOPIC_SUFFIX.length())); - log.warn("[{}] Already have topic {} which contains partition " + - "suffix '-partition-' and end with numeric value. Creation of partitioned topic {}" - + "could cause conflict.", clientAppId(), existingTopicName, topicName); - throw new RestException(Status.PRECONDITION_FAILED, - "Already have topic " + existingTopicName + " which contains partition suffix '-partition-' " + - "and end with numeric value, Creation of partitioned topic " + topicName + - " could cause conflict."); - } catch (NumberFormatException e) { - // Do nothing, if value after partition suffix is not pure numeric value, - // as it can't conflict with internal created partitioned topic's name. - } - } - } - } - /** * Validate non partition topic name, * Validation will fail and throw RestException if diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java index 4bc0ddfd154d9..2338b0f055af5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java @@ -124,41 +124,15 @@ public PersistentTopicInternalStats getInternalStats(@PathParam("property") Stri @ApiOperation(hidden = true, value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 409, message = "Partitioned topic already exist") }) - public void createPartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, + public void createPartitionedTopic(@Suspended final AsyncResponse asyncResponse, @PathParam("property") String property, @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic, int numPartitions) { - validateTopicName(property, cluster, namespace, encodedTopic); - validateAdminAccessForTenant(topicName.getTenant()); - if (numPartitions <= 0) { - throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 0"); - } try { - boolean topicExist = pulsar().getNamespaceService() - .getListOfTopics(topicName.getNamespaceObject(), PulsarApi.CommandGetTopicsOfNamespace.Mode.ALL) - .join() - .contains(topicName.toString()); - if (topicExist) { - log.warn("[{}] Failed to create already existing topic {}", clientAppId(), topicName); - throw new RestException(Status.CONFLICT, "This topic already exists"); - } - } catch (Exception e) { - log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e); - throw new RestException(e); - } - try { - String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(), - topicName.getEncodedLocalName()); - byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions)); - zkCreateOptimistic(path, data); - // Sync data to all quorums and the observers - zkSync(path); - log.info("[{}] Successfully created partitioned topic {}", clientAppId(), topicName); - } catch (KeeperException.NodeExistsException e) { - log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), topicName); - throw new RestException(Status.CONFLICT, "Partitioned topic already exist"); + validateTopicName(property, cluster, namespace, encodedTopic); + internalCreatePartitionedTopic(asyncResponse, numPartitions); } catch (Exception e) { log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e); - throw new RestException(e); + resumeAsyncResponseExceptionally(asyncResponse, e); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java index 836ca14282781..362adc81c4a52 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/PersistentTopics.java @@ -57,7 +57,8 @@ import io.swagger.annotations.ApiOperation; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; -import javax.ws.rs.container.AsyncResponse; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** */ @@ -66,7 +67,7 @@ @Api(value = "/persistent", description = "Persistent topic admin apis", tags = "persistent topic", hidden = true) @SuppressWarnings("deprecation") public class PersistentTopics extends PersistentTopicsBase { - + private static final Logger log = LoggerFactory.getLogger(PersistentTopics.class); @GET @Path("/{property}/{cluster}/{namespace}") @ApiOperation(hidden = true, value = "Get the list of topics under a namespace.", response = String.class, responseContainer = "List") @@ -147,11 +148,16 @@ public void revokePermissionsOnTopic(@PathParam("property") String property, @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 409, message = "Partitioned topic already exist") }) - public void createPartitionedTopic(@PathParam("property") String property, @PathParam("cluster") String cluster, + public void createPartitionedTopic(@Suspended final AsyncResponse asyncResponse, @PathParam("property") String property, @PathParam("cluster") String cluster, @PathParam("namespace") String namespace, @PathParam("topic") @Encoded String encodedTopic, int numPartitions) { - validateTopicName(property, cluster, namespace, encodedTopic); - internalCreatePartitionedTopic(numPartitions); + try { + validateTopicName(property, cluster, namespace, encodedTopic); + internalCreatePartitionedTopic(asyncResponse, numPartitions); + } catch (Exception e) { + log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + } } /** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index 7e88eed2fe5ab..3756f8285e7d4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -164,6 +164,7 @@ public PersistentTopicInternalStats getInternalStats( @ApiResponse(code = 503, message = "Failed to validate global cluster configuration"), }) public void createPartitionedTopic( + @Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Specify the tenant", required = true) @PathParam("tenant") String tenant, @ApiParam(value = "Specify the namespace", required = true) @@ -172,39 +173,15 @@ public void createPartitionedTopic( @PathParam("topic") @Encoded String encodedTopic, @ApiParam(value = "The number of partitions for the topic", required = true, type = "int", defaultValue = "0") int numPartitions) { - validateGlobalNamespaceOwnership(tenant,namespace); - validateTopicName(tenant, namespace, encodedTopic); - validateAdminAccessForTenant(topicName.getTenant()); - if (numPartitions <= 0) { - throw new RestException(Status.NOT_ACCEPTABLE, "Number of partitions should be more than 0"); - } - try { - boolean topicExist = pulsar().getNamespaceService() - .getListOfTopics(topicName.getNamespaceObject(), PulsarApi.CommandGetTopicsOfNamespace.Mode.ALL) - .join() - .contains(topicName.toString()); - if (topicExist) { - log.warn("[{}] Failed to create already existing topic {}", clientAppId(), topicName); - throw new RestException(Status.CONFLICT, "This topic already exists"); - } - } catch (Exception e) { - log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e); - throw new RestException(e); - } + try { - String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(), - topicName.getEncodedLocalName()); - byte[] data = jsonMapper().writeValueAsBytes(new PartitionedTopicMetadata(numPartitions)); - zkCreateOptimistic(path, data); - // Sync data to all quorums and the observers - zkSync(path); - log.info("[{}] Successfully created partitioned topic {}", clientAppId(), topicName); - } catch (KeeperException.NodeExistsException e) { - log.warn("[{}] Failed to create already existing partitioned topic {}", clientAppId(), topicName); - throw new RestException(Status.CONFLICT, "Partitioned topic already exists"); + validateGlobalNamespaceOwnership(tenant,namespace); + validateTopicName(tenant, namespace, encodedTopic); + validateAdminAccessForTenant(topicName.getTenant()); + internalCreatePartitionedTopic(asyncResponse, numPartitions); } catch (Exception e) { log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e); - throw new RestException(e); + resumeAsyncResponseExceptionally(asyncResponse, e); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java index 8c59fa5b6b079..b2fc28b010a69 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/PersistentTopics.java @@ -55,6 +55,9 @@ import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; import io.swagger.annotations.ApiParam; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import static org.apache.pulsar.common.util.Codec.decode; /** @@ -192,6 +195,7 @@ public void revokePermissionsOnTopic( @ApiResponse(code = 503, message = "Failed to validate global cluster configuration") }) public void createPartitionedTopic( + @Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Specify the tenant", required = true) @PathParam("tenant") String tenant, @ApiParam(value = "Specify the namespace", required = true) @@ -200,9 +204,15 @@ public void createPartitionedTopic( @PathParam("topic") @Encoded String encodedTopic, @ApiParam(value = "The number of partitions for the topic", required = true, type = "int", defaultValue = "0") int numPartitions) { - validateGlobalNamespaceOwnership(tenant,namespace); - validatePartitionedTopicName(tenant, namespace, encodedTopic); - internalCreatePartitionedTopic(numPartitions); + try { + validateGlobalNamespaceOwnership(tenant,namespace); + validatePartitionedTopicName(tenant, namespace, encodedTopic); + validateAdminAccessForTenant(topicName.getTenant()); + internalCreatePartitionedTopic(asyncResponse, numPartitions); + } catch (Exception e) { + log.error("[{}] Failed to create partitioned topic {}", clientAppId(), topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + } } @PUT @@ -276,7 +286,7 @@ public void updatePartitionedTopic( @POST @Path("/{tenant}/{namespace}/{topic}/createMissedPartitions") - @ApiOperation(value = "Create missed partitions of an existing partitioned topic.", notes = "This is a best-effort operation for create missed partitions of existing non-global partitioned-topic and does't throw any exceptions when create failed") + @ApiOperation(value = "Create missed partitions of an existing partitioned topic.") @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to adminisActions to be grantedtrate resources on this tenant"), @@ -287,6 +297,7 @@ public void updatePartitionedTopic( @ApiResponse(code = 500, message = "Internal server error") }) public void createMissedPartitions( + @Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Specify the tenant", required = true) @PathParam("tenant") String tenant, @ApiParam(value = "Specify the namespace", required = true) @@ -294,8 +305,12 @@ public void createMissedPartitions( @ApiParam(value = "Specify topic name", required = true) @PathParam("topic") @Encoded String encodedTopic) { - validatePartitionedTopicName(tenant, namespace, encodedTopic); - internalCreateMissedPartitions(); + try { + validatePartitionedTopicName(tenant, namespace, encodedTopic); + internalCreateMissedPartitions(asyncResponse); + } catch (Exception e) { + resumeAsyncResponseExceptionally(asyncResponse, e); + } } @GET @@ -1072,4 +1087,6 @@ public MessageId getLastMessageId( validateTopicName(tenant, namespace, encodedTopic); return internalGetLastMessageId(authoritative); } + + private static final Logger log = LoggerFactory.getLogger(PersistentTopics.class); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java index f0ee4f60e985c..d8326285076fb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiTest.java @@ -919,9 +919,7 @@ public void partitionedTopics(String topicName) throws Exception { try { admin.topics().createPartitionedTopic(partitionedTopicName, 32); fail("Should have failed as the partitioned topic already exists"); - } catch (PreconditionFailedException e) { - // Expecting PreconditionFailedException instead of ConflictException as it'll - // fail validation before actually try to create metadata in ZK. + } catch (ConflictException ignore) { } producer = client.newProducer(Schema.BYTES) @@ -2010,6 +2008,10 @@ public void testPersistentTopicCreation() throws Exception { } catch (PulsarAdminException e) { assertTrue(e instanceof ConflictException); } + + // Check create partitioned topic with substring topic name + admin.topics().createPartitionedTopic("persistent://prop-xyz/ns1/create_substring_topic", 1); + admin.topics().createPartitionedTopic("persistent://prop-xyz/ns1/substring_topic", 1); } /** diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java index 00d1a31b1ef53..3cae752f436b8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTest.java @@ -677,7 +677,11 @@ void persistentTopics() throws Exception { verify(response, times(1)).resume(Lists.newArrayList()); // create topic assertEquals(persistentTopics.getPartitionedTopicList(property, cluster, namespace), Lists.newArrayList()); - persistentTopics.createPartitionedTopic(property, cluster, namespace, topic, 5); + response = mock(AsyncResponse.class); + ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); + persistentTopics.createPartitionedTopic(response, property, cluster, namespace, topic, 5); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); assertEquals(persistentTopics.getPartitionedTopicList(property, cluster, namespace), Lists .newArrayList(String.format("persistent://%s/%s/%s/%s", property, cluster, namespace, topic))); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java index 1825d313124e7..cac35a6e1eb18 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java @@ -56,6 +56,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; @@ -157,13 +158,17 @@ public void testGetSubscriptions() { "Partitioned Topic not found: persistent://my-tenant/my-namespace/topic-not-found-partition-0 has zero partitions"); // 3) Create the partitioned topic - persistentTopics.createPartitionedTopic(testTenant, testNamespace, testLocalTopicName, 3); + response = mock(AsyncResponse.class); + ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); + persistentTopics.createPartitionedTopic(response, testTenant, testNamespace, testLocalTopicName, 3); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); // 4) Create a subscription response = mock(AsyncResponse.class); persistentTopics.createSubscription(response, testTenant, testNamespace, testLocalTopicName, "test", true, (MessageIdImpl) MessageId.earliest, false); - ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); + responseCaptor = ArgumentCaptor.forClass(Response.class); verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); @@ -239,7 +244,7 @@ public void testCreateNonPartitionedTopicWithInvalidName() { persistentTopics.createNonPartitionedTopic(testTenant, testNamespace, topicName, true); } - @Test(expectedExceptions = RestException.class) + @Test public void testCreatePartitionedTopicHavingNonPartitionTopicWithPartitionSuffix() throws KeeperException, InterruptedException { // Test the case in which user already has topic like topic-name-partition-123 created before we enforce the validation. final String nonPartitionTopicName1 = "standard-topic"; @@ -250,7 +255,12 @@ public void testCreatePartitionedTopicHavingNonPartitionTopicWithPartitionSuffix doReturn(mockLocalZooKeeperCacheService).when(pulsar).getLocalZkCacheService(); doReturn(mockZooKeeperChildrenCache).when(mockLocalZooKeeperCacheService).managedLedgerListCache(); doReturn(ImmutableSet.of(nonPartitionTopicName1, nonPartitionTopicName2)).when(mockZooKeeperChildrenCache).get(anyString()); - persistentTopics.createPartitionedTopic(testTenant, testNamespace, partitionedTopicName, 5); + doReturn(CompletableFuture.completedFuture(ImmutableSet.of(nonPartitionTopicName1, nonPartitionTopicName2))).when(mockZooKeeperChildrenCache).getAsync(anyString()); + AsyncResponse response = mock(AsyncResponse.class); + ArgumentCaptor errCaptor = ArgumentCaptor.forClass(RestException.class); + persistentTopics.createPartitionedTopic(response, testTenant, testNamespace, partitionedTopicName, 5); + verify(response, timeout(5000).times(1)).resume(errCaptor.capture()); + Assert.assertEquals(errCaptor.getValue().getResponse().getStatus(), Response.Status.CONFLICT.getStatusCode()); } @Test(expectedExceptions = RestException.class) @@ -263,13 +273,18 @@ public void testUpdatePartitionedTopicHavingNonPartitionTopicWithPartitionSuffix doReturn(mockLocalZooKeeperCacheService).when(pulsar).getLocalZkCacheService(); doReturn(mockZooKeeperChildrenCache).when(mockLocalZooKeeperCacheService).managedLedgerListCache(); doReturn(ImmutableSet.of(nonPartitionTopicName2)).when(mockZooKeeperChildrenCache).get(anyString()); + doReturn(CompletableFuture.completedFuture(ImmutableSet.of(nonPartitionTopicName2))).when(mockZooKeeperChildrenCache).getAsync(anyString()); doAnswer(invocation -> { persistentTopics.namespaceName = NamespaceName.get("tenant", "namespace"); persistentTopics.topicName = TopicName.get("persistent", "tenant", "cluster", "namespace", "topicname"); return null; }).when(persistentTopics).validatePartitionedTopicName(any(), any(), any()); doNothing().when(persistentTopics).validateAdminAccessForTenant(anyString()); - persistentTopics.createPartitionedTopic(testTenant, testNamespace, partitionedTopicName, 5); + AsyncResponse response = mock(AsyncResponse.class); + ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); + persistentTopics.createPartitionedTopic(response, testTenant, testNamespace, partitionedTopicName, 5); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); persistentTopics.updatePartitionedTopic(testTenant, testNamespace, partitionedTopicName, true, 10); } @@ -295,7 +310,11 @@ public void testUnloadTopic() { // 3) create partitioned topic and unload response = mock(AsyncResponse.class); - persistentTopics.createPartitionedTopic(testTenant, testNamespace, partitionTopicName, 6); + responseCaptor = ArgumentCaptor.forClass(Response.class); + persistentTopics.createPartitionedTopic(response, testTenant, testNamespace, partitionTopicName, 6); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); + response = mock(AsyncResponse.class); persistentTopics.unloadTopic(response, testTenant, testNamespace, partitionTopicName, true); responseCaptor = ArgumentCaptor.forClass(Response.class); verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); @@ -320,10 +339,17 @@ public void testUnloadTopicShallThrowNotFoundWhenTopicNotExist() { @Test public void testGetPartitionedTopicsList() throws KeeperException, InterruptedException, PulsarAdminException { + AsyncResponse response = mock(AsyncResponse.class); + ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); + persistentTopics.createPartitionedTopic(response, testTenant, testNamespace, "test-topic1", 3); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); - persistentTopics.createPartitionedTopic(testTenant, testNamespace, "test-topic1", 3); - - nonPersistentTopic.createPartitionedTopic(testTenant, testNamespace, "test-topic2", 3); + response = mock(AsyncResponse.class); + responseCaptor = ArgumentCaptor.forClass(Response.class); + nonPersistentTopic.createPartitionedTopic(response, testTenant, testNamespace, "test-topic2", 3); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); List persistentPartitionedTopics = persistentTopics.getPartitionedTopicList(testTenant, testNamespace); @@ -351,7 +377,11 @@ public void testGrantNonPartitionedTopic() { public void testGrantPartitionedTopic() { final String partitionedTopicName = "partitioned-topic"; final int numPartitions = 5; - persistentTopics.createPartitionedTopic(testTenant, testNamespace, partitionedTopicName, numPartitions); + AsyncResponse response = mock(AsyncResponse.class); + ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); + persistentTopics.createPartitionedTopic(response, testTenant, testNamespace, partitionedTopicName, numPartitions); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); String role = "role"; Set expectActions = new HashSet<>(); @@ -387,8 +417,11 @@ public void testRevokeNonPartitionedTopic() { public void testRevokePartitionedTopic() { final String partitionedTopicName = "partitioned-topic"; final int numPartitions = 5; - persistentTopics.createPartitionedTopic(testTenant, testNamespace, partitionedTopicName, numPartitions); - + AsyncResponse response = mock(AsyncResponse.class); + ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); + persistentTopics.createPartitionedTopic(response, testTenant, testNamespace, partitionedTopicName, numPartitions); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + Assert.assertEquals(responseCaptor.getValue().getStatus(), Response.Status.NO_CONTENT.getStatusCode()); String role = "role"; Set expectActions = new HashSet<>(); expectActions.add(AuthAction.produce); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java index 373a6b7c48766..78e3dc16592ba 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java @@ -879,9 +879,7 @@ public void partitionedTopics(String topicName) throws Exception { try { admin.topics().createPartitionedTopic(partitionedTopicName, 32); fail("Should have failed as the partitioned topic exists with its partition created"); - } catch (PreconditionFailedException e) { - // Expecting PreconditionFailedException instead of ConflictException as it'll - // fail validation before actually try to create metadata in ZK. + } catch (ConflictException ignore) { } producer = client.newProducer(Schema.BYTES) From a3e1efc229f4839c4e8edec690f8ca52ddb5894a Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Fri, 6 Mar 2020 18:26:50 +0800 Subject: [PATCH 59/74] Bump jcloud version to 2.2.0 and remove jcloud-shade module (#6494) In jclouds 2.2.0, the [gson is shaded internally](https://issues.apache.org/jira/browse/JCLOUDS-1166). We could safely remove the jcloud-shade module as a cleanup. --- jclouds-shaded/pom.xml | 141 ------------------ pom.xml | 11 +- tiered-storage/jcloud/pom.xml | 90 ++--------- .../impl/BlobStoreBackedReadHandleImpl.java | 1 - .../impl/BlobStoreManagedLedgerOffloader.java | 2 +- .../BlobStoreBackedInputStreamTest.java | 3 - .../BlobStoreManagedLedgerOffloaderTest.java | 2 +- 7 files changed, 20 insertions(+), 230 deletions(-) delete mode 100644 jclouds-shaded/pom.xml diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml deleted file mode 100644 index 63fd4c9fa4079..0000000000000 --- a/jclouds-shaded/pom.xml +++ /dev/null @@ -1,141 +0,0 @@ - - - - 4.0.0 - - org.apache.pulsar - pulsar - 2.6.0-SNAPSHOT - .. - - - jclouds-shaded - Apache Pulsar :: Jclouds shaded - - - - com.google.code.gson - gson - 2.5 - - - org.apache.jclouds - jclouds-allblobstore - - - - - - - - com.google.guava - guava - 18.0 - - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - package - - shade - - - true - true - false - - - - com.google.code.gson:gson - com.google.guava:guava - org.apache.jclouds:* - org.apache.jclouds.api:* - org.apache.jclouds.common:* - org.apache.jclouds.provider:* - com.google.inject.extensions:guice-assistedinject - com.google.inject:guice - com.google.inject.extensions:guice-multibindings - javax.ws.rs:* - com.jamesmurty.utils:* - net.iharder:* - aopalliance:* - javax.inject:* - javax.annotation:* - com.google.errorprone:* - - - - - - com.google - org.apache.pulsar.jcloud.shade.com.google - - - javax.ws - org.apache.pulsar.jcloud.shade.javax.ws - - - com.jamesmurty.utils - org.apache.pulsar.jcloud.shade.com.jamesmurty.utils - - - aopalliance - org.apache.pulsar.jcloud.shade.aopalliance - - - net.iharder - org.apache.pulsar.jcloud.shade.net.iharder - - - javax.inject - org.apache.pulsar.jcloud.shade.javax.inject - - - javax.annotation - org.apache.pulsar.jcloud.shade.javax.annotation - - - com.google.errorprone - org.apache.pulsar.jcloud.shade.com.google.errorprone - - - - - - - - - - - - - - diff --git a/pom.xml b/pom.xml index 6c3c1bcf45cce..59aff775ecfeb 100644 --- a/pom.xml +++ b/pom.xml @@ -110,9 +110,6 @@ flexible messaging model and an intuitive client API. pulsar-transaction - - jclouds-shaded - pulsar-functions @@ -185,7 +182,7 @@ flexible messaging model and an intuitive client API. 1.11.297 1.9.1 2.10.1 - 2.1.1 + 2.2.0 3.8.11.2 8.0.11 3.2.0 @@ -1027,6 +1024,12 @@ flexible messaging model and an intuitive client API. ${jclouds.version} + + org.apache.jclouds + jclouds-blobstore + ${jclouds.version} + + com.lmax diff --git a/tiered-storage/jcloud/pom.xml b/tiered-storage/jcloud/pom.xml index 8544d204005f2..47896ae7573dc 100644 --- a/tiered-storage/jcloud/pom.xml +++ b/tiered-storage/jcloud/pom.xml @@ -38,50 +38,27 @@ managed-ledger ${project.version} + - org.apache.pulsar - jclouds-shaded - ${pulsar.jclouds.shaded.version} - - - com.google.code.gson - gson - - - com.google.guava - guava - - - org.apache.jclouds - * - - - org.apache.jclouds.api - * - - - org.apache.jclouds.common - * - - - org.apache.jclouds.provider - * - - + org.apache.jclouds + jclouds-allblobstore + + + + org.apache.jclouds + jclouds-blobstore + com.amazonaws aws-java-sdk-core + com.amazonaws aws-java-sdk-sts - - com.jamesmurty.utils - java-xmlbuilder - 1.1 - + org.apache.pulsar managed-ledger @@ -89,51 +66,6 @@ test-jar test - - - - net.iharder - base64 - 2.3.8 - test - - - javax.ws.rs - javax.ws.rs-api - 2.1 - test - - - aopalliance - aopalliance - 1.0 - test - - - javax.inject - javax.inject - 1 - test - - - javax.xml.bind - jaxb-api - test - - - javax.annotation - jsr250-api - 1.0 - test - - - com.google.errorprone - error_prone_annotations - 2.1.0 - test - diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java index d64deb890df9c..47a9ce5ff7535 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java @@ -39,7 +39,6 @@ import org.apache.bookkeeper.mledger.offload.jcloud.BackedInputStream; import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlock; import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlockBuilder; -import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexEntry; import org.apache.bookkeeper.mledger.offload.jcloud.impl.BlobStoreManagedLedgerOffloader.VersionCheck; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.jclouds.blobstore.BlobStore; diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java index 0c92b96695e72..7f554403241e7 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java @@ -23,6 +23,7 @@ import com.amazonaws.auth.AWSSessionCredentials; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; +import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -49,7 +50,6 @@ import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlockBuilder; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.policies.data.OffloadPolicies; -import org.apache.pulsar.jcloud.shade.com.google.common.base.Supplier; import org.jclouds.Constants; import org.jclouds.ContextBuilder; import org.jclouds.aws.domain.SessionCredentials; diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java index baefb95a13b00..59dc9a8aae011 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java @@ -38,14 +38,11 @@ import org.jclouds.io.Payload; import org.jclouds.io.Payloads; import org.mockito.Mockito; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.testng.Assert; import org.testng.annotations.Test; @Slf4j public class BlobStoreBackedInputStreamTest extends BlobStoreTestBase { - private static final Logger log = LoggerFactory.getLogger(BlobStoreBackedInputStreamTest.class); class RandomInputStream extends InputStream { final Random r; diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java index 8aa977a0e8366..ed46a0187b84e 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java @@ -28,6 +28,7 @@ import com.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.AWSSessionCredentials; +import com.google.common.base.Supplier; import com.google.common.util.concurrent.MoreExecutors; import java.io.File; import java.io.IOException; @@ -55,7 +56,6 @@ import org.apache.bookkeeper.mledger.offload.jcloud.CredentialsUtil; import org.apache.bookkeeper.util.ZkUtils; import org.apache.pulsar.common.policies.data.OffloadPolicies; -import org.apache.pulsar.jcloud.shade.com.google.common.base.Supplier; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.MockZooKeeper; import org.apache.zookeeper.data.ACL; From 73e8beb62f9a9bfc13a347611859f1b2af0fd33c Mon Sep 17 00:00:00 2001 From: Sergii Zhevzhyk Date: Fri, 6 Mar 2020 21:50:55 +0100 Subject: [PATCH 60/74] Refactor tests in pulsar client tools test (#6472) ### Modifications The main modification was the reduction of repeated initialization of the variables in the tests. --- .../pulsar/admin/cli/CmdFunctionsTest.java | 390 +++++++----------- .../pulsar/admin/cli/utils/IOUtilsTest.java | 3 +- .../client/cli/PulsarClientToolTest.java | 3 +- 3 files changed, 157 insertions(+), 239 deletions(-) diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java index 04d833b3ffc74..c162c4b3f345a 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/CmdFunctionsTest.java @@ -81,6 +81,12 @@ public IObjectFactory getObjectFactory() { private static final String TEST_NAME = "test_name"; private static final String JAR_NAME = CmdFunctionsTest.class.getClassLoader().getResource("dummyexamples.jar").getFile(); + private static final String URL ="file:" + JAR_NAME; + private static final String FN_NAME = TEST_NAME + "-function"; + private static final String INPUT_TOPIC_NAME = TEST_NAME + "-input-topic"; + private static final String OUTPUT_TOPIC_NAME = TEST_NAME + "-output-topic"; + private static final String TENANT = TEST_NAME + "-tenant"; + private static final String NAMESPACE = TEST_NAME + "-namespace"; private PulsarAdmin admin; private Functions functions; @@ -172,14 +178,11 @@ public void testLocalRunnerCmdYaml() throws Exception { @Test public void testCreateFunction() throws Exception { - String fnName = TEST_NAME + "-function"; - String inputTopicName = TEST_NAME + "-input-topic"; - String outputTopicName = TEST_NAME + "-output-topic"; cmd.run(new String[] { "create", - "--name", fnName, - "--inputs", inputTopicName, - "--output", outputTopicName, + "--name", FN_NAME, + "--inputs", INPUT_TOPIC_NAME, + "--output", OUTPUT_TOPIC_NAME, "--jar", JAR_NAME, "--auto-ack", "false", "--tenant", "sample", @@ -190,10 +193,10 @@ public void testCreateFunction() throws Exception { }); CreateFunction creater = cmd.getCreater(); - assertEquals(fnName, creater.getFunctionName()); - assertEquals(inputTopicName, creater.getInputs()); - assertEquals(outputTopicName, creater.getOutput()); - assertEquals(new Boolean(false), creater.getAutoAck()); + assertEquals(FN_NAME, creater.getFunctionName()); + assertEquals(INPUT_TOPIC_NAME, creater.getInputs()); + assertEquals(OUTPUT_TOPIC_NAME, creater.getOutput()); + assertEquals(Boolean.FALSE, creater.getAutoAck()); assertEquals("test-dead-letter-topic", creater.getDeadLetterTopic()); assertEquals("custom-runtime-options", creater.getCustomRuntimeOptions()); @@ -203,117 +206,105 @@ public void testCreateFunction() throws Exception { @Test public void restartFunction() throws Exception { - String fnName = TEST_NAME + "-function"; String tenant = "sample"; String namespace = "ns1"; int instanceId = 0; - cmd.run(new String[] { "restart", "--tenant", tenant, "--namespace", namespace, "--name", fnName, + cmd.run(new String[] { "restart", "--tenant", tenant, "--namespace", namespace, "--name", FN_NAME, "--instance-id", Integer.toString(instanceId)}); RestartFunction restarter = cmd.getRestarter(); - assertEquals(fnName, restarter.getFunctionName()); + assertEquals(FN_NAME, restarter.getFunctionName()); - verify(functions, times(1)).restartFunction(tenant, namespace, fnName, instanceId); + verify(functions, times(1)).restartFunction(tenant, namespace, FN_NAME, instanceId); } @Test public void restartFunctionInstances() throws Exception { - String fnName = TEST_NAME + "-function"; String tenant = "sample"; String namespace = "ns1"; - cmd.run(new String[] { "restart", "--tenant", tenant, "--namespace", namespace, "--name", fnName }); + cmd.run(new String[] { "restart", "--tenant", tenant, "--namespace", namespace, "--name", FN_NAME}); RestartFunction restarter = cmd.getRestarter(); - assertEquals(fnName, restarter.getFunctionName()); + assertEquals(FN_NAME, restarter.getFunctionName()); - verify(functions, times(1)).restartFunction(tenant, namespace, fnName); + verify(functions, times(1)).restartFunction(tenant, namespace, FN_NAME); } @Test public void stopFunction() throws Exception { - String fnName = TEST_NAME + "-function"; String tenant = "sample"; String namespace = "ns1"; int instanceId = 0; - cmd.run(new String[] { "stop", "--tenant", tenant, "--namespace", namespace, "--name", fnName, + cmd.run(new String[] { "stop", "--tenant", tenant, "--namespace", namespace, "--name", FN_NAME, "--instance-id", Integer.toString(instanceId)}); StopFunction stop = cmd.getStopper(); - assertEquals(fnName, stop.getFunctionName()); + assertEquals(FN_NAME, stop.getFunctionName()); - verify(functions, times(1)).stopFunction(tenant, namespace, fnName, instanceId); + verify(functions, times(1)).stopFunction(tenant, namespace, FN_NAME, instanceId); } @Test public void stopFunctionInstances() throws Exception { - String fnName = TEST_NAME + "-function"; String tenant = "sample"; String namespace = "ns1"; - cmd.run(new String[] { "stop", "--tenant", tenant, "--namespace", namespace, "--name", fnName }); + cmd.run(new String[] { "stop", "--tenant", tenant, "--namespace", namespace, "--name", FN_NAME}); StopFunction stop = cmd.getStopper(); - assertEquals(fnName, stop.getFunctionName()); + assertEquals(FN_NAME, stop.getFunctionName()); - verify(functions, times(1)).stopFunction(tenant, namespace, fnName); + verify(functions, times(1)).stopFunction(tenant, namespace, FN_NAME); } @Test public void startFunction() throws Exception { - String fnName = TEST_NAME + "-function"; String tenant = "sample"; String namespace = "ns1"; int instanceId = 0; - cmd.run(new String[] { "start", "--tenant", tenant, "--namespace", namespace, "--name", fnName, + cmd.run(new String[] { "start", "--tenant", tenant, "--namespace", namespace, "--name", FN_NAME, "--instance-id", Integer.toString(instanceId)}); CmdFunctions.StartFunction stop = cmd.getStarter(); - assertEquals(fnName, stop.getFunctionName()); + assertEquals(FN_NAME, stop.getFunctionName()); - verify(functions, times(1)).startFunction(tenant, namespace, fnName, instanceId); + verify(functions, times(1)).startFunction(tenant, namespace, FN_NAME, instanceId); } @Test public void startFunctionInstances() throws Exception { - String fnName = TEST_NAME + "-function"; String tenant = "sample"; String namespace = "ns1"; - cmd.run(new String[] { "start", "--tenant", tenant, "--namespace", namespace, "--name", fnName }); + cmd.run(new String[] { "start", "--tenant", tenant, "--namespace", namespace, "--name", FN_NAME}); CmdFunctions.StartFunction stop = cmd.getStarter(); - assertEquals(fnName, stop.getFunctionName()); + assertEquals(FN_NAME, stop.getFunctionName()); - verify(functions, times(1)).startFunction(tenant, namespace, fnName); + verify(functions, times(1)).startFunction(tenant, namespace, FN_NAME); } @Test public void testGetFunctionStatus() throws Exception { - String fnName = TEST_NAME + "-function"; String tenant = "sample"; String namespace = "ns1"; int instanceId = 0; - cmd.run(new String[] { "getstatus", "--tenant", tenant, "--namespace", namespace, "--name", fnName, + cmd.run(new String[] { "getstatus", "--tenant", tenant, "--namespace", namespace, "--name", FN_NAME, "--instance-id", Integer.toString(instanceId)}); GetFunctionStatus status = cmd.getStatuser(); - assertEquals(fnName, status.getFunctionName()); + assertEquals(FN_NAME, status.getFunctionName()); - verify(functions, times(1)).getFunctionStatus(tenant, namespace, fnName, instanceId); + verify(functions, times(1)).getFunctionStatus(tenant, namespace, FN_NAME, instanceId); } @Test public void testCreateFunctionWithFileUrl() throws Exception { - String fnName = TEST_NAME + "-function"; - String inputTopicName = TEST_NAME + "-input-topic"; - String outputTopicName = TEST_NAME + "-output-topic"; - - final String url = "file:" + JAR_NAME; cmd.run(new String[] { "create", - "--name", fnName, - "--inputs", inputTopicName, - "--output", outputTopicName, - "--jar", url, + "--name", FN_NAME, + "--inputs", INPUT_TOPIC_NAME, + "--output", OUTPUT_TOPIC_NAME, + "--jar", URL, "--tenant", "sample", "--namespace", "ns1", "--className", DummyFunction.class.getName(), @@ -321,24 +312,19 @@ public void testCreateFunctionWithFileUrl() throws Exception { CreateFunction creater = cmd.getCreater(); - assertEquals(fnName, creater.getFunctionName()); - assertEquals(inputTopicName, creater.getInputs()); - assertEquals(outputTopicName, creater.getOutput()); + assertEquals(FN_NAME, creater.getFunctionName()); + assertEquals(INPUT_TOPIC_NAME, creater.getInputs()); + assertEquals(OUTPUT_TOPIC_NAME, creater.getOutput()); verify(functions, times(1)).createFunctionWithUrl(any(FunctionConfig.class), anyString()); } @Test public void testCreateFunctionWithoutBasicArguments() throws Exception { - String fnName = TEST_NAME + "-function"; - String inputTopicName = TEST_NAME + "-input-topic"; - String outputTopicName = TEST_NAME + "-output-topic"; - - final String url = "file:" + JAR_NAME; cmd.run(new String[] { "create", - "--inputs", inputTopicName, - "--output", outputTopicName, - "--jar", url, + "--inputs", INPUT_TOPIC_NAME, + "--output", OUTPUT_TOPIC_NAME, + "--jar", URL, "--className", IdentityFunction.class.getName(), }); @@ -348,21 +334,19 @@ public void testCreateFunctionWithoutBasicArguments() throws Exception { assertEquals("public", creater.getFunctionConfig().getTenant()); assertEquals("default", creater.getFunctionConfig().getNamespace()); - assertEquals(inputTopicName, creater.getInputs()); - assertEquals(outputTopicName, creater.getOutput()); + assertEquals(INPUT_TOPIC_NAME, creater.getInputs()); + assertEquals(OUTPUT_TOPIC_NAME, creater.getOutput()); verify(functions, times(1)).createFunctionWithUrl(any(FunctionConfig.class), anyString()); } @Test public void testCreateFunctionWithTopicPatterns() throws Exception { - String fnName = TEST_NAME + "-function"; String topicPatterns = "persistent://tenant/ns/topicPattern*"; - String outputTopicName = TEST_NAME + "-output-topic"; cmd.run(new String[] { "create", - "--name", fnName, + "--name", FN_NAME, "--topicsPattern", topicPatterns, - "--output", outputTopicName, + "--output", OUTPUT_TOPIC_NAME, "--jar", JAR_NAME, "--tenant", "sample", "--namespace", "ns1", @@ -370,9 +354,9 @@ public void testCreateFunctionWithTopicPatterns() throws Exception { }); CreateFunction creater = cmd.getCreater(); - assertEquals(fnName, creater.getFunctionName()); + assertEquals(FN_NAME, creater.getFunctionName()); assertEquals(topicPatterns, creater.getTopicsPattern()); - assertEquals(outputTopicName, creater.getOutput()); + assertEquals(OUTPUT_TOPIC_NAME, creater.getOutput()); verify(functions, times(1)).createFunction(any(FunctionConfig.class), anyString()); @@ -380,8 +364,6 @@ public void testCreateFunctionWithTopicPatterns() throws Exception { @Test public void testCreateUsingFullyQualifiedFunctionName() throws Exception { - String inputTopicName = TEST_NAME + "-input-topic"; - String outputTopicName = TEST_NAME + "-output-topic"; String tenant = "sample"; String namespace = "ns1"; String functionName = "func"; @@ -389,8 +371,8 @@ public void testCreateUsingFullyQualifiedFunctionName() throws Exception { cmd.run(new String[] { "create", - "--inputs", inputTopicName, - "--output", outputTopicName, + "--inputs", INPUT_TOPIC_NAME, + "--output", OUTPUT_TOPIC_NAME, "--fqfn", fqfn, "--jar", JAR_NAME, "--className", DummyFunction.class.getName(), @@ -405,10 +387,9 @@ public void testCreateUsingFullyQualifiedFunctionName() throws Exception { @Test public void testCreateWithoutOutputTopicWithSkipFlag() throws Exception { - String inputTopicName = TEST_NAME + "-input-topic"; cmd.run(new String[] { "create", - "--inputs", inputTopicName, + "--inputs", INPUT_TOPIC_NAME, "--jar", JAR_NAME, "--tenant", "sample", "--namespace", "ns1", @@ -428,10 +409,9 @@ public void testCreateWithoutOutputTopic() { ConsoleOutputCapturer consoleOutputCapturer = new ConsoleOutputCapturer(); consoleOutputCapturer.start(); - String inputTopicName = TEST_NAME + "-input-topic"; cmd.run(new String[] { "create", - "--inputs", inputTopicName, + "--inputs", INPUT_TOPIC_NAME, "--jar", JAR_NAME, "--tenant", "sample", "--namespace", "ns1", @@ -446,57 +426,45 @@ public void testCreateWithoutOutputTopic() { @Test public void testGetFunction() throws Exception { - String tenant = TEST_NAME + "-tenant"; - String namespace = TEST_NAME + "-namespace"; - String fnName = TEST_NAME + "-function"; - cmd.run(new String[] { "get", - "--name", fnName, - "--tenant", tenant, - "--namespace", namespace + "--name", FN_NAME, + "--tenant", TENANT, + "--namespace", NAMESPACE }); GetFunction getter = cmd.getGetter(); - assertEquals(fnName, getter.getFunctionName()); - assertEquals(tenant, getter.getTenant()); - assertEquals(namespace, getter.getNamespace()); + assertEquals(FN_NAME, getter.getFunctionName()); + assertEquals(TENANT, getter.getTenant()); + assertEquals(NAMESPACE, getter.getNamespace()); - verify(functions, times(1)).getFunction(eq(tenant), eq(namespace), eq(fnName)); + verify(functions, times(1)).getFunction(eq(TENANT), eq(NAMESPACE), eq(FN_NAME)); } @Test public void testDeleteFunction() throws Exception { - String tenant = TEST_NAME + "-tenant"; - String namespace = TEST_NAME + "-namespace"; - String fnName = TEST_NAME + "-function"; - cmd.run(new String[] { "delete", - "--name", fnName, - "--tenant", tenant, - "--namespace", namespace + "--name", FN_NAME, + "--tenant", TENANT, + "--namespace", NAMESPACE }); DeleteFunction deleter = cmd.getDeleter(); - assertEquals(fnName, deleter.getFunctionName()); - assertEquals(tenant, deleter.getTenant()); - assertEquals(namespace, deleter.getNamespace()); + assertEquals(FN_NAME, deleter.getFunctionName()); + assertEquals(TENANT, deleter.getTenant()); + assertEquals(NAMESPACE, deleter.getNamespace()); - verify(functions, times(1)).deleteFunction(eq(tenant), eq(namespace), eq(fnName)); + verify(functions, times(1)).deleteFunction(eq(TENANT), eq(NAMESPACE), eq(FN_NAME)); } @Test public void testUpdateFunction() throws Exception { - String fnName = TEST_NAME + "-function"; - String inputTopicName = TEST_NAME + "-input-topic"; - String outputTopicName = TEST_NAME + "-output-topic"; - cmd.run(new String[] { "update", - "--name", fnName, - "--inputs", inputTopicName, - "--output", outputTopicName, + "--name", FN_NAME, + "--inputs", INPUT_TOPIC_NAME, + "--output", OUTPUT_TOPIC_NAME, "--jar", JAR_NAME, "--tenant", "sample", "--namespace", "ns1", @@ -504,95 +472,77 @@ public void testUpdateFunction() throws Exception { }); UpdateFunction updater = cmd.getUpdater(); - assertEquals(fnName, updater.getFunctionName()); - assertEquals(inputTopicName, updater.getInputs()); - assertEquals(outputTopicName, updater.getOutput()); + assertEquals(FN_NAME, updater.getFunctionName()); + assertEquals(INPUT_TOPIC_NAME, updater.getInputs()); + assertEquals(OUTPUT_TOPIC_NAME, updater.getOutput()); verify(functions, times(1)).updateFunction(any(FunctionConfig.class), anyString(), eq(new UpdateOptions())); } @Test public void testListFunctions() throws Exception { - String tenant = TEST_NAME + "-tenant"; - String namespace = TEST_NAME + "-namespace"; - cmd.run(new String[] { "list", - "--tenant", tenant, - "--namespace", namespace + "--tenant", TENANT, + "--namespace", NAMESPACE }); ListFunctions lister = cmd.getLister(); - assertEquals(tenant, lister.getTenant()); - assertEquals(namespace, lister.getNamespace()); + assertEquals(TENANT, lister.getTenant()); + assertEquals(NAMESPACE, lister.getNamespace()); - verify(functions, times(1)).getFunctions(eq(tenant), eq(namespace)); + verify(functions, times(1)).getFunctions(eq(TENANT), eq(NAMESPACE)); } @Test public void testStateGetter() throws Exception { - String tenant = TEST_NAME + "-tenant"; - String namespace = TEST_NAME + "-namespace"; - String fnName = TEST_NAME + "-function"; String key = TEST_NAME + "-key"; cmd.run(new String[] { "querystate", - "--tenant", tenant, - "--namespace", namespace, - "--name", fnName, + "--tenant", TENANT, + "--namespace", NAMESPACE, + "--name", FN_NAME, "--key", key }); StateGetter stateGetter = cmd.getStateGetter(); - assertEquals(tenant, stateGetter.getTenant()); - assertEquals(namespace, stateGetter.getNamespace()); - assertEquals(fnName, stateGetter.getFunctionName()); + assertEquals(TENANT, stateGetter.getTenant()); + assertEquals(NAMESPACE, stateGetter.getNamespace()); + assertEquals(FN_NAME, stateGetter.getFunctionName()); - verify(functions, times(1)).getFunctionState(eq(tenant), eq(namespace), eq(fnName), eq(key)); + verify(functions, times(1)).getFunctionState(eq(TENANT), eq(NAMESPACE), eq(FN_NAME), eq(key)); } @Test public void testStateGetterWithoutKey() throws Exception { - String tenant = TEST_NAME + "-tenant"; - String namespace = TEST_NAME + "-namespace"; - String fnName = TEST_NAME + "-function"; ConsoleOutputCapturer consoleOutputCapturer = new ConsoleOutputCapturer(); consoleOutputCapturer.start(); cmd.run(new String[] { "querystate", - "--tenant", tenant, - "--namespace", namespace, - "--name", fnName, + "--tenant", TENANT, + "--namespace", NAMESPACE, + "--name", FN_NAME, }); consoleOutputCapturer.stop(); String output = consoleOutputCapturer.getStderr(); assertTrue(output.replace("\n", "").contains("State key needs to be specified")); StateGetter stateGetter = cmd.getStateGetter(); - assertEquals(tenant, stateGetter.getTenant()); - assertEquals(namespace, stateGetter.getNamespace()); - assertEquals(fnName, stateGetter.getFunctionName()); + assertEquals(TENANT, stateGetter.getTenant()); + assertEquals(NAMESPACE, stateGetter.getNamespace()); + assertEquals(FN_NAME, stateGetter.getFunctionName()); verify(functions, times(0)).getFunctionState(any(), any(), any(), any()); } - private static final String fnName = TEST_NAME + "-function"; - private static final String inputTopicName = TEST_NAME + "-input-topic"; - private static final String outputTopicName = TEST_NAME + "-output-topic"; - @Test public void testCreateFunctionWithCpu() throws Exception { - String fnName = TEST_NAME + "-function"; - String inputTopicName = TEST_NAME + "-input-topic"; - String outputTopicName = TEST_NAME + "-output-topic"; - - final String url = "file:" + JAR_NAME; cmd.run(new String[] { "create", - "--name", fnName, - "--inputs", inputTopicName, - "--output", outputTopicName, - "--jar", url, + "--name", FN_NAME, + "--inputs", INPUT_TOPIC_NAME, + "--output", OUTPUT_TOPIC_NAME, + "--jar", URL, "--tenant", "sample", "--namespace", "ns1", "--className", DummyFunction.class.getName(), @@ -601,29 +551,24 @@ public void testCreateFunctionWithCpu() throws Exception { CreateFunction creater = cmd.getCreater(); - assertEquals(fnName, creater.getFunctionName()); - assertEquals(inputTopicName, creater.getInputs()); - assertEquals(outputTopicName, creater.getOutput()); + assertEquals(FN_NAME, creater.getFunctionName()); + assertEquals(INPUT_TOPIC_NAME, creater.getInputs()); + assertEquals(OUTPUT_TOPIC_NAME, creater.getOutput()); assertEquals(creater.getFunctionConfig().getResources().getCpu(), 5.0); // Disk/Ram should be default - assertEquals(creater.getFunctionConfig().getResources().getRam(), new Long(1073741824l)); - assertEquals(creater.getFunctionConfig().getResources().getDisk(), new Long(10737418240l)); + assertEquals(creater.getFunctionConfig().getResources().getRam(), Long.valueOf(1073741824L)); + assertEquals(creater.getFunctionConfig().getResources().getDisk(), Long.valueOf(10737418240L)); verify(functions, times(1)).createFunctionWithUrl(any(FunctionConfig.class), anyString()); } @Test public void testCreateFunctionWithRam() throws Exception { - String fnName = TEST_NAME + "-function"; - String inputTopicName = TEST_NAME + "-input-topic"; - String outputTopicName = TEST_NAME + "-output-topic"; - - final String url = "file:" + JAR_NAME; cmd.run(new String[] { "create", - "--name", fnName, - "--inputs", inputTopicName, - "--output", outputTopicName, - "--jar", url, + "--name", FN_NAME, + "--inputs", INPUT_TOPIC_NAME, + "--output", OUTPUT_TOPIC_NAME, + "--jar", URL, "--tenant", "sample", "--namespace", "ns1", "--className", DummyFunction.class.getName(), @@ -632,29 +577,24 @@ public void testCreateFunctionWithRam() throws Exception { CreateFunction creater = cmd.getCreater(); - assertEquals(fnName, creater.getFunctionName()); - assertEquals(inputTopicName, creater.getInputs()); - assertEquals(outputTopicName, creater.getOutput()); - assertEquals(creater.getFunctionConfig().getResources().getRam(), new Long(5656565656l)); + assertEquals(FN_NAME, creater.getFunctionName()); + assertEquals(INPUT_TOPIC_NAME, creater.getInputs()); + assertEquals(OUTPUT_TOPIC_NAME, creater.getOutput()); + assertEquals(creater.getFunctionConfig().getResources().getRam(), Long.valueOf(5656565656L)); // cpu/disk should be default assertEquals(creater.getFunctionConfig().getResources().getCpu(), 1.0); - assertEquals(creater.getFunctionConfig().getResources().getDisk(), new Long(10737418240l)); + assertEquals(creater.getFunctionConfig().getResources().getDisk(), Long.valueOf(10737418240L)); verify(functions, times(1)).createFunctionWithUrl(any(FunctionConfig.class), anyString()); } @Test public void testCreateFunctionWithDisk() throws Exception { - String fnName = TEST_NAME + "-function"; - String inputTopicName = TEST_NAME + "-input-topic"; - String outputTopicName = TEST_NAME + "-output-topic"; - - final String url = "file:" + JAR_NAME; cmd.run(new String[] { "create", - "--name", fnName, - "--inputs", inputTopicName, - "--output", outputTopicName, - "--jar", url, + "--name", FN_NAME, + "--inputs", INPUT_TOPIC_NAME, + "--output", OUTPUT_TOPIC_NAME, + "--jar", URL, "--tenant", "sample", "--namespace", "ns1", "--className", DummyFunction.class.getName(), @@ -663,12 +603,12 @@ public void testCreateFunctionWithDisk() throws Exception { CreateFunction creater = cmd.getCreater(); - assertEquals(fnName, creater.getFunctionName()); - assertEquals(inputTopicName, creater.getInputs()); - assertEquals(outputTopicName, creater.getOutput()); - assertEquals(creater.getFunctionConfig().getResources().getDisk(), new Long(8080808080808080l)); + assertEquals(FN_NAME, creater.getFunctionName()); + assertEquals(INPUT_TOPIC_NAME, creater.getInputs()); + assertEquals(OUTPUT_TOPIC_NAME, creater.getOutput()); + assertEquals(creater.getFunctionConfig().getResources().getDisk(), Long.valueOf(8080808080808080L)); // cpu/Ram should be default - assertEquals(creater.getFunctionConfig().getResources().getRam(), new Long(1073741824l)); + assertEquals(creater.getFunctionConfig().getResources().getRam(), Long.valueOf(1073741824L)); assertEquals(creater.getFunctionConfig().getResources().getCpu(), 1.0); verify(functions, times(1)).createFunctionWithUrl(any(FunctionConfig.class), anyString()); } @@ -676,17 +616,12 @@ public void testCreateFunctionWithDisk() throws Exception { @Test public void testUpdateFunctionWithCpu() throws Exception { - String fnName = TEST_NAME + "-function"; - String inputTopicName = TEST_NAME + "-input-topic"; - String outputTopicName = TEST_NAME + "-output-topic"; - - final String url = "file:" + JAR_NAME; cmd.run(new String[] { "update", - "--name", fnName, - "--inputs", inputTopicName, - "--output", outputTopicName, - "--jar", url, + "--name", FN_NAME, + "--inputs", INPUT_TOPIC_NAME, + "--output", OUTPUT_TOPIC_NAME, + "--jar", URL, "--tenant", "sample", "--namespace", "ns1", "--className", DummyFunction.class.getName(), @@ -695,29 +630,24 @@ public void testUpdateFunctionWithCpu() throws Exception { UpdateFunction updater = cmd.getUpdater(); - assertEquals(fnName, updater.getFunctionName()); - assertEquals(inputTopicName, updater.getInputs()); - assertEquals(outputTopicName, updater.getOutput()); + assertEquals(FN_NAME, updater.getFunctionName()); + assertEquals(INPUT_TOPIC_NAME, updater.getInputs()); + assertEquals(OUTPUT_TOPIC_NAME, updater.getOutput()); assertEquals(updater.getFunctionConfig().getResources().getCpu(), 5.0); // Disk/Ram should be default - assertEquals(updater.getFunctionConfig().getResources().getRam(), new Long(1073741824l)); - assertEquals(updater.getFunctionConfig().getResources().getDisk(), new Long(10737418240l)); + assertEquals(updater.getFunctionConfig().getResources().getRam(), Long.valueOf(1073741824L)); + assertEquals(updater.getFunctionConfig().getResources().getDisk(), Long.valueOf(10737418240L)); verify(functions, times(1)).updateFunctionWithUrl(any(FunctionConfig.class), anyString(), eq(new UpdateOptions())); } @Test public void testUpdateFunctionWithRam() throws Exception { - String fnName = TEST_NAME + "-function"; - String inputTopicName = TEST_NAME + "-input-topic"; - String outputTopicName = TEST_NAME + "-output-topic"; - - final String url = "file:" + JAR_NAME; cmd.run(new String[] { "update", - "--name", fnName, - "--inputs", inputTopicName, - "--output", outputTopicName, - "--jar", url, + "--name", FN_NAME, + "--inputs", INPUT_TOPIC_NAME, + "--output", OUTPUT_TOPIC_NAME, + "--jar", URL, "--tenant", "sample", "--namespace", "ns1", "--className", DummyFunction.class.getName(), @@ -726,29 +656,24 @@ public void testUpdateFunctionWithRam() throws Exception { UpdateFunction updater = cmd.getUpdater(); - assertEquals(fnName, updater.getFunctionName()); - assertEquals(inputTopicName, updater.getInputs()); - assertEquals(outputTopicName, updater.getOutput()); - assertEquals(updater.getFunctionConfig().getResources().getRam(), new Long(5656565656l)); + assertEquals(FN_NAME, updater.getFunctionName()); + assertEquals(INPUT_TOPIC_NAME, updater.getInputs()); + assertEquals(OUTPUT_TOPIC_NAME, updater.getOutput()); + assertEquals(updater.getFunctionConfig().getResources().getRam(), Long.valueOf(5656565656L)); // cpu/disk should be default assertEquals(updater.getFunctionConfig().getResources().getCpu(), 1.0); - assertEquals(updater.getFunctionConfig().getResources().getDisk(), new Long(10737418240l)); + assertEquals(updater.getFunctionConfig().getResources().getDisk(), Long.valueOf(10737418240L)); verify(functions, times(1)).updateFunctionWithUrl(any(FunctionConfig.class), anyString(), eq(new UpdateOptions())); } @Test public void testUpdateFunctionWithDisk() throws Exception { - String fnName = TEST_NAME + "-function"; - String inputTopicName = TEST_NAME + "-input-topic"; - String outputTopicName = TEST_NAME + "-output-topic"; - - final String url = "file:" + JAR_NAME; cmd.run(new String[] { "update", - "--name", fnName, - "--inputs", inputTopicName, - "--output", outputTopicName, - "--jar", url, + "--name", FN_NAME, + "--inputs", INPUT_TOPIC_NAME, + "--output", OUTPUT_TOPIC_NAME, + "--jar", URL, "--tenant", "sample", "--namespace", "ns1", "--className", DummyFunction.class.getName(), @@ -757,29 +682,24 @@ public void testUpdateFunctionWithDisk() throws Exception { UpdateFunction updater = cmd.getUpdater(); - assertEquals(fnName, updater.getFunctionName()); - assertEquals(inputTopicName, updater.getInputs()); - assertEquals(outputTopicName, updater.getOutput()); - assertEquals(updater.getFunctionConfig().getResources().getDisk(), new Long(8080808080808080l)); + assertEquals(FN_NAME, updater.getFunctionName()); + assertEquals(INPUT_TOPIC_NAME, updater.getInputs()); + assertEquals(OUTPUT_TOPIC_NAME, updater.getOutput()); + assertEquals(updater.getFunctionConfig().getResources().getDisk(), Long.valueOf(8080808080808080L)); // cpu/Ram should be default - assertEquals(updater.getFunctionConfig().getResources().getRam(), new Long(1073741824l)); + assertEquals(updater.getFunctionConfig().getResources().getRam(), Long.valueOf(1073741824L)); assertEquals(updater.getFunctionConfig().getResources().getCpu(), 1.0); verify(functions, times(1)).updateFunctionWithUrl(any(FunctionConfig.class), anyString(), eq(new UpdateOptions())); } @Test public void testUpdateAuthData() throws Exception { - String fnName = TEST_NAME + "-function"; - String inputTopicName = TEST_NAME + "-input-topic"; - String outputTopicName = TEST_NAME + "-output-topic"; - - final String url = "file:" + JAR_NAME; cmd.run(new String[] { "update", - "--name", fnName, - "--inputs", inputTopicName, - "--output", outputTopicName, - "--jar", url, + "--name", FN_NAME, + "--inputs", INPUT_TOPIC_NAME, + "--output", OUTPUT_TOPIC_NAME, + "--jar", URL, "--tenant", "sample", "--namespace", "ns1", "--className", DummyFunction.class.getName(), @@ -789,12 +709,12 @@ public void testUpdateAuthData() throws Exception { UpdateFunction updater = cmd.getUpdater(); - assertEquals(fnName, updater.getFunctionName()); - assertEquals(inputTopicName, updater.getInputs()); - assertEquals(outputTopicName, updater.getOutput()); - assertEquals(updater.getFunctionConfig().getResources().getDisk(), new Long(8080808080808080l)); + assertEquals(FN_NAME, updater.getFunctionName()); + assertEquals(INPUT_TOPIC_NAME, updater.getInputs()); + assertEquals(OUTPUT_TOPIC_NAME, updater.getOutput()); + assertEquals(updater.getFunctionConfig().getResources().getDisk(), Long.valueOf(8080808080808080L)); // cpu/Ram should be default - assertEquals(updater.getFunctionConfig().getResources().getRam(), new Long(1073741824l)); + assertEquals(updater.getFunctionConfig().getResources().getRam(), Long.valueOf(1073741824L)); assertEquals(updater.getFunctionConfig().getResources().getCpu(), 1.0); UpdateOptions updateOptions = new UpdateOptions(); updateOptions.setUpdateAuthData(true); diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/utils/IOUtilsTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/utils/IOUtilsTest.java index 934b94182f5dc..726def1f0469d 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/utils/IOUtilsTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/utils/IOUtilsTest.java @@ -29,7 +29,6 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import org.apache.pulsar.admin.cli.utils.IOUtils; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -145,4 +144,4 @@ public void test6() { // expected } } -} \ No newline at end of file +} diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java index 21b71eb042755..29b7ca949500c 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.client.cli; -import java.net.MalformedURLException; import java.util.List; import java.util.Properties; import java.util.UUID; @@ -51,7 +50,7 @@ public void cleanup() throws Exception { } @Test - public void testInitialzation() throws MalformedURLException, InterruptedException, ExecutionException, PulsarAdminException { + public void testInitialzation() throws InterruptedException, ExecutionException, PulsarAdminException { Properties properties = new Properties(); properties.setProperty("serviceUrl", brokerUrl.toString()); From ffe9a92f59d9c82bc325a2f5b603ea9669b9179b Mon Sep 17 00:00:00 2001 From: Ilya Mashchenko Date: Sat, 7 Mar 2020 02:02:44 +0300 Subject: [PATCH 61/74] Fix Topic metrics documentation (#6495) ### Motivation *Explain here the context, and why you're making that change. What is the problem you're trying to solve.* Motivation is to have correct reference-metrics documentation. ### Modifications *Describe the modifications you've done.* There is an error in the `Topic metrics` section `pulsar_producers_count` => `pulsar_in_messages_total` --- site2/docs/reference-metrics.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site2/docs/reference-metrics.md b/site2/docs/reference-metrics.md index d3e21bd896b7e..f62934451e552 100644 --- a/site2/docs/reference-metrics.md +++ b/site2/docs/reference-metrics.md @@ -174,7 +174,7 @@ All the topic metrics are labelled with the following labels: | pulsar_storage_write_latency_le_* | Histogram | The entry rate of a topic that the storage write latency is smaller with a given threshold.
Available thresholds:

  • pulsar_storage_write_latency_le_0_5: <= 0.5ms
  • pulsar_storage_write_latency_le_1: <= 1ms
  • pulsar_storage_write_latency_le_5: <= 5ms
  • pulsar_storage_write_latency_le_10: <= 10ms
  • pulsar_storage_write_latency_le_20: <= 20ms
  • pulsar_storage_write_latency_le_50: <= 50ms
  • pulsar_storage_write_latency_le_100: <= 100ms
  • pulsar_storage_write_latency_le_200: <= 200ms
  • pulsar_storage_write_latency_le_1000: <= 1s
  • pulsar_storage_write_latency_le_overflow: > 1s
| | pulsar_entry_size_le_* | Histogram | The entry rate of a topic that the entry size is smaller with a given threshold.
Available thresholds:
  • pulsar_entry_size_le_128: <= 128 bytes
  • pulsar_entry_size_le_512: <= 512 bytes
  • pulsar_entry_size_le_1_kb: <= 1 KB
  • pulsar_entry_size_le_2_kb: <= 2 KB
  • pulsar_entry_size_le_4_kb: <= 4 KB
  • pulsar_entry_size_le_16_kb: <= 16 KB
  • pulsar_entry_size_le_100_kb: <= 100 KB
  • pulsar_entry_size_le_1_mb: <= 1 MB
  • pulsar_entry_size_le_overflow: > 1 MB
| | pulsar_in_bytes_total | Counter | The total number of bytes received for this topic | -| pulsar_producers_count | Counter | The total number of messages received for this topic | +| pulsar_in_messages_total | Counter | The total number of messages received for this topic | #### Replication metrics From f9ada1076e52574b24126948d366d19f020b448c Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 6 Mar 2020 15:03:51 -0800 Subject: [PATCH 62/74] [pulsar-client] remove duplicate cnx method (#6490) ### Motivation Remove duplicate `cnx()` method for `producer` --- .../apache/pulsar/client/api/ProducerCreationTest.java | 2 +- .../apache/pulsar/client/impl/ConnectionHandler.java | 8 ++------ .../org/apache/pulsar/client/impl/ConsumerImpl.java | 2 +- .../org/apache/pulsar/client/impl/ProducerImpl.java | 10 +++++----- 4 files changed, 9 insertions(+), 13 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerCreationTest.java index b13b61d4d74b2..8aed4f978f4be 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ProducerCreationTest.java @@ -86,7 +86,7 @@ public void testGeneratedNameProducerReconnect(TopicDomain domain) throws Pulsar //simulate create producer timeout. Thread.sleep(3000); - producer.getConnectionHandler().connectionClosed(producer.getConnectionHandler().getClientCnx()); + producer.getConnectionHandler().connectionClosed(producer.getConnectionHandler().cnx()); Assert.assertFalse(producer.isConnected()); Thread.sleep(3000); Assert.assertEquals(producer.getConnectionHandler().getEpoch(), 1); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java index ef58da6930a59..8eab9ab469e30 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConnectionHandler.java @@ -125,7 +125,8 @@ protected void resetBackoff() { backoff.reset(); } - protected ClientCnx cnx() { + @VisibleForTesting + public ClientCnx cnx() { return CLIENT_CNX_UPDATER.get(this); } @@ -133,11 +134,6 @@ protected boolean isRetriableError(PulsarClientException e) { return e instanceof PulsarClientException.LookupException; } - @VisibleForTesting - public ClientCnx getClientCnx() { - return CLIENT_CNX_UPDATER.get(this); - } - protected void setClientCnx(ClientCnx clientCnx) { CLIENT_CNX_UPDATER.set(this, clientCnx); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index ce4ee1ebc8ea8..14d0aeb14daee 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -1842,7 +1842,7 @@ void connectionClosed(ClientCnx cnx) { @VisibleForTesting public ClientCnx getClientCnx() { - return this.connectionHandler.getClientCnx(); + return this.connectionHandler.cnx(); } void setClientCnx(ClientCnx clientCnx) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index 7e5cb3ba79036..fc16fe869415b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -572,8 +572,8 @@ protected ByteBufPair sendMessage(long producerId, long lowestSequenceId, long h } private ChecksumType getChecksumType() { - if (connectionHandler.getClientCnx() == null - || connectionHandler.getClientCnx().getRemoteEndpointProtocolVersion() >= brokerChecksumSupportedVersion()) { + if (connectionHandler.cnx() == null + || connectionHandler.cnx().getRemoteEndpointProtocolVersion() >= brokerChecksumSupportedVersion()) { return ChecksumType.Crc32c; } else { return ChecksumType.None; @@ -782,11 +782,11 @@ public CompletableFuture closeAsync() { @Override public boolean isConnected() { - return connectionHandler.getClientCnx() != null && (getState() == State.Ready); + return connectionHandler.cnx() != null && (getState() == State.Ready); } public boolean isWritable() { - ClientCnx cnx = connectionHandler.getClientCnx(); + ClientCnx cnx = connectionHandler.cnx(); return cnx != null && cnx.channel().isWritable(); } @@ -1605,7 +1605,7 @@ void connectionClosed(ClientCnx cnx) { } ClientCnx getClientCnx() { - return this.connectionHandler.getClientCnx(); + return this.connectionHandler.cnx(); } void setClientCnx(ClientCnx clientCnx) { From 329e2310069b61e25ce3f87f2828fab78f97187a Mon Sep 17 00:00:00 2001 From: Addison Higham Date: Fri, 6 Mar 2020 16:06:11 -0700 Subject: [PATCH 63/74] [proxy] Fix proxy routing to functions worker (#6486) ### Motivation Currently, the proxy only works to proxy v1/v2 functions routes to the function worker. ### Modifications This changes this code to proxy all routes for the function worker when those routes match. At the moment this is still a static list of prefixes, but in the future it may be possible to have this list of prefixes be dynamically fetched from the REST routes. ### Verifying this change - added some tests to ensure the routing works as expected --- .../proxy/server/AdminProxyHandler.java | 26 +++++++- .../server/FunctionWorkerRoutingTest.java | 66 +++++++++++++++++++ 2 files changed, 89 insertions(+), 3 deletions(-) create mode 100644 pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/FunctionWorkerRoutingTest.java diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java index ca44c8fe0c08f..56a933bb5f5dd 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/AdminProxyHandler.java @@ -26,9 +26,12 @@ import java.net.URI; import java.nio.ByteBuffer; import java.security.cert.X509Certificate; +import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.Iterator; import java.util.Objects; +import java.util.Set; import java.util.concurrent.Executor; import javax.net.ssl.SSLContext; @@ -60,6 +63,21 @@ class AdminProxyHandler extends ProxyServlet { private static final Logger LOG = LoggerFactory.getLogger(AdminProxyHandler.class); + private static final Set functionRoutes = new HashSet<>(Arrays.asList( + "/admin/v3/function", + "/admin/v2/function", + "/admin/function", + "/admin/v3/source", + "/admin/v2/source", + "/admin/source", + "/admin/v3/sink", + "/admin/v2/sink", + "/admin/sink", + "/admin/v2/worker", + "/admin/v2/worker-stats", + "/admin/worker", + "/admin/worker-stats" + )); private final ProxyConfiguration config; private final BrokerDiscoveryProvider discoveryProvider; @@ -260,9 +278,11 @@ protected String rewriteTarget(HttpServletRequest request) { boolean isFunctionsRestRequest = false; String requestUri = request.getRequestURI(); - if (requestUri.startsWith("/admin/v2/functions") - || requestUri.startsWith("/admin/functions")) { - isFunctionsRestRequest = true; + for (String routePrefix: functionRoutes) { + if (requestUri.startsWith(routePrefix)) { + isFunctionsRestRequest = true; + break; + } } if (isFunctionsRestRequest && !isBlank(functionWorkerWebServiceUrl)) { diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/FunctionWorkerRoutingTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/FunctionWorkerRoutingTest.java new file mode 100644 index 0000000000000..b5d89cce6647a --- /dev/null +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/FunctionWorkerRoutingTest.java @@ -0,0 +1,66 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.proxy.server; + +import org.testng.Assert; +import org.testng.annotations.Test; + +import javax.servlet.http.HttpServletRequest; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class FunctionWorkerRoutingTest { + + @Test + public void testFunctionWorkerRedirect() throws Exception { + String functionWorkerUrl = "http://function"; + String brokerUrl = "http://broker"; + + ProxyConfiguration proxyConfig = new ProxyConfiguration(); + proxyConfig.setBrokerWebServiceURL(brokerUrl); + proxyConfig.setFunctionWorkerWebServiceURL(functionWorkerUrl); + + BrokerDiscoveryProvider discoveryProvider = mock(BrokerDiscoveryProvider.class); + AdminProxyHandler handler = new AdminProxyHandler(proxyConfig, discoveryProvider); + + String funcUrl = handler.rewriteTarget(buildRequest("/admin/v3/functions/test/test")); + Assert.assertEquals(funcUrl, String.format("%s/admin/v3/functions/%s/%s", + functionWorkerUrl, "test", "test")); + + String sourceUrl = handler.rewriteTarget(buildRequest("/admin/v3/sources/test/test")); + Assert.assertEquals(sourceUrl, String.format("%s/admin/v3/sources/%s/%s", + functionWorkerUrl, "test", "test")); + + String sinkUrl = handler.rewriteTarget(buildRequest("/admin/v3/sinks/test/test")); + Assert.assertEquals(sinkUrl, String.format("%s/admin/v3/sinks/%s/%s", + functionWorkerUrl, "test", "test")); + + String tenantUrl = handler.rewriteTarget(buildRequest("/admin/v2/tenants/test")); + Assert.assertEquals(tenantUrl, String.format("%s/admin/v2/tenants/%s", + brokerUrl, "test")); + } + + static HttpServletRequest buildRequest(String url) { + HttpServletRequest mockReq = mock(HttpServletRequest.class); + when(mockReq.getRequestURI()).thenReturn(url); + return mockReq; + } + +} From 47ca8e64d8f36d89af587a7b9a8865622701e109 Mon Sep 17 00:00:00 2001 From: lipenghui Date: Sat, 7 Mar 2020 07:07:12 +0800 Subject: [PATCH 64/74] Fix some async method problems at PersistentTopicsBase. (#6483) --- .../pulsar/broker/admin/AdminResource.java | 20 +- .../admin/impl/PersistentTopicsBase.java | 1047 ++++++++++------- 2 files changed, 610 insertions(+), 457 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 a21698298488b..98d370f26084f 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 @@ -571,16 +571,24 @@ protected ZooKeeperChildrenCache failureDomainListCache() { protected CompletableFuture getPartitionedTopicMetadataAsync( TopicName topicName, boolean authoritative, boolean checkAllowAutoCreation) { - validateClusterOwnership(topicName.getCluster()); - // validates global-namespace contains local/peer cluster: if peer/local cluster present then lookup can - // serve/redirect request else fail partitioned-metadata-request so, client fails while creating - // producer/consumer - validateGlobalNamespaceOwnership(topicName.getNamespaceObject()); + try { + validateClusterOwnership(topicName.getCluster()); + // validates global-namespace contains local/peer cluster: if peer/local cluster present then lookup can + // serve/redirect request else fail partitioned-metadata-request so, client fails while creating + // producer/consumer + validateGlobalNamespaceOwnership(topicName.getNamespaceObject()); + } catch (Exception e) { + return FutureUtil.failedFuture(e); + } try { checkConnect(topicName); } catch (WebApplicationException e) { - validateAdminAccessForTenant(topicName.getTenant()); + try { + validateAdminAccessForTenant(topicName.getTenant()); + } catch (Exception ex) { + return FutureUtil.failedFuture(ex); + } } catch (Exception e) { // unknown error marked as internal server error log.warn("Unexpected error while authorizing lookup. topic={}, role={}. Error: {}", topicName, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index f2a95d1879a84..d74017c4e6a88 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -539,50 +539,58 @@ protected PartitionedTopicMetadata internalGetPartitionedMetadata(boolean author } protected void internalDeletePartitionedTopic(AsyncResponse asyncResponse, boolean authoritative, boolean force) { - validateAdminAccessForTenant(topicName.getTenant()); - + try { + validateAdminAccessForTenant(topicName.getTenant()); + } catch (Exception e) { + log.error("[{}] Failed to delete partitioned topic {}", clientAppId(), topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + return; + } final CompletableFuture future = new CompletableFuture<>(); - - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - final int numPartitions = partitionMetadata.partitions; - if (numPartitions > 0) { - final AtomicInteger count = new AtomicInteger(numPartitions); - for (int i = 0; i < numPartitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - pulsar().getAdminClient().topics().deleteAsync(topicNamePartition.toString(), force) - .whenComplete((r, ex) -> { - if (ex != null) { - if (ex instanceof NotFoundException) { - // if the sub-topic is not found, the client might not have called create - // producer or it might have been deleted earlier, so we ignore the 404 error. - // For all other exception, we fail the delete partition method even if a single - // partition is failed to be deleted - if (log.isDebugEnabled()) { - log.debug("[{}] Partition not found: {}", clientAppId(), - topicNamePartition); + getPartitionedTopicMetadataAsync(topicName, authoritative, false).thenAccept(partitionMeta -> { + final int numPartitions = partitionMeta.partitions; + if (numPartitions > 0) { + final AtomicInteger count = new AtomicInteger(numPartitions); + for (int i = 0; i < numPartitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + pulsar().getAdminClient().topics().deleteAsync(topicNamePartition.toString(), force) + .whenComplete((r, ex) -> { + if (ex != null) { + if (ex instanceof NotFoundException) { + // if the sub-topic is not found, the client might not have called create + // producer or it might have been deleted earlier, so we ignore the 404 error. + // For all other exception, we fail the delete partition method even if a single + // partition is failed to be deleted + if (log.isDebugEnabled()) { + log.debug("[{}] Partition not found: {}", clientAppId(), + topicNamePartition); + } + } else { + log.error("[{}] Failed to delete partition {}", clientAppId(), + topicNamePartition, ex); + future.completeExceptionally(ex); + return; } } else { - log.error("[{}] Failed to delete partition {}", clientAppId(), - topicNamePartition, ex); - future.completeExceptionally(ex); - return; + log.info("[{}] Deleted partition {}", clientAppId(), topicNamePartition); } - } else { - log.info("[{}] Deleted partition {}", clientAppId(), topicNamePartition); - } - if (count.decrementAndGet() == 0) { - future.complete(null); - } - }); - } catch (Exception e) { - log.error("[{}] Failed to delete partition {}", clientAppId(), topicNamePartition, e); - future.completeExceptionally(e); + if (count.decrementAndGet() == 0) { + future.complete(null); + } + }); + } catch (Exception e) { + log.error("[{}] Failed to delete partition {}", clientAppId(), topicNamePartition, e); + future.completeExceptionally(e); + } } + } else { + future.complete(null); } - } else { - future.complete(null); - } + }).exceptionally(ex -> { + future.completeExceptionally(ex); + return null; + }); future.whenComplete((r, ex) -> { if (ex != null) { @@ -593,6 +601,9 @@ protected void internalDeletePartitionedTopic(AsyncResponse asyncResponse, boole } else if (ex instanceof PulsarAdminException) { asyncResponse.resume(new RestException((PulsarAdminException) ex)); return; + } else if (ex instanceof WebApplicationException) { + asyncResponse.resume(ex); + return; } else { asyncResponse.resume(new RestException(ex)); return; @@ -602,30 +613,48 @@ protected void internalDeletePartitionedTopic(AsyncResponse asyncResponse, boole // Only tries to delete the znode for partitioned topic when all its partitions are successfully deleted String path = path(PARTITIONED_TOPIC_PATH_ZNODE, namespaceName.toString(), domain(), topicName.getEncodedLocalName()); - try { - globalZk().delete(path, -1); - globalZkCache().invalidate(path); - // Sync data to all quorums and the observers - zkSync(path); - log.info("[{}] Deleted partitioned topic {}", clientAppId(), topicName); - asyncResponse.resume(Response.noContent().build()); - } catch (KeeperException.NoNodeException nne) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, "Partitioned topic does not exist")); - } catch (KeeperException.BadVersionException e) { - log.warn("[{}] Failed to delete partitioned topic {}: concurrent modification", clientAppId(), - topicName); - asyncResponse.resume(new RestException(Status.CONFLICT, "Concurrent modification")); - } catch (Exception e) { - log.error("[{}] Failed to delete partitioned topic {}", clientAppId(), topicName, e); - asyncResponse.resume(new RestException(e)); - } + + globalZk().delete(path, -1, (rc, s, o) -> { + if (KeeperException.Code.OK.intValue() == rc) { + try { + globalZkCache().invalidate(path); + globalZk().sync(path, (rc2, s2, ctx) -> { + if (KeeperException.Code.OK.intValue() == rc2) { + log.info("[{}] Deleted partitioned topic {}", clientAppId(), topicName); + asyncResponse.resume(Response.noContent().build()); + } else { + log.error("[{}] Failed to delete partitioned topic {}", clientAppId(), topicName, KeeperException.create(KeeperException.Code.get(rc2))); + asyncResponse.resume(new RestException(KeeperException.create(KeeperException.Code.get(rc2)))); + } + }, null); + } catch (Exception e) { + log.error("Failed to delete partitioned topic.", e); + asyncResponse.resume(new RestException(e)); + } + } else if (KeeperException.Code.NONODE.intValue() == rc) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Partitioned topic does not exist")); + } else if (KeeperException.Code.BADVERSION.intValue() == rc) { + log.warn("[{}] Failed to delete partitioned topic {}: concurrent modification", clientAppId(), + topicName); + asyncResponse.resume(new RestException(Status.CONFLICT, "Concurrent modification")); + } else { + log.error("[{}] Failed to delete partitioned topic {}", clientAppId(), topicName, KeeperException.create(KeeperException.Code.get(rc))); + asyncResponse.resume(new RestException(KeeperException.create(KeeperException.Code.get(rc)))); + } + }, null); }); } protected void internalUnloadTopic(AsyncResponse asyncResponse, boolean authoritative) { log.info("[{}] Unloading topic {}", clientAppId(), topicName); - if (topicName.isGlobal()) { - validateGlobalNamespaceOwnership(namespaceName); + try { + if (topicName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + } catch (Exception e) { + log.error("[{}] Failed to unload topic {}", clientAppId(), topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + return; } // If the topic name is a partition name, no need to get partition topic metadata again if (topicName.isPartitioned()) { @@ -651,32 +680,43 @@ protected void internalUnloadTopic(AsyncResponse asyncResponse, boolean authorit Throwable th = exception.getCause(); if (th instanceof NotFoundException) { asyncResponse.resume(new RestException(Status.NOT_FOUND, th.getMessage())); + } else if (th instanceof WebApplicationException) { + asyncResponse.resume(th); } else { log.error("[{}] Failed to unload topic {}", clientAppId(), topicName, exception); asyncResponse.resume(new RestException(exception)); } - return null; + } else { + asyncResponse.resume(Response.noContent().build()); } - - asyncResponse.resume(Response.noContent().build()); return null; }); } else { internalUnloadNonPartitionedTopic(asyncResponse, authoritative); } }).exceptionally(t -> { - Throwable th = t.getCause(); - asyncResponse.resume(new RestException(th)); + log.error("[{}] Failed to unload topic {}", clientAppId(), topicName, t); + if (t instanceof WebApplicationException) { + asyncResponse.resume(t); + } else { + asyncResponse.resume(new RestException(t)); + } return null; }); } } private void internalUnloadNonPartitionedTopic(AsyncResponse asyncResponse, boolean authoritative) { - validateAdminAccessForTenant(topicName.getTenant()); - validateTopicOwnership(topicName, authoritative); - - Topic topic = getTopicReference(topicName); + Topic topic; + try { + validateAdminAccessForTenant(topicName.getTenant()); + validateTopicOwnership(topicName, authoritative); + topic = getTopicReference(topicName); + } catch (Exception e) { + log.error("[{}] Failed to unload topic {}", clientAppId(), topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + return; + } topic.close(false).whenComplete((r, ex) -> { if (ex != null) { log.error("[{}] Failed to unload topic {}, {}", clientAppId(), topicName, ex.getMessage(), ex); @@ -726,66 +766,73 @@ protected void internalDeleteTopic(boolean authoritative) { protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean authoritative) { if (topicName.isGlobal()) { - validateGlobalNamespaceOwnership(namespaceName); + try { + validateGlobalNamespaceOwnership(namespaceName); + } catch (Exception e) { + log.error("[{}] Failed to get subscriptions for topic {}", clientAppId(), topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + return; + } } // If the topic name is a partition name, no need to get partition topic metadata again if (topicName.isPartitioned()) { internalGetSubscriptionsForNonPartitionedTopic(asyncResponse, authoritative); } else { - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - if (partitionMetadata.partitions > 0) { - try { - // get the subscriptions only from the 1st partition since all the other partitions will have the same - // subscriptions - pulsar().getAdminClient().topics().getSubscriptionsAsync(topicName.getPartition(0).toString()) - .whenComplete((r, ex) -> { - if (ex != null) { - log.warn("[{}] Failed to get list of subscriptions for {}: {}", clientAppId(), - topicName, ex.getMessage()); - - if (ex instanceof PulsarAdminException) { - PulsarAdminException pae = (PulsarAdminException) ex; - if (pae.getStatusCode() == Status.NOT_FOUND.getStatusCode()) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, - "Internal topics have not been generated yet")); - return; - } else { - asyncResponse.resume(new RestException(pae)); - return; + getPartitionedTopicMetadataAsync(topicName, authoritative, false).thenAccept(partitionMetadata -> { + if (partitionMetadata.partitions > 0) { + try { + // get the subscriptions only from the 1st partition since all the other partitions will have the same + // subscriptions + pulsar().getAdminClient().topics().getSubscriptionsAsync(topicName.getPartition(0).toString()) + .whenComplete((r, ex) -> { + if (ex != null) { + log.warn("[{}] Failed to get list of subscriptions for {}: {}", clientAppId(), + topicName, ex.getMessage()); + + if (ex instanceof PulsarAdminException) { + PulsarAdminException pae = (PulsarAdminException) ex; + if (pae.getStatusCode() == Status.NOT_FOUND.getStatusCode()) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, + "Internal topics have not been generated yet")); + return; + } else { + asyncResponse.resume(new RestException(pae)); + return; + } + } else { + asyncResponse.resume(new RestException(ex)); + return; + } } - } else { - asyncResponse.resume(new RestException(ex)); - return; - } - } - final List subscriptions = Lists.newArrayList(); - subscriptions.addAll(r); - asyncResponse.resume(subscriptions); - return; - }); - } catch (Exception e) { - log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), topicName, e); - asyncResponse.resume(e); - return; + final List subscriptions = Lists.newArrayList(); + subscriptions.addAll(r); + asyncResponse.resume(subscriptions); + }); + } catch (Exception e) { + log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), topicName, e); + asyncResponse.resume(e); + } + } else { + internalGetSubscriptionsForNonPartitionedTopic(asyncResponse, authoritative); } - } else { - internalGetSubscriptionsForNonPartitionedTopic(asyncResponse, authoritative); - } + }).exceptionally(ex -> { + log.error("[{}] Failed to get subscriptions for topic {}", clientAppId(), topicName, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); } } private void internalGetSubscriptionsForNonPartitionedTopic(AsyncResponse asyncResponse, boolean authoritative) { - validateAdminOperationOnTopic(authoritative); - Topic topic = getTopicReference(topicName); try { + validateAdminOperationOnTopic(authoritative); + Topic topic = getTopicReference(topicName); final List subscriptions = Lists.newArrayList(); topic.getSubscriptions().forEach((subName, sub) -> subscriptions.add(subName)); asyncResponse.resume(subscriptions); - return; } catch (Exception e) { log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), topicName, e); - asyncResponse.resume(new RestException(e)); - return; + resumeAsyncResponseExceptionally(asyncResponse, e); } } @@ -810,11 +857,18 @@ protected PersistentTopicInternalStats internalGetInternalStats(boolean authorit } protected void internalGetManagedLedgerInfo(AsyncResponse asyncResponse) { - validateAdminAccessForTenant(topicName.getTenant()); - if (topicName.isGlobal()) { - validateGlobalNamespaceOwnership(namespaceName); + String managedLedger; + try { + validateAdminAccessForTenant(topicName.getTenant()); + if (topicName.isGlobal()) { + validateGlobalNamespaceOwnership(namespaceName); + } + managedLedger = topicName.getPersistenceNamingEncoding(); + } catch (Exception e) { + log.error("[{}] Failed to get managed info for {}", clientAppId(), topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + return; } - String managedLedger = topicName.getPersistenceNamingEncoding(); pulsar().getManagedLedgerFactory().asyncGetManagedLedgerInfo(managedLedger, new ManagedLedgerInfoCallback() { @Override public void getInfoComplete(ManagedLedgerInfo info, Object ctx) { @@ -832,252 +886,304 @@ public void getInfoFailed(ManagedLedgerException exception, Object ctx) { protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean authoritative, boolean perPartition, boolean getPreciseBacklog) { - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - if (partitionMetadata.partitions == 0) { - throw new RestException(Status.NOT_FOUND, "Partitioned Topic not found"); - } if (topicName.isGlobal()) { - validateGlobalNamespaceOwnership(namespaceName); - } - PartitionedTopicStats stats = new PartitionedTopicStats(partitionMetadata); - - List> topicStatsFutureList = Lists.newArrayList(); - for (int i = 0; i < partitionMetadata.partitions; i++) { try { - topicStatsFutureList - .add(pulsar().getAdminClient().topics().getStatsAsync((topicName.getPartition(i).toString()), getPreciseBacklog)); - } catch (PulsarServerException e) { - asyncResponse.resume(new RestException(e)); + validateGlobalNamespaceOwnership(namespaceName); + } catch (Exception e) { + log.error("[{}] Failed to get partitioned stats for {}", clientAppId(), topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); return; } } + getPartitionedTopicMetadataAsync(topicName, authoritative, false).thenAccept(partitionMetadata -> { + if (partitionMetadata.partitions == 0) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Partitioned Topic not found")); + return; + } + PartitionedTopicStats stats = new PartitionedTopicStats(partitionMetadata); + List> topicStatsFutureList = Lists.newArrayList(); + for (int i = 0; i < partitionMetadata.partitions; i++) { + try { + topicStatsFutureList + .add(pulsar().getAdminClient().topics().getStatsAsync((topicName.getPartition(i).toString()), getPreciseBacklog)); + } catch (PulsarServerException e) { + asyncResponse.resume(new RestException(e)); + return; + } + } - FutureUtil.waitForAll(topicStatsFutureList).handle((result, exception) -> { - CompletableFuture statFuture = null; - for (int i = 0; i < topicStatsFutureList.size(); i++) { - statFuture = topicStatsFutureList.get(i); - if (statFuture.isDone() && !statFuture.isCompletedExceptionally()) { - try { - stats.add(statFuture.get()); - if (perPartition) { - stats.partitions.put(topicName.getPartition(i).toString(), statFuture.get()); + FutureUtil.waitForAll(topicStatsFutureList).handle((result, exception) -> { + CompletableFuture statFuture = null; + for (int i = 0; i < topicStatsFutureList.size(); i++) { + statFuture = topicStatsFutureList.get(i); + if (statFuture.isDone() && !statFuture.isCompletedExceptionally()) { + try { + stats.add(statFuture.get()); + if (perPartition) { + stats.partitions.put(topicName.getPartition(i).toString(), statFuture.get()); + } + } catch (Exception e) { + asyncResponse.resume(new RestException(e)); + return null; } - } catch (Exception e) { - asyncResponse.resume(new RestException(e)); - return null; } } - } - if (perPartition && stats.partitions.isEmpty()) { - String path = ZkAdminPaths.partitionedTopicPath(topicName); - try { - boolean zkPathExists = zkPathExists(path); - if (zkPathExists) { - stats.partitions.put(topicName.toString(), new TopicStats()); - } else { - asyncResponse.resume( - new RestException(Status.NOT_FOUND, "Internal topics have not been generated yet")); + if (perPartition && stats.partitions.isEmpty()) { + String path = ZkAdminPaths.partitionedTopicPath(topicName); + try { + boolean zkPathExists = zkPathExists(path); + if (zkPathExists) { + stats.partitions.put(topicName.toString(), new TopicStats()); + } else { + asyncResponse.resume( + new RestException(Status.NOT_FOUND, "Internal topics have not been generated yet")); + return null; + } + } catch (KeeperException | InterruptedException e) { + asyncResponse.resume(new RestException(e)); return null; } - } catch (KeeperException | InterruptedException e) { - asyncResponse.resume(new RestException(e)); - return null; } - } - asyncResponse.resume(stats); + asyncResponse.resume(stats); + return null; + }); + }).exceptionally(ex -> { + log.error("[{}] Failed to get partitioned stats for {}", clientAppId(), topicName, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); return null; }); } protected void internalGetPartitionedStatsInternal(AsyncResponse asyncResponse, boolean authoritative) { - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - if (partitionMetadata.partitions == 0) { - throw new RestException(Status.NOT_FOUND, "Partitioned Topic not found"); - } if (topicName.isGlobal()) { - validateGlobalNamespaceOwnership(namespaceName); - } - PartitionedTopicInternalStats stats = new PartitionedTopicInternalStats(partitionMetadata); - - List> topicStatsFutureList = Lists.newArrayList(); - for (int i = 0; i < partitionMetadata.partitions; i++) { try { - topicStatsFutureList.add(pulsar().getAdminClient().topics() - .getInternalStatsAsync((topicName.getPartition(i).toString()))); - } catch (PulsarServerException e) { - asyncResponse.resume(new RestException(e)); + validateGlobalNamespaceOwnership(namespaceName); + } catch (Exception e) { + log.error("[{}] Failed to get partitioned internal stats for {}", clientAppId(), topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); return; } } + getPartitionedTopicMetadataAsync(topicName, authoritative, false).thenAccept(partitionMetadata -> { + if (partitionMetadata.partitions == 0) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Partitioned Topic not found")); + return; + } - FutureUtil.waitForAll(topicStatsFutureList).handle((result, exception) -> { - CompletableFuture statFuture = null; - for (int i = 0; i < topicStatsFutureList.size(); i++) { - statFuture = topicStatsFutureList.get(i); - if (statFuture.isDone() && !statFuture.isCompletedExceptionally()) { - try { - stats.partitions.put(topicName.getPartition(i).toString(), statFuture.get()); - } catch (Exception e) { - asyncResponse.resume(new RestException(e)); - return null; - } + PartitionedTopicInternalStats stats = new PartitionedTopicInternalStats(partitionMetadata); + + List> topicStatsFutureList = Lists.newArrayList(); + for (int i = 0; i < partitionMetadata.partitions; i++) { + try { + topicStatsFutureList.add(pulsar().getAdminClient().topics() + .getInternalStatsAsync((topicName.getPartition(i).toString()))); + } catch (PulsarServerException e) { + asyncResponse.resume(new RestException(e)); + return; } } - asyncResponse.resume(!stats.partitions.isEmpty() ? stats - : new RestException(Status.NOT_FOUND, "Internal topics have not been generated yet")); + + FutureUtil.waitForAll(topicStatsFutureList).handle((result, exception) -> { + CompletableFuture statFuture = null; + for (int i = 0; i < topicStatsFutureList.size(); i++) { + statFuture = topicStatsFutureList.get(i); + if (statFuture.isDone() && !statFuture.isCompletedExceptionally()) { + try { + stats.partitions.put(topicName.getPartition(i).toString(), statFuture.get()); + } catch (Exception e) { + asyncResponse.resume(new RestException(e)); + return null; + } + } + } + asyncResponse.resume(!stats.partitions.isEmpty() ? stats + : new RestException(Status.NOT_FOUND, "Internal topics have not been generated yet")); + return null; + }); + }).exceptionally(ex -> { + log.error("[{}] Failed to get partitioned internal stats for {}", clientAppId(), topicName, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); return null; }); } protected void internalDeleteSubscription(AsyncResponse asyncResponse, String subName, boolean authoritative) { if (topicName.isGlobal()) { - validateGlobalNamespaceOwnership(namespaceName); + try { + validateGlobalNamespaceOwnership(namespaceName); + } catch (Exception e) { + log.error("[{}] Failed to delete subscription {} from topic {}", clientAppId(), subName, topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + return; + } } // If the topic name is a partition name, no need to get partition topic metadata again if (topicName.isPartitioned()) { internalDeleteSubscriptionForNonPartitionedTopic(asyncResponse, subName, authoritative); } else { - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - if (partitionMetadata.partitions > 0) { - final List> futures = Lists.newArrayList(); + getPartitionedTopicMetadataAsync(topicName, authoritative, false).thenAccept(partitionMetadata -> { + if (partitionMetadata.partitions > 0) { + final List> futures = Lists.newArrayList(); - for (int i = 0; i < partitionMetadata.partitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - futures.add(pulsar().getAdminClient().topics() - .deleteSubscriptionAsync(topicNamePartition.toString(), subName)); - } catch (Exception e) { - log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicNamePartition, subName, - e); - asyncResponse.resume(new RestException(e)); - return; + for (int i = 0; i < partitionMetadata.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + futures.add(pulsar().getAdminClient().topics() + .deleteSubscriptionAsync(topicNamePartition.toString(), subName)); + } catch (Exception e) { + log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicNamePartition, subName, + e); + asyncResponse.resume(new RestException(e)); + return; + } } - } - FutureUtil.waitForAll(futures).handle((result, exception) -> { - if (exception != null) { - Throwable t = exception.getCause(); - if (t instanceof NotFoundException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); - return null; - } else if (t instanceof PreconditionFailedException) { - asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, - "Subscription has active connected consumers")); - return null; - } else { - log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicName, subName, t); - asyncResponse.resume(new RestException(t)); - return null; + FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable t = exception.getCause(); + if (t instanceof NotFoundException) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); + return null; + } else if (t instanceof PreconditionFailedException) { + asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, + "Subscription has active connected consumers")); + return null; + } else { + log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicName, subName, t); + asyncResponse.resume(new RestException(t)); + return null; + } } - } - asyncResponse.resume(Response.noContent().build()); - return null; - }); - } else { - internalDeleteSubscriptionForNonPartitionedTopic(asyncResponse, subName, authoritative); - } + asyncResponse.resume(Response.noContent().build()); + return null; + }); + } else { + internalDeleteSubscriptionForNonPartitionedTopic(asyncResponse, subName, authoritative); + } + }).exceptionally(ex -> { + log.error("[{}] Failed to delete subscription {} from topic {}", clientAppId(), subName, topicName, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); } } private void internalDeleteSubscriptionForNonPartitionedTopic(AsyncResponse asyncResponse, String subName, boolean authoritative) { - validateAdminAccessForSubscriber(subName, authoritative); - Topic topic = getTopicReference(topicName); try { + validateAdminAccessForSubscriber(subName, authoritative); + Topic topic = getTopicReference(topicName); Subscription sub = topic.getSubscription(subName); - checkNotNull(sub); + if (sub == null) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); + return; + } sub.delete().get(); log.info("[{}][{}] Deleted subscription {}", clientAppId(), topicName, subName); asyncResponse.resume(Response.noContent().build()); } catch (Exception e) { - Throwable t = e.getCause(); - if (e instanceof NullPointerException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); - } else if (t instanceof SubscriptionBusyException) { + log.error("[{}] Failed to delete subscription {} from topic {}", clientAppId(), subName, topicName, e); + if (e.getCause() instanceof SubscriptionBusyException) { asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, "Subscription has active connected consumers")); + } else if (e instanceof WebApplicationException) { + asyncResponse.resume(e); } else { log.error("[{}] Failed to delete subscription {} {}", clientAppId(), topicName, subName, e); - asyncResponse.resume(new RestException(t)); + asyncResponse.resume(new RestException(e)); } } } protected void internalSkipAllMessages(AsyncResponse asyncResponse, String subName, boolean authoritative) { if (topicName.isGlobal()) { - validateGlobalNamespaceOwnership(namespaceName); + try { + validateGlobalNamespaceOwnership(namespaceName); + } catch (Exception e) { + log.error("[{}] Failed to skip all messages for subscription {} on topic {}", clientAppId(), subName, topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + return; + } } // If the topic name is a partition name, no need to get partition topic metadata again if (topicName.isPartitioned()) { internalSkipAllMessagesForNonPartitionedTopic(asyncResponse, subName, authoritative); } else { - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - if (partitionMetadata.partitions > 0) { - final List> futures = Lists.newArrayList(); + getPartitionedTopicMetadataAsync(topicName, authoritative, false).thenAccept(partitionMetadata -> { + if (partitionMetadata.partitions > 0) { + final List> futures = Lists.newArrayList(); - for (int i = 0; i < partitionMetadata.partitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - futures.add(pulsar().getAdminClient().topics().skipAllMessagesAsync(topicNamePartition.toString(), - subName)); - } catch (Exception e) { - log.error("[{}] Failed to skip all messages {} {}", clientAppId(), topicNamePartition, subName, e); - asyncResponse.resume(new RestException(e)); - return; + for (int i = 0; i < partitionMetadata.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + futures.add(pulsar().getAdminClient().topics().skipAllMessagesAsync(topicNamePartition.toString(), + subName)); + } catch (Exception e) { + log.error("[{}] Failed to skip all messages {} {}", clientAppId(), topicNamePartition, subName, e); + asyncResponse.resume(new RestException(e)); + return; + } } - } - FutureUtil.waitForAll(futures).handle((result, exception) -> { - if (exception != null) { - Throwable t = exception.getCause(); - if (t instanceof NotFoundException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); - return null; - } else { - log.error("[{}] Failed to skip all messages {} {}", clientAppId(), topicName, subName, t); - asyncResponse.resume(new RestException(t)); - return null; + FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable t = exception.getCause(); + if (t instanceof NotFoundException) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); + return null; + } else { + log.error("[{}] Failed to skip all messages {} {}", clientAppId(), topicName, subName, t); + asyncResponse.resume(new RestException(t)); + return null; + } } - } - asyncResponse.resume(Response.noContent().build()); - return null; - }); - } else { - internalSkipAllMessagesForNonPartitionedTopic(asyncResponse, subName, authoritative); - } + asyncResponse.resume(Response.noContent().build()); + return null; + }); + } else { + internalSkipAllMessagesForNonPartitionedTopic(asyncResponse, subName, authoritative); + } + }).exceptionally(ex -> { + log.error("[{}] Failed to skip all messages for subscription {} on topic {}", clientAppId(), subName, topicName, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); } } private void internalSkipAllMessagesForNonPartitionedTopic(AsyncResponse asyncResponse, String subName, boolean authoritative) { - validateAdminAccessForSubscriber(subName, authoritative); - PersistentTopic topic = (PersistentTopic) getTopicReference(topicName); - BiConsumer biConsumer = (v, ex) -> { - if (ex != null) { - asyncResponse.resume(new RestException(ex)); - log.error("[{}] Failed to skip all messages {} {}", clientAppId(), topicName, subName, ex); - } else { - asyncResponse.resume(Response.noContent().build()); - log.info("[{}] Cleared backlog on {} {}", clientAppId(), topicName, subName); - } - }; try { + validateAdminAccessForSubscriber(subName, authoritative); + PersistentTopic topic = (PersistentTopic) getTopicReference(topicName); + BiConsumer biConsumer = (v, ex) -> { + if (ex != null) { + asyncResponse.resume(new RestException(ex)); + log.error("[{}] Failed to skip all messages {} {}", clientAppId(), topicName, subName, ex); + } else { + asyncResponse.resume(Response.noContent().build()); + log.info("[{}] Cleared backlog on {} {}", clientAppId(), topicName, subName); + } + }; if (subName.startsWith(topic.getReplicatorPrefix())) { String remoteCluster = PersistentReplicator.getRemoteCluster(subName); PersistentReplicator repl = (PersistentReplicator) topic.getPersistentReplicator(remoteCluster); - checkNotNull(repl); + if (repl == null) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); + return; + } repl.clearBacklog().whenComplete(biConsumer); } else { PersistentSubscription sub = topic.getSubscription(subName); - checkNotNull(sub); + if (sub == null) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); + return; + } sub.clearBacklog().whenComplete(biConsumer); } } catch (Exception e) { - if (e instanceof NullPointerException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); - } else { - asyncResponse.resume(new RestException(e)); - } + log.error("[{}] Failed to skip all messages for subscription {} on topic {}", clientAppId(), subName, topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); } } @@ -1115,54 +1221,72 @@ protected void internalSkipMessages(String subName, int numMessages, boolean aut protected void internalExpireMessagesForAllSubscriptions(AsyncResponse asyncResponse, int expireTimeInSeconds, boolean authoritative) { if (topicName.isGlobal()) { - validateGlobalNamespaceOwnership(namespaceName); + try { + validateGlobalNamespaceOwnership(namespaceName); + } catch (Exception e) { + log.error("[{}] Failed to expire messages for all subscription on topic {}", clientAppId(), topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + return; + } } // If the topic name is a partition name, no need to get partition topic metadata again if (topicName.isPartitioned()) { internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(asyncResponse, expireTimeInSeconds, authoritative); } else { - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - if (partitionMetadata.partitions > 0) { - final List> futures = Lists.newArrayList(); + getPartitionedTopicMetadataAsync(topicName, authoritative, false).thenAccept(partitionMetadata -> { + if (partitionMetadata.partitions > 0) { + final List> futures = Lists.newArrayList(); - // expire messages for each partition topic - for (int i = 0; i < partitionMetadata.partitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - futures.add(pulsar().getAdminClient().topics().expireMessagesForAllSubscriptionsAsync( - topicNamePartition.toString(), expireTimeInSeconds)); - } catch (Exception e) { - log.error("[{}] Failed to expire messages up to {} on {}", clientAppId(), expireTimeInSeconds, - topicNamePartition, e); - asyncResponse.resume(new RestException(e)); - return; + // expire messages for each partition topic + for (int i = 0; i < partitionMetadata.partitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + futures.add(pulsar().getAdminClient().topics().expireMessagesForAllSubscriptionsAsync( + topicNamePartition.toString(), expireTimeInSeconds)); + } catch (Exception e) { + log.error("[{}] Failed to expire messages up to {} on {}", clientAppId(), expireTimeInSeconds, + topicNamePartition, e); + asyncResponse.resume(new RestException(e)); + return; + } } - } - FutureUtil.waitForAll(futures).handle((result, exception) -> { - if (exception != null) { - Throwable t = exception.getCause(); - log.error("[{}] Failed to expire messages up to {} on {}", clientAppId(), expireTimeInSeconds, - topicName, t); - asyncResponse.resume(new RestException(t)); - return null; - } + FutureUtil.waitForAll(futures).handle((result, exception) -> { + if (exception != null) { + Throwable t = exception.getCause(); + log.error("[{}] Failed to expire messages up to {} on {}", clientAppId(), expireTimeInSeconds, + topicName, t); + asyncResponse.resume(new RestException(t)); + return null; + } - asyncResponse.resume(Response.noContent().build()); - return null; - }); - } else { - internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(asyncResponse, expireTimeInSeconds, authoritative); - } + asyncResponse.resume(Response.noContent().build()); + return null; + }); + } else { + internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(asyncResponse, expireTimeInSeconds, authoritative); + } + }).exceptionally(ex -> { + log.error("[{}] Failed to expire messages for all subscription on topic {}", clientAppId(), topicName, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); } } private void internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(AsyncResponse asyncResponse, int expireTimeInSeconds, boolean authoritative) { // validate ownership and redirect if current broker is not owner - validateAdminOperationOnTopic(authoritative); + PersistentTopic topic; + try { + validateAdminOperationOnTopic(authoritative); - PersistentTopic topic = (PersistentTopic) getTopicReference(topicName); + topic = (PersistentTopic) getTopicReference(topicName); + } catch (Exception e) { + log.error("[{}] Failed to expire messages for all subscription on topic {}", clientAppId(), topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + return; + } final AtomicReference exception = new AtomicReference<>(); topic.getReplicators().forEach((subName, replicator) -> { @@ -1198,111 +1322,124 @@ private void internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(Asy protected void internalResetCursor(AsyncResponse asyncResponse, String subName, long timestamp, boolean authoritative) { if (topicName.isGlobal()) { - validateGlobalNamespaceOwnership(namespaceName); + try { + validateGlobalNamespaceOwnership(namespaceName); + } catch (Exception e) { + log.error("[{}] Failed to expire messages for all subscription on topic {}", clientAppId(), topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + return; + } } // If the topic name is a partition name, no need to get partition topic metadata again if (topicName.isPartitioned()) { internalResetCursorForNonPartitionedTopic(asyncResponse, subName, timestamp, authoritative); } else { - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - final int numPartitions = partitionMetadata.partitions; - if (numPartitions > 0) { - final CompletableFuture future = new CompletableFuture<>(); - final AtomicInteger count = new AtomicInteger(numPartitions); - final AtomicInteger failureCount = new AtomicInteger(0); - final AtomicReference partitionException = new AtomicReference<>(); - - for (int i = 0; i < numPartitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - pulsar().getAdminClient().topics() - .resetCursorAsync(topicNamePartition.toString(), subName, timestamp).handle((r, ex) -> { - if (ex != null) { - if (ex instanceof PreconditionFailedException) { - // throw the last exception if all partitions get this error - // any other exception on partition is reported back to user - failureCount.incrementAndGet(); - partitionException.set(ex); - } else { - log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", - clientAppId(), topicNamePartition, subName, timestamp, ex); - future.completeExceptionally(ex); - return null; + getPartitionedTopicMetadataAsync(topicName, authoritative, false).thenAccept(partitionMetadata -> { + final int numPartitions = partitionMetadata.partitions; + if (numPartitions > 0) { + final CompletableFuture future = new CompletableFuture<>(); + final AtomicInteger count = new AtomicInteger(numPartitions); + final AtomicInteger failureCount = new AtomicInteger(0); + final AtomicReference partitionException = new AtomicReference<>(); + + for (int i = 0; i < numPartitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + pulsar().getAdminClient().topics() + .resetCursorAsync(topicNamePartition.toString(), subName, timestamp).handle((r, ex) -> { + if (ex != null) { + if (ex instanceof PreconditionFailedException) { + // throw the last exception if all partitions get this error + // any other exception on partition is reported back to user + failureCount.incrementAndGet(); + partitionException.set(ex); + } else { + log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", + clientAppId(), topicNamePartition, subName, timestamp, ex); + future.completeExceptionally(ex); + return null; + } } - } - if (count.decrementAndGet() == 0) { - future.complete(null); - } + if (count.decrementAndGet() == 0) { + future.complete(null); + } - return null; - }); - } catch (Exception e) { - log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), - topicNamePartition, subName, timestamp, e); - future.completeExceptionally(e); + return null; + }); + } catch (Exception e) { + log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), + topicNamePartition, subName, timestamp, e); + future.completeExceptionally(e); + } } - } - future.whenComplete((r, ex) -> { - if (ex != null) { - if (ex instanceof PulsarAdminException) { - asyncResponse.resume(new RestException((PulsarAdminException) ex)); - return; - } else { - asyncResponse.resume(new RestException(ex)); - return; + future.whenComplete((r, ex) -> { + if (ex != null) { + if (ex instanceof PulsarAdminException) { + asyncResponse.resume(new RestException((PulsarAdminException) ex)); + return; + } else { + asyncResponse.resume(new RestException(ex)); + return; + } } - } - // report an error to user if unable to reset for all partitions - if (failureCount.get() == numPartitions) { - log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), topicName, - subName, timestamp, partitionException.get()); - asyncResponse.resume( - new RestException(Status.PRECONDITION_FAILED, partitionException.get().getMessage())); - return; - } else if (failureCount.get() > 0) { - log.warn("[{}] [{}] Partial errors for reset cursor on subscription {} to time {}", clientAppId(), - topicName, subName, timestamp, partitionException.get()); - } + // report an error to user if unable to reset for all partitions + if (failureCount.get() == numPartitions) { + log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), topicName, + subName, timestamp, partitionException.get()); + asyncResponse.resume( + new RestException(Status.PRECONDITION_FAILED, partitionException.get().getMessage())); + return; + } else if (failureCount.get() > 0) { + log.warn("[{}] [{}] Partial errors for reset cursor on subscription {} to time {}", clientAppId(), + topicName, subName, timestamp, partitionException.get()); + } - asyncResponse.resume(Response.noContent().build()); - }); - } else { - internalResetCursorForNonPartitionedTopic(asyncResponse, subName, timestamp, authoritative); - } + asyncResponse.resume(Response.noContent().build()); + }); + } else { + internalResetCursorForNonPartitionedTopic(asyncResponse, subName, timestamp, authoritative); + } + }).exceptionally(ex -> { + log.error("[{}] Failed to expire messages for all subscription on topic {}", clientAppId(), topicName, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); } } private void internalResetCursorForNonPartitionedTopic(AsyncResponse asyncResponse, String subName, long timestamp, boolean authoritative) { - validateAdminAccessForSubscriber(subName, authoritative); - log.info("[{}] [{}] Received reset cursor on subscription {} to time {}", clientAppId(), topicName, subName, - timestamp); - PersistentTopic topic = (PersistentTopic) getTopicReference(topicName); - if (topic == null) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, "Topic not found")); - return; - } try { + validateAdminAccessForSubscriber(subName, authoritative); + log.info("[{}] [{}] Received reset cursor on subscription {} to time {}", clientAppId(), topicName, subName, + timestamp); + PersistentTopic topic = (PersistentTopic) getTopicReference(topicName); + if (topic == null) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Topic not found")); + return; + } PersistentSubscription sub = topic.getSubscription(subName); - checkNotNull(sub); + if (sub == null) { + asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); + return; + } sub.resetCursor(timestamp).get(); log.info("[{}] [{}] Reset cursor on subscription {} to time {}", clientAppId(), topicName, subName, timestamp); asyncResponse.resume(Response.noContent().build()); } catch (Exception e) { - Throwable t = e.getCause(); log.warn("[{}] [{}] Failed to reset cursor on subscription {} to time {}", clientAppId(), topicName, subName, timestamp, e); - if (e instanceof NullPointerException) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, "Subscription not found")); - } else if (e instanceof NotAllowedException) { + if (e instanceof NotAllowedException) { asyncResponse.resume(new RestException(Status.METHOD_NOT_ALLOWED, e.getMessage())); - } else if (t instanceof SubscriptionInvalidCursorPosition) { + } else if (e instanceof SubscriptionInvalidCursorPosition) { asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, - "Unable to find position for timestamp specified -" + t.getMessage())); + "Unable to find position for timestamp specified -" + e.getMessage())); + } else if (e instanceof WebApplicationException) { + asyncResponse.resume(e); } else { asyncResponse.resume(new RestException(e)); } @@ -1312,7 +1449,13 @@ private void internalResetCursorForNonPartitionedTopic(AsyncResponse asyncRespon protected void internalCreateSubscription(AsyncResponse asyncResponse, String subscriptionName, MessageIdImpl messageId, boolean authoritative, boolean replicated) { if (topicName.isGlobal()) { - validateGlobalNamespaceOwnership(namespaceName); + try { + validateGlobalNamespaceOwnership(namespaceName); + } catch (Exception e) { + log.error("[{}] Failed to create subscription {} on topic {}", clientAppId(), subscriptionName, topicName, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + return; + } } final MessageIdImpl targetMessageId = messageId == null ? (MessageIdImpl) MessageId.earliest : messageId; log.info("[{}][{}] Creating subscription {} at message id {}", clientAppId(), topicName, subscriptionName, @@ -1321,86 +1464,88 @@ protected void internalCreateSubscription(AsyncResponse asyncResponse, String su if (topicName.isPartitioned()) { internalCreateSubscriptionForNonPartitionedTopic(asyncResponse, subscriptionName, targetMessageId, authoritative, replicated); } else { - PartitionedTopicMetadata partitionMetadata = getPartitionedTopicMetadata(topicName, authoritative, false); - final int numPartitions = partitionMetadata.partitions; - if (numPartitions > 0) { - final CompletableFuture future = new CompletableFuture<>(); - final AtomicInteger count = new AtomicInteger(numPartitions); - final AtomicInteger failureCount = new AtomicInteger(0); - final AtomicReference partitionException = new AtomicReference<>(); - - // Create the subscription on each partition - for (int i = 0; i < numPartitions; i++) { - TopicName topicNamePartition = topicName.getPartition(i); - try { - pulsar().getAdminClient().topics() - .createSubscriptionAsync(topicNamePartition.toString(), subscriptionName, targetMessageId) - .handle((r, ex) -> { - if (ex != null) { - // fail the operation on unknown exception or if all the partitioned failed due to - // subscription-already-exist - if (failureCount.incrementAndGet() == numPartitions - || !(ex instanceof PulsarAdminException.ConflictException)) { - partitionException.set(ex); - } - } + getPartitionedTopicMetadataAsync(topicName, authoritative, false).thenAccept(partitionMetadata -> { + final int numPartitions = partitionMetadata.partitions; + if (numPartitions > 0) { + final CompletableFuture future = new CompletableFuture<>(); + final AtomicInteger count = new AtomicInteger(numPartitions); + final AtomicInteger failureCount = new AtomicInteger(0); + final AtomicReference partitionException = new AtomicReference<>(); + + // Create the subscription on each partition + for (int i = 0; i < numPartitions; i++) { + TopicName topicNamePartition = topicName.getPartition(i); + try { + pulsar().getAdminClient().topics() + .createSubscriptionAsync(topicNamePartition.toString(), subscriptionName, targetMessageId) + .handle((r, ex) -> { + if (ex != null) { + // fail the operation on unknown exception or if all the partitioned failed due to + // subscription-already-exist + if (failureCount.incrementAndGet() == numPartitions + || !(ex instanceof PulsarAdminException.ConflictException)) { + partitionException.set(ex); + } + } - if (count.decrementAndGet() == 0) { - future.complete(null); - } + if (count.decrementAndGet() == 0) { + future.complete(null); + } - return null; - }); - } catch (Exception e) { - log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(), - topicNamePartition, subscriptionName, targetMessageId, e); - future.completeExceptionally(e); + return null; + }); + } catch (Exception e) { + log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(), + topicNamePartition, subscriptionName, targetMessageId, e); + future.completeExceptionally(e); + } } - } - future.whenComplete((r, ex) -> { - if (ex != null) { - if (ex instanceof PulsarAdminException) { - asyncResponse.resume(new RestException((PulsarAdminException) ex)); - return; - } else { - asyncResponse.resume(new RestException(ex)); - return; + future.whenComplete((r, ex) -> { + if (ex != null) { + if (ex instanceof PulsarAdminException) { + asyncResponse.resume(new RestException((PulsarAdminException) ex)); + return; + } else { + asyncResponse.resume(new RestException(ex)); + return; + } } - } - if (partitionException.get() != null) { - log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(), topicName, - subscriptionName, targetMessageId, partitionException.get()); - if (partitionException.get() instanceof PulsarAdminException) { - asyncResponse.resume(new RestException((PulsarAdminException) partitionException.get())); - return; - } else { - asyncResponse.resume(new RestException(partitionException.get())); - return; + if (partitionException.get() != null) { + log.warn("[{}] [{}] Failed to create subscription {} at message id {}", clientAppId(), topicName, + subscriptionName, targetMessageId, partitionException.get()); + if (partitionException.get() instanceof PulsarAdminException) { + asyncResponse.resume(new RestException((PulsarAdminException) partitionException.get())); + return; + } else { + asyncResponse.resume(new RestException(partitionException.get())); + return; + } } - } - asyncResponse.resume(Response.noContent().build()); - }); - } else { - internalCreateSubscriptionForNonPartitionedTopic(asyncResponse, subscriptionName, targetMessageId, authoritative, replicated); - } + asyncResponse.resume(Response.noContent().build()); + }); + } else { + internalCreateSubscriptionForNonPartitionedTopic(asyncResponse, subscriptionName, targetMessageId, authoritative, replicated); + } + }).exceptionally(ex -> { + log.error("[{}] Failed to create subscription {} on topic {}", clientAppId(), subscriptionName, topicName, ex); + resumeAsyncResponseExceptionally(asyncResponse, ex); + return null; + }); } } private void internalCreateSubscriptionForNonPartitionedTopic(AsyncResponse asyncResponse, String subscriptionName, MessageIdImpl targetMessageId, boolean authoritative, boolean replicated) { - validateAdminAccessForSubscriber(subscriptionName, authoritative); - - PersistentTopic topic = (PersistentTopic) getOrCreateTopic(topicName); - - if (topic.getSubscriptions().containsKey(subscriptionName)) { - asyncResponse.resume(new RestException(Status.CONFLICT, "Subscription already exists for topic")); - return; - } - try { + validateAdminAccessForSubscriber(subscriptionName, authoritative); + PersistentTopic topic = (PersistentTopic) getOrCreateTopic(topicName); + if (topic.getSubscriptions().containsKey(subscriptionName)) { + asyncResponse.resume(new RestException(Status.CONFLICT, "Subscription already exists for topic")); + return; + } PersistentSubscription subscription = (PersistentSubscription) topic .createSubscription(subscriptionName, InitialPosition.Latest, replicated).get(); // Mark the cursor as "inactive" as it was created without a real consumer connected @@ -1414,10 +1559,10 @@ private void internalCreateSubscriptionForNonPartitionedTopic(AsyncResponse asyn if (t instanceof SubscriptionInvalidCursorPosition) { asyncResponse.resume(new RestException(Status.PRECONDITION_FAILED, "Unable to find position for position specified: " + t.getMessage())); - return; + } else if (e instanceof WebApplicationException) { + asyncResponse.resume(e); } else { asyncResponse.resume(new RestException(e)); - return; } } From 36ea153c0ff4fc3e3f04de4a37b658daa9f116fa Mon Sep 17 00:00:00 2001 From: Sanjeev Kulkarni Date: Sat, 7 Mar 2020 18:10:03 -0800 Subject: [PATCH 65/74] Instead of always using admin access for topic, use read/write/admin access for topic (#6504) Co-authored-by: Sanjeev Kulkarni --- .../admin/impl/PersistentTopicsBase.java | 64 +++++++++++++++---- 1 file changed, 51 insertions(+), 13 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index d74017c4e6a88..685d1952535c6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -234,6 +234,44 @@ public void validateAdminOperationOnTopic(boolean authoritative) { validateTopicOwnership(topicName, authoritative); } + public void validateReadOperationOnTopic(boolean authoritative) { + validateTopicOwnership(topicName, authoritative); + try { + validateAdminAccessForTenant(topicName.getTenant()); + } catch (Exception e) { + if (log.isDebugEnabled()) { + log.debug("[{}] failed to validate admin access for {}", topicName, clientAppId()); + } + validateAdminAccessForSubscriber(""); + } + } + + public void validateWriteOperationOnTopic(boolean authoritative) { + validateTopicOwnership(topicName, authoritative); + try { + validateAdminAccessForTenant(topicName.getTenant()); + } catch (Exception e) { + if (log.isDebugEnabled()) { + log.debug("[{}] failed to validate admin access for {}", topicName, clientAppId()); + } + try { + if (!pulsar().getBrokerService().getAuthorizationService().canProduce(topicName, clientAppId(), + clientAuthData())) { + log.warn("[{}} Subscriber {} is not authorized to access api", topicName, clientAppId()); + throw new RestException(Status.UNAUTHORIZED, + String.format("Subscriber %s is not authorized to access this operation", clientAppId())); + } + } catch (RestException re) { + throw re; + } catch (Exception ex) { + // unknown error marked as internal server error + log.warn("Unexpected error while authorizing request. topic={}, role={}. Error: {}", topicName, + clientAppId(), e.getMessage(), ex); + throw new RestException(ex); + } + } + } + protected void validateAdminAccessForSubscriber(String subscriptionName, boolean authoritative) { validateTopicOwnership(topicName, authoritative); try { @@ -317,7 +355,7 @@ protected void internalGrantPermissionsOnTopic(String role, Set acti } protected void internalDeleteTopicForcefully(boolean authoritative) { - validateAdminOperationOnTopic(authoritative); + validateWriteOperationOnTopic(authoritative); Topic topic = getTopicReference(topicName); try { topic.deleteForcefully().get(); @@ -391,7 +429,7 @@ protected void internalRevokePermissionsOnTopic(String role) { } protected void internalCreateNonPartitionedTopic(boolean authoritative) { - validateAdminAccessForTenant(topicName.getTenant()); + validateWriteOperationOnTopic(authoritative); validateNonPartitionTopicName(topicName.getLocalName()); if (topicName.isGlobal()) { validateGlobalNamespaceOwnership(namespaceName); @@ -426,7 +464,7 @@ protected void internalCreateNonPartitionedTopic(boolean authoritative) { * @param numPartitions */ protected void internalUpdatePartitionedTopic(int numPartitions, boolean updateLocalTopicOnly) { - validateAdminAccessForTenant(topicName.getTenant()); + validateWriteOperationOnTopic(false); // Only do the validation if it's the first hop. if (!updateLocalTopicOnly) { validatePartitionTopicUpdate(topicName.getLocalName(), numPartitions); @@ -540,7 +578,7 @@ protected PartitionedTopicMetadata internalGetPartitionedMetadata(boolean author protected void internalDeletePartitionedTopic(AsyncResponse asyncResponse, boolean authoritative, boolean force) { try { - validateAdminAccessForTenant(topicName.getTenant()); + validateWriteOperationOnTopic(authoritative); } catch (Exception e) { log.error("[{}] Failed to delete partitioned topic {}", clientAppId(), topicName, e); resumeAsyncResponseExceptionally(asyncResponse, e); @@ -738,7 +776,7 @@ protected void internalDeleteTopic(boolean authoritative, boolean force) { } protected void internalDeleteTopic(boolean authoritative) { - validateAdminOperationOnTopic(authoritative); + validateWriteOperationOnTopic(authoritative); Topic topic = getTopicReference(topicName); // v2 topics have a global name so check if the topic is replicated. @@ -825,7 +863,7 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut private void internalGetSubscriptionsForNonPartitionedTopic(AsyncResponse asyncResponse, boolean authoritative) { try { - validateAdminOperationOnTopic(authoritative); + validateReadOperationOnTopic(authoritative); Topic topic = getTopicReference(topicName); final List subscriptions = Lists.newArrayList(); topic.getSubscriptions().forEach((subName, sub) -> subscriptions.add(subName)); @@ -1279,7 +1317,7 @@ private void internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(Asy // validate ownership and redirect if current broker is not owner PersistentTopic topic; try { - validateAdminOperationOnTopic(authoritative); + validateWriteOperationOnTopic(authoritative); topic = (PersistentTopic) getTopicReference(topicName); } catch (Exception e) { @@ -1744,7 +1782,7 @@ protected MessageId internalTerminate(boolean authoritative) { if (partitionMetadata.partitions > 0) { throw new RestException(Status.METHOD_NOT_ALLOWED, "Termination of a partitioned topic is not allowed"); } - validateAdminOperationOnTopic(authoritative); + validateWriteOperationOnTopic(authoritative); Topic topic = getTopicReference(topicName); try { return ((PersistentTopic) topic).terminate().get(); @@ -1867,7 +1905,7 @@ private void internalExpireMessagesForSinglePartition(String subName, int expire } protected void internalTriggerCompaction(boolean authoritative) { - validateAdminOperationOnTopic(authoritative); + validateWriteOperationOnTopic(authoritative); PersistentTopic topic = (PersistentTopic) getTopicReference(topicName); try { @@ -1880,13 +1918,13 @@ protected void internalTriggerCompaction(boolean authoritative) { } protected LongRunningProcessStatus internalCompactionStatus(boolean authoritative) { - validateAdminOperationOnTopic(authoritative); + validateReadOperationOnTopic(authoritative); PersistentTopic topic = (PersistentTopic) getTopicReference(topicName); return topic.compactionStatus(); } protected void internalTriggerOffload(boolean authoritative, MessageIdImpl messageId) { - validateAdminOperationOnTopic(authoritative); + validateWriteOperationOnTopic(authoritative); PersistentTopic topic = (PersistentTopic) getTopicReference(topicName); try { topic.triggerOffload(messageId); @@ -1899,7 +1937,7 @@ protected void internalTriggerOffload(boolean authoritative, MessageIdImpl messa } protected OffloadProcessStatus internalOffloadStatus(boolean authoritative) { - validateAdminOperationOnTopic(authoritative); + validateReadOperationOnTopic(authoritative); PersistentTopic topic = (PersistentTopic) getTopicReference(topicName); return topic.offloadStatus(); } @@ -2237,7 +2275,7 @@ private void validateNonPartitionTopicName(String topicName) { } protected MessageId internalGetLastMessageId(boolean authoritative) { - validateAdminOperationOnTopic(authoritative); + validateReadOperationOnTopic(authoritative); if (!(getTopicReference(topicName) instanceof PersistentTopic)) { log.error("[{}] Not supported operation of non-persistent topic {}", clientAppId(), topicName); From 29990000fa9d901426306ccd2e2f8be118390f98 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Sun, 8 Mar 2020 10:17:26 +0800 Subject: [PATCH 66/74] [Minor]Remove unused property from pom (#6500) This PR is a follow-up of #6494 --- tiered-storage/pom.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tiered-storage/pom.xml b/tiered-storage/pom.xml index 0612efab38f9a..d65d7eea21271 100644 --- a/tiered-storage/pom.xml +++ b/tiered-storage/pom.xml @@ -32,10 +32,6 @@ tiered-storage-parent Apache Pulsar :: Tiered Storage :: Parent - - ${project.version} - - jcloud file-system From 33b7383ca3ac284375a918604a7fb9e66e467e4a Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Sat, 7 Mar 2020 18:18:17 -0800 Subject: [PATCH 67/74] [pulsar-common] Remove duplicate RestException references (#6475) ### Motivation Right now, various pulsar-modules have duplicate `RestException` class and repo has multiple duplicate class. So, move `RestException` to common place and all modules should use the same Exception class to avoid duplicate classes. --- pulsar-common/pom.xml | 5 ++ .../pulsar/common/util}/RestException.java | 22 +++---- .../service/server/ServerManager.java | 2 +- .../service/web/DiscoveryServiceServlet.java | 1 + .../discovery/service/web/RestException.java | 57 ---------------- .../service/web/DiscoveryServiceWebTest.java | 8 +-- .../functions/worker/rest/RestUtils.java | 2 + .../worker/rest/api/ComponentImpl.java | 2 +- .../worker/rest/api/FunctionsImpl.java | 2 +- .../worker/rest/api/FunctionsImplV2.java | 2 +- .../functions/worker/rest/api/SinksImpl.java | 2 +- .../worker/rest/api/SourcesImpl.java | 2 +- .../functions/worker/rest/api/WorkerImpl.java | 2 +- .../api/v2/FunctionApiV2ResourceTest.java | 2 +- .../api/v3/FunctionApiV3ResourceTest.java | 2 +- .../rest/api/v3/SinkApiV3ResourceTest.java | 2 +- .../rest/api/v3/SourceApiV3ResourceTest.java | 2 +- .../pulsar/websocket/admin/RestException.java | 66 ------------------- .../admin/WebSocketProxyStatsBase.java | 1 + .../websocket/admin/WebSocketWebResource.java | 1 + .../admin/WebSocketWebResourceTest.java | 1 + 21 files changed, 34 insertions(+), 152 deletions(-) rename {pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest => pulsar-common/src/main/java/org/apache/pulsar/common/util}/RestException.java (80%) delete mode 100644 pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/web/RestException.java delete mode 100644 pulsar-websocket/src/main/java/org/apache/pulsar/websocket/admin/RestException.java diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index a9cb2e2fdbbd3..604941f0efab1 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -151,6 +151,11 @@ jackson-dataformat-yaml + + javax.ws.rs + javax.ws.rs-api + + diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/RestException.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/RestException.java similarity index 80% rename from pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/RestException.java rename to pulsar-common/src/main/java/org/apache/pulsar/common/util/RestException.java index 537ec1db4c3cb..3d986065901f4 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/RestException.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/RestException.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.functions.worker.rest; +package org.apache.pulsar.common.util; import java.io.PrintWriter; import java.io.StringWriter; @@ -26,7 +26,6 @@ import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; -import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.policies.data.ErrorData; /** @@ -34,7 +33,7 @@ */ @SuppressWarnings("serial") public class RestException extends WebApplicationException { - static String getExceptionData(Throwable t) { + public static String getExceptionData(Throwable t) { StringWriter writer = new StringWriter(); writer.append("\n --- An unexpected error occurred in the server ---\n\n"); if (t != null) { @@ -58,21 +57,16 @@ public RestException(Throwable t) { super(getResponse(t)); } - public RestException(PulsarAdminException cae) { - this(cae.getStatusCode(), cae.getHttpError()); - } - private static Response getResponse(Throwable t) { - if (t instanceof RestException - || t instanceof WebApplicationException) { + if (t instanceof WebApplicationException) { WebApplicationException e = (WebApplicationException) t; return e.getResponse(); } else { return Response - .status(Status.INTERNAL_SERVER_ERROR) - .entity(getExceptionData(t)) - .type(MediaType.TEXT_PLAIN) - .build(); + .status(Status.INTERNAL_SERVER_ERROR) + .entity(getExceptionData(t)) + .type(MediaType.TEXT_PLAIN) + .build(); } } -} \ No newline at end of file +} diff --git a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServerManager.java b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServerManager.java index 945074c55e7fd..fa4761c4f090d 100644 --- a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServerManager.java +++ b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/server/ServerManager.java @@ -28,8 +28,8 @@ import javax.servlet.Servlet; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.common.util.SecurityUtility; -import org.apache.pulsar.discovery.service.web.RestException; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; diff --git a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceServlet.java b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceServlet.java index 71c42b4503563..6324ca7ecc837 100644 --- a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceServlet.java +++ b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceServlet.java @@ -33,6 +33,7 @@ import javax.servlet.http.HttpServletResponse; import javax.ws.rs.core.Response.Status; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport; import org.apache.pulsar.zookeeper.ZooKeeperClientFactory; import org.apache.pulsar.zookeeper.ZookeeperClientFactoryImpl; diff --git a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/web/RestException.java b/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/web/RestException.java deleted file mode 100644 index 844b25ddeaad3..0000000000000 --- a/pulsar-discovery-service/src/main/java/org/apache/pulsar/discovery/service/web/RestException.java +++ /dev/null @@ -1,57 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.discovery.service.web; - -import java.io.PrintWriter; -import java.io.StringWriter; - -import javax.ws.rs.WebApplicationException; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; - -import org.apache.pulsar.common.policies.data.ErrorData; - -/** - * Exception used to provide better error messages to clients of the REST API. - */ -@SuppressWarnings("serial") -public class RestException extends WebApplicationException { - static String getExceptionData(Throwable t) { - StringWriter writer = new StringWriter(); - writer.append("\n --- An unexpected error occurred in the server ---\n\n"); - writer.append("Message: ").append(t.getMessage()).append("\n\n"); - writer.append("Stacktrace:\n\n"); - - t.printStackTrace(new PrintWriter(writer)); - return writer.toString(); - } - - public RestException(Response.Status status, String message) { - this(status.getStatusCode(), message); - } - - public RestException(int code, String message) { - super(Response.status(code).entity(new ErrorData(message)).type(MediaType.APPLICATION_JSON).build()); - } - - public RestException(Throwable t) { - super(Response.status(500).entity(getExceptionData(t)).type(MediaType.TEXT_PLAIN).build()); - } - -} diff --git a/pulsar-discovery-service/src/test/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceWebTest.java b/pulsar-discovery-service/src/test/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceWebTest.java index bf149699fb059..1aa84641282bc 100644 --- a/pulsar-discovery-service/src/test/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceWebTest.java +++ b/pulsar-discovery-service/src/test/java/org/apache/pulsar/discovery/service/web/DiscoveryServiceWebTest.java @@ -56,6 +56,7 @@ import org.apache.bookkeeper.util.ZkUtils; import org.apache.pulsar.common.policies.data.BundlesData; import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.discovery.service.server.ServerManager; import org.apache.pulsar.discovery.service.server.ServiceConfig; import org.apache.pulsar.policies.data.loadbalancer.LoadReport; @@ -247,12 +248,11 @@ public void testTlsEnable() throws Exception { @Test public void testException() { RestException exception1 = new RestException(BAD_GATEWAY, "test-msg"); - assertTrue(exception1.getMessage().contains(BAD_GATEWAY.toString())); + assertTrue(exception1.getMessage().contains("test-msg")); RestException exception2 = new RestException(BAD_GATEWAY.getStatusCode(), "test-msg"); - assertTrue(exception2.getMessage().contains(BAD_GATEWAY.toString())); + assertTrue(exception2.getMessage().contains("test-msg")); RestException exception3 = new RestException(exception2); - assertTrue(exception3.getMessage().contains(INTERNAL_SERVER_ERROR.toString())); - assertTrue(RestException.getExceptionData(exception2).contains(BAD_GATEWAY.toString())); + assertTrue(exception3.getMessage().contains(BAD_GATEWAY.toString())); } public List validateRequest(List brokers, String method, String url, BundlesData bundle) { diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/RestUtils.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/RestUtils.java index 6378b342c920d..db065cbef9a82 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/RestUtils.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/RestUtils.java @@ -26,6 +26,8 @@ import javax.ws.rs.core.Response; +import org.apache.pulsar.common.util.RestException; + @NoArgsConstructor(access = AccessLevel.PRIVATE) public final class RestUtils { diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java index cab2b07e4022b..924f71cfcb175 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java @@ -50,6 +50,7 @@ import org.apache.pulsar.common.policies.data.FunctionStats; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.util.Codec; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.instance.InstanceUtils; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.Function.FunctionDetails; @@ -68,7 +69,6 @@ import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.request.RequestResult; -import org.apache.pulsar.functions.worker.rest.RestException; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import javax.ws.rs.WebApplicationException; diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java index 026fb77bf0918..6e931b1eda0e3 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java @@ -28,6 +28,7 @@ import org.apache.pulsar.common.functions.Utils; import org.apache.pulsar.common.policies.data.ExceptionInformation; import org.apache.pulsar.common.policies.data.FunctionStatus; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.auth.FunctionAuthData; import org.apache.pulsar.functions.auth.FunctionAuthProvider; import org.apache.pulsar.functions.instance.InstanceUtils; @@ -39,7 +40,6 @@ import org.apache.pulsar.functions.worker.FunctionMetaDataManager; import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.WorkerUtils; -import org.apache.pulsar.functions.worker.rest.RestException; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import javax.ws.rs.WebApplicationException; diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplV2.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplV2.java index 50fe96c5c90b7..86118cea3b208 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplV2.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplV2.java @@ -24,13 +24,13 @@ import org.apache.pulsar.common.functions.FunctionState; import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.policies.data.FunctionStatus; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.InstanceCommunication; import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.FunctionConfigUtils; import org.apache.pulsar.functions.worker.FunctionMetaDataManager; import org.apache.pulsar.functions.worker.WorkerService; -import org.apache.pulsar.functions.worker.rest.RestException; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import javax.ws.rs.core.Response; diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java index 70e7b85149574..76e5686353513 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java @@ -29,6 +29,7 @@ import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.common.policies.data.ExceptionInformation; import org.apache.pulsar.common.policies.data.SinkStatus; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.auth.FunctionAuthData; import org.apache.pulsar.functions.instance.InstanceUtils; import org.apache.pulsar.functions.proto.Function; @@ -39,7 +40,6 @@ import org.apache.pulsar.functions.worker.FunctionMetaDataManager; import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.WorkerUtils; -import org.apache.pulsar.functions.worker.rest.RestException; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import javax.ws.rs.WebApplicationException; diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java index 8cec959e990bb..4c98004aaed7d 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java @@ -29,6 +29,7 @@ import org.apache.pulsar.common.io.SourceConfig; import org.apache.pulsar.common.policies.data.ExceptionInformation; import org.apache.pulsar.common.policies.data.SourceStatus; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.auth.FunctionAuthData; import org.apache.pulsar.functions.instance.InstanceUtils; import org.apache.pulsar.functions.proto.Function; @@ -39,7 +40,6 @@ import org.apache.pulsar.functions.worker.FunctionMetaDataManager; import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.WorkerUtils; -import org.apache.pulsar.functions.worker.rest.RestException; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import javax.ws.rs.WebApplicationException; diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/WorkerImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/WorkerImpl.java index 801d32eb22d65..88e74689f87be 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/WorkerImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/WorkerImpl.java @@ -23,6 +23,7 @@ import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.policies.data.FunctionStats; import org.apache.pulsar.common.policies.data.WorkerFunctionInstanceStats; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.worker.FunctionRuntimeInfo; @@ -30,7 +31,6 @@ import org.apache.pulsar.functions.worker.MembershipManager; import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.WorkerUtils; -import org.apache.pulsar.functions.worker.rest.RestException; import javax.ws.rs.core.Response.Status; import java.io.IOException; diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java index 90c134d902c94..88142e6476937 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java @@ -63,6 +63,7 @@ import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.apache.pulsar.functions.instance.InstanceUtils; @@ -82,7 +83,6 @@ import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.request.RequestResult; -import org.apache.pulsar.functions.worker.rest.RestException; import org.apache.pulsar.functions.worker.rest.api.FunctionsImpl; import org.apache.pulsar.functions.worker.rest.api.FunctionsImplV2; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java index 16e5e3077f72e..f1128ef5ad1e7 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java @@ -60,6 +60,7 @@ import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.apache.pulsar.functions.instance.InstanceUtils; @@ -79,7 +80,6 @@ import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.request.RequestResult; -import org.apache.pulsar.functions.worker.rest.RestException; import org.apache.pulsar.functions.worker.rest.api.FunctionsImpl; import org.apache.pulsar.functions.worker.rest.api.v2.FunctionsApiV2Resource; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java index c47f9e52b7b4d..b9aeccb58eec8 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java @@ -31,6 +31,7 @@ import org.apache.pulsar.common.nar.NarClassLoader; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.api.utils.IdentityFunction; import org.apache.pulsar.functions.instance.InstanceUtils; import org.apache.pulsar.functions.proto.Function; @@ -46,7 +47,6 @@ import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.request.RequestResult; -import org.apache.pulsar.functions.worker.rest.RestException; import org.apache.pulsar.functions.worker.rest.api.SinksImpl; import org.apache.pulsar.io.cassandra.CassandraStringSink; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java index faf3f8812ff95..88dcff4fcaf55 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java @@ -54,6 +54,7 @@ import org.apache.pulsar.common.nar.NarClassLoader; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.api.utils.IdentityFunction; import org.apache.pulsar.functions.instance.InstanceUtils; import org.apache.pulsar.functions.proto.Function.FunctionDetails; @@ -73,7 +74,6 @@ import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.request.RequestResult; -import org.apache.pulsar.functions.worker.rest.RestException; import org.apache.pulsar.functions.worker.rest.api.SourcesImpl; import org.apache.pulsar.io.twitter.TwitterFireHose; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/admin/RestException.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/admin/RestException.java deleted file mode 100644 index f57d19432286d..0000000000000 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/admin/RestException.java +++ /dev/null @@ -1,66 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.pulsar.websocket.admin; - -import java.io.PrintWriter; -import java.io.StringWriter; - -import javax.ws.rs.WebApplicationException; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; - -import org.apache.pulsar.common.policies.data.ErrorData; - -/** - * Exception used to provide better error messages to clients of the REST API. - */ -@SuppressWarnings("serial") -public class RestException extends WebApplicationException { - static String getExceptionData(Throwable t) { - StringWriter writer = new StringWriter(); - writer.append("\n --- An unexpected error occurred in the server ---\n\n"); - writer.append("Message: ").append(t.getMessage()).append("\n\n"); - writer.append("Stacktrace:\n\n"); - - t.printStackTrace(new PrintWriter(writer)); - return writer.toString(); - } - - public RestException(Response.Status status, String message) { - this(status.getStatusCode(), message); - } - - public RestException(int code, String message) { - super(Response.status(code).entity(new ErrorData(message)).type(MediaType.APPLICATION_JSON).build()); - } - - public RestException(Throwable t) { - super(getResponse(t)); - } - - private static Response getResponse(Throwable t) { - if (t instanceof RestException) { - RestException e = (RestException) t; - return Response.status(e.getResponse().getStatus()).entity(e.getResponse().getEntity()) - .type(e.getResponse().getMediaType()).build(); - } else { - return Response.status(500).entity(getExceptionData(t)).type(MediaType.TEXT_PLAIN).build(); - } - } -} diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/admin/WebSocketProxyStatsBase.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/admin/WebSocketProxyStatsBase.java index cbbbddcf468c3..c3de46b79839b 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/admin/WebSocketProxyStatsBase.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/admin/WebSocketProxyStatsBase.java @@ -27,6 +27,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.stats.Metrics; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.websocket.stats.ProxyTopicStat; import org.apache.pulsar.websocket.stats.ProxyTopicStat.ConsumerStats; import org.apache.pulsar.websocket.stats.ProxyTopicStat.ProducerStats; diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/admin/WebSocketWebResource.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/admin/WebSocketWebResource.java index 10e3664edf65d..255fe654a5353 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/admin/WebSocketWebResource.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/admin/WebSocketWebResource.java @@ -29,6 +29,7 @@ import org.apache.pulsar.broker.authentication.AuthenticationDataHttps; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.websocket.WebSocketService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/admin/WebSocketWebResourceTest.java b/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/admin/WebSocketWebResourceTest.java index 6d7e8cf084140..5d1d5a58ac49a 100644 --- a/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/admin/WebSocketWebResourceTest.java +++ b/pulsar-websocket/src/test/java/org/apache/pulsar/websocket/admin/WebSocketWebResourceTest.java @@ -50,6 +50,7 @@ import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authorization.AuthorizationService; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.websocket.WebSocketService; public class WebSocketWebResourceTest { From 5c2c058ff9d8c783fe7a6b00e55da524da7ef4bb Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Sat, 7 Mar 2020 18:18:33 -0800 Subject: [PATCH 68/74] pulsar-proxy: fix correct name for proxy thread executor name (#6460) ### Motivation fix correct name for proxy thread executor name --- .../apache/pulsar/proxy/server/util/ZookeeperCacheLoader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/util/ZookeeperCacheLoader.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/util/ZookeeperCacheLoader.java index c3afdec9ec381..82dd42c5e19f6 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/util/ZookeeperCacheLoader.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/util/ZookeeperCacheLoader.java @@ -54,7 +54,7 @@ public class ZookeeperCacheLoader implements Closeable { private volatile List availableBrokers; private final OrderedScheduler orderedExecutor = OrderedScheduler.newSchedulerBuilder().numThreads(8) - .name("pulsar-discovery-ordered-cache").build(); + .name("pulsar-proxy-ordered-cache").build(); public static final String LOADBALANCE_BROKERS_ROOT = "/loadbalance/brokers"; From 9fc51856a8c8dca6f1b668ae73d01514f153d895 Mon Sep 17 00:00:00 2001 From: Fangbin Sun Date: Sun, 8 Mar 2020 17:26:51 +0800 Subject: [PATCH 69/74] Add subscribe initial position for consumer cli. (#6442) ### Motivation In some case, users expect to consume messages from beginning similar to the option `--from-beginning` of kafka consumer CLI. ### Modifications Add `--subscription-position` for `pulsar-client` and `pulsar-perf`. --- .../java/org/apache/pulsar/client/cli/CmdConsume.java | 11 ++++++++--- .../apache/pulsar/testclient/PerformanceConsumer.java | 7 ++++++- site2/docs/reference-cli-tools.md | 5 ++++- 3 files changed, 18 insertions(+), 5 deletions(-) diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java index 895f59516dc30..673a45e132985 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/client/cli/CmdConsume.java @@ -50,6 +50,7 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.util.collections.GrowableArrayBlockingQueue; @@ -78,9 +79,12 @@ public class CmdConsume { @Parameter(description = "TopicName", required = true) private List mainOptions = new ArrayList(); - @Parameter(names = { "-t", "--subscription-type" }, description = "Subscription type: Exclusive, Shared, Failover.") + @Parameter(names = { "-t", "--subscription-type" }, description = "Subscription type.") private SubscriptionType subscriptionType = SubscriptionType.Exclusive; + @Parameter(names = { "-p", "--subscription-position" }, description = "Subscription position.") + private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest; + @Parameter(names = { "-s", "--subscription-name" }, required = true, description = "Subscription name.") private String subscriptionName; @@ -95,7 +99,7 @@ public class CmdConsume { + "value 0 means to consume messages as fast as possible.") private double consumeRate = 0; - @Parameter(names = { "--regex" }, description = "Indicate thetopic name is a regex pattern") + @Parameter(names = { "--regex" }, description = "Indicate the topic name is a regex pattern") private boolean isRegex = false; private ClientBuilder clientBuilder; @@ -182,7 +186,8 @@ private int consume(String topic) { PulsarClient client = clientBuilder.build(); ConsumerBuilder builder = client.newConsumer() .subscriptionName(this.subscriptionName) - .subscriptionType(subscriptionType); + .subscriptionType(subscriptionType) + .subscriptionInitialPosition(subscriptionInitialPosition); if (isRegex) { builder.topicsPattern(Pattern.compile(topic)); diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java index 5c569fd0bd64e..cf38f271c8779 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java @@ -41,6 +41,7 @@ import org.apache.pulsar.client.api.EncryptionKeyInfo; import org.apache.pulsar.client.api.MessageListener; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.naming.TopicName; import org.slf4j.Logger; @@ -86,9 +87,12 @@ static class Arguments { @Parameter(names = { "-s", "--subscriber-name" }, description = "Subscriber name prefix") public String subscriberName = "sub"; - @Parameter(names = { "-st", "--subscription-type" }, description = "Subscriber name prefix") + @Parameter(names = { "-st", "--subscription-type" }, description = "Subscription type") public SubscriptionType subscriptionType = SubscriptionType.Exclusive; + @Parameter(names = { "-sp", "--subscription-position" }, description = "Subscription position") + private SubscriptionInitialPosition subscriptionInitialPosition = SubscriptionInitialPosition.Latest; + @Parameter(names = { "-r", "--rate" }, description = "Simulate a slow message consumer (rate in msg/s)") public double rate = 0; @@ -257,6 +261,7 @@ public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMe .receiverQueueSize(arguments.receiverQueueSize) // .acknowledgmentGroupTime(arguments.acknowledgmentsGroupingDelayMillis, TimeUnit.MILLISECONDS) // .subscriptionType(arguments.subscriptionType) + .subscriptionInitialPosition(arguments.subscriptionInitialPosition) .replicateSubscriptionState(arguments.replicatedSubscription); if (arguments.encKeyName != null) { diff --git a/site2/docs/reference-cli-tools.md b/site2/docs/reference-cli-tools.md index cd737cd3dcc7e..9a87c8c92c5b7 100644 --- a/site2/docs/reference-cli-tools.md +++ b/site2/docs/reference-cli-tools.md @@ -325,8 +325,10 @@ Options |`--hex`|Display binary messages in hexadecimal format.|false| |`-n`, `--num-messages`|Number of messages to consume, 0 means to consume forever.|1| |`-r`, `--rate`|Rate (in messages per second) at which to consume; a value 0 means to consume messages as fast as possible|0.0| +|`--regex`|Indicate the topic name is a regex pattern|false| |`-s`, `--subscription-name`|Subscription name|| |`-t`, `--subscription-type`|The type of the subscription. Possible values: Exclusive, Shared, Failover, Key_Shared.|Exclusive| +|`-p`, `--subscription-position`|The position of the subscription. Possible values: Latest, Earliest.|Latest| @@ -426,7 +428,8 @@ Options |`-u`, `--service-url`|Pulsar service URL|| |`-i`, `--stats-interval-seconds`|Statistics interval seconds. If 0, statistics will be disabled|0| |`-s`, `--subscriber-name`|Subscriber name prefix|sub| -|`-st`, `--subscription-type`|Subscriber name prefix. Possible values are Exclusive, Shared, Failover.|Exclusive| +|`-st`, `--subscription-type`|Subscriber type. Possible values are Exclusive, Shared, Failover, Key_Shared.|Exclusive| +|`-sp`, `--subscription-position`|Subscriber position. Possible values are Latest, Earliest.|Latest| |`--trust-cert-file`|Path for the trusted TLS certificate file|| From ff01b10c69b1623f9433efbb7ada7d00202d0789 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Mon, 9 Mar 2020 15:44:15 +0800 Subject: [PATCH 70/74] [Cleanup] Log format does not match arguments (#6509) --- .../bookkeeper/mledger/impl/ManagedLedgerImpl.java | 8 ++++---- .../broker/admin/impl/PersistentTopicsBase.java | 2 +- .../pulsar/broker/namespace/NamespaceService.java | 3 +-- .../apache/pulsar/broker/service/ServerCnx.java | 14 +++++++------- .../service/persistent/PersistentSubscription.java | 4 ++-- .../org/apache/pulsar/client/impl/ClientCnx.java | 5 +++-- .../client/impl/MultiTopicsConsumerImpl.java | 12 ++++++------ .../client/impl/TransactionMetaStoreHandler.java | 2 +- 8 files changed, 25 insertions(+), 25 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index ecf99a426dbd4..a1d41b6c6a000 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -302,7 +302,7 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { executor.executeOrdered(name, safeRun(() -> { mbean.endDataLedgerOpenOp(); if (log.isDebugEnabled()) { - log.debug("[{}] Opened ledger {}: ", name, id, BKException.getMessage(rc)); + log.debug("[{}] Opened ledger {}: {}", name, id, BKException.getMessage(rc)); } if (rc == BKException.Code.OK) { LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(id) @@ -1986,7 +1986,7 @@ void internalTrimConsumedLedgers(CompletableFuture promise) { overRetentionQuota, currentLedger.getId()); } if (ls.getLedgerId() == currentLedger.getId()) { - log.debug("[{}] ledger id skipped for deletion as it is currently being written to", name, + log.debug("[{}] Ledger {} skipped for deletion as it is currently being written to", name, ls.getLedgerId()); break; } else if (expired) { @@ -2146,7 +2146,7 @@ public void deleteCursorComplete(Object ctx) { @Override public void deleteCursorFailed(ManagedLedgerException exception, Object ctx) { - log.warn("[{}] Failed to delete cursor {}", name, cursor, exception); + log.warn("[{}] Failed to delete cursor {} : {}", name, cursor, exception); cursorDeleteException.compareAndSet(null, exception); if (cursorsToDelete.decrementAndGet() == 0) { // Trigger callback only once @@ -2185,7 +2185,7 @@ private void asyncDeleteLedger(long ledgerId, long retry) { if (isNoSuchLedgerExistsException(rc)) { log.warn("[{}] Ledger was already deleted {}", name, ledgerId); } else if (rc != BKException.Code.OK) { - log.error("[{}] Error deleting ledger {}", name, ledgerId, BKException.getMessage(rc)); + log.error("[{}] Error deleting ledger {} : {}", name, ledgerId, BKException.getMessage(rc)); scheduledExecutor.schedule(safeRun(() -> { asyncDeleteLedger(ledgerId, retry - 1); }), DEFAULT_LEDGER_DELETE_BACKOFF_TIME_SEC, TimeUnit.SECONDS); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java index 685d1952535c6..684c7c8a7271c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java @@ -387,7 +387,7 @@ private void revokePermissions(String topicUri, String role) { if (!policies.auth_policies.destination_auth.containsKey(topicUri) || !policies.auth_policies.destination_auth.get(topicUri).containsKey(role)) { - log.warn("[{}] Failed to revoke permission from role {} on topic: Not set at topic level", + log.warn("[{}] Failed to revoke permission from role {} on topic: Not set at topic level {}", clientAppId(), role, topicUri); throw new RestException(Status.PRECONDITION_FAILED, "Permissions are not set at the topic level"); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index d4e90be705d99..62876f014561a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -446,8 +446,7 @@ private void searchForCandidateBroker(NamespaceBundle bundle, lookupFuture.complete(Optional.of(new LookupResult(ownerInfo))); } }).exceptionally(exception -> { - LOG.warn("Failed to acquire ownership for namespace bundle {}: ", bundle, exception.getMessage(), - exception); + LOG.warn("Failed to acquire ownership for namespace bundle {}: {}", bundle, exception); lookupFuture.completeExceptionally(new PulsarServerException( "Failed to acquire ownership for namespace bundle " + bundle, exception)); return null; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index 9294330c87f42..1037952755d36 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -54,6 +54,7 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.util.SafeRun; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; @@ -216,7 +217,7 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { try { consumer.close(); } catch (BrokerServiceException e) { - log.warn("Consumer {} was already closed: {}", consumer, e.getMessage(), e); + log.warn("Consumer {} was already closed: {}", consumer, e); } }); } @@ -232,14 +233,14 @@ public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exceptio public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { if (state != State.Failed) { // No need to report stack trace for known exceptions that happen in disconnections - log.warn("[{}] Got exception {} : {}", remoteAddress, cause.getClass().getSimpleName(), cause.getMessage(), - ClientCnx.isKnownException(cause) ? null : cause); + log.warn("[{}] Got exception {}", remoteAddress, + ClientCnx.isKnownException(cause) ? cause : ExceptionUtils.getStackTrace(cause)); state = State.Failed; } else { // At default info level, suppress all subsequent exceptions that are thrown when the connection has already // failed if (log.isDebugEnabled()) { - log.debug("[{}] Got exception: {}", remoteAddress, cause.getMessage(), cause); + log.debug("[{}] Got exception: {}", remoteAddress, cause); } } ctx.close(); @@ -582,8 +583,7 @@ public void refreshAuthenticationCredentials() { pendingAuthChallengeResponse = true; } catch (AuthenticationException e) { - log.warn("[{}] Failed to refresh authentication: ", - remoteAddress, e.getMessage()); + log.warn("[{}] Failed to refresh authentication: {}", remoteAddress, e); ctx.close(); } })); @@ -1391,7 +1391,7 @@ protected void handleCloseConsumer(CommandCloseConsumer closeConsumer) { ctx.writeAndFlush(Commands.newSuccess(requestId)); log.info("[{}] Closed consumer {}", remoteAddress, consumer); } catch (BrokerServiceException e) { - log.warn("[{]] Error closing consumer: ", remoteAddress, consumer, e); + log.warn("[{]] Error closing consumer {} : {}", remoteAddress, consumer, e); ctx.writeAndFlush( Commands.newError(requestId, BrokerServiceException.getClientErrorCode(e), e.getMessage())); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 9bcd2874dc1bc..1ae209d4a305a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -515,7 +515,7 @@ public void markDeleteComplete(Object ctx) { public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { // TODO: cut consumer connection on markDeleteFailed if (log.isDebugEnabled()) { - log.debug("[{}][{}] Failed to mark delete for position ", topicName, subName, ctx, exception); + log.debug("[{}][{}] Failed to mark delete for position {}: {}", topicName, subName, ctx, exception); } } }; @@ -530,7 +530,7 @@ public void deleteComplete(Object position) { @Override public void deleteFailed(ManagedLedgerException exception, Object ctx) { - log.warn("[{}][{}] Failed to delete message at {}", topicName, subName, ctx, exception); + log.warn("[{}][{}] Failed to delete message at {}: {}", topicName, subName, ctx, exception); } }; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java index dfa60e8294df5..3205608ca7ad1 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java @@ -50,6 +50,7 @@ import javax.net.ssl.SSLSession; import lombok.Getter; +import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.http.conn.ssl.DefaultHostnameVerifier; import org.apache.pulsar.PulsarVersion; @@ -261,8 +262,8 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { if (state != State.Failed) { // No need to report stack trace for known exceptions that happen in disconnections - log.warn("[{}] Got exception {} : {}", remoteAddress, cause.getClass().getSimpleName(), cause.getMessage(), - isKnownException(cause) ? null : cause); + log.warn("[{}] Got exception {}", remoteAddress, + ClientCnx.isKnownException(cause) ? cause : ExceptionUtils.getStackTrace(cause)); state = State.Failed; } else { // At default info level, suppress all subsequent exceptions that are thrown when the connection has already diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 970e1343c3adb..558282ae0108f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -1094,12 +1094,12 @@ private CompletableFuture subscribeIncreasedTopicPartitions(String topicNa CompletableFuture future = new CompletableFuture<>(); client.getPartitionsForTopic(topicName).thenCompose(list -> { - int oldPartitionNumber = topics.get(topicName.toString()); + int oldPartitionNumber = topics.get(topicName); int currentPartitionNumber = list.size(); if (log.isDebugEnabled()) { log.debug("[{}] partitions number. old: {}, new: {}", - topicName.toString(), oldPartitionNumber, currentPartitionNumber); + topicName, oldPartitionNumber, currentPartitionNumber); } if (oldPartitionNumber == currentPartitionNumber) { @@ -1123,7 +1123,7 @@ private CompletableFuture subscribeIncreasedTopicPartitions(String topicNa consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); if (log.isDebugEnabled()) { log.debug("[{}] create consumer {} for partitionName: {}", - topicName.toString(), newConsumer.getTopic(), partitionName); + topicName, newConsumer.getTopic(), partitionName); } return subFuture; }) @@ -1140,14 +1140,14 @@ private CompletableFuture subscribeIncreasedTopicPartitions(String topicNa future.complete(null); }) .exceptionally(ex -> { - log.warn("[{}] Failed to subscribe {} partition: {} - {}", - topic, topicName.toString(), oldPartitionNumber, currentPartitionNumber, ex.getMessage()); + log.warn("[{}] Failed to subscribe {} partition: {} - {} : {}", + topic, topicName, oldPartitionNumber, currentPartitionNumber, ex); future.completeExceptionally(ex); return null; }); } else { log.error("[{}] not support shrink topic partitions. old: {}, new: {}", - topicName.toString(), oldPartitionNumber, currentPartitionNumber); + topicName, oldPartitionNumber, currentPartitionNumber); future.completeExceptionally(new NotSupportedException("not support shrink topic partitions")); } return future; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java index 7a99fe1658691..f8584ab28f598 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java @@ -140,7 +140,7 @@ void handleNewTxnResponse(PulsarApi.CommandNewTxnResponse response) { public CompletableFuture addPublishPartitionToTxnAsync(TxnID txnID, List partitions) { if (LOG.isDebugEnabled()) { - LOG.debug("Add publish partition to txn request with txnId, with partitions", txnID, partitions); + LOG.debug("Add publish partition {} to txn {}", partitions, txnID); } CompletableFuture callback = new CompletableFuture<>(); From 5285c68bee24bd50a2ee1064243ba60ed333d451 Mon Sep 17 00:00:00 2001 From: lipenghui Date: Mon, 9 Mar 2020 15:45:18 +0800 Subject: [PATCH 71/74] Start namespace service and schema registry service before start broker. (#6499) ### Motivation If the broker service is started, the client can connect to the broker and send requests depends on the namespace service, so we should create the namespace service before starting the broker. Otherwise, NPE occurs. ![image](https://user-images.githubusercontent.com/12592133/76090515-a9961400-5ff6-11ea-9077-cb8e79fa27c0.png) ![image](https://user-images.githubusercontent.com/12592133/76099838-b15db480-6006-11ea-8f39-31d820563c88.png) ### Modifications Move the namespace service creation and the schema registry service creation before start broker service. --- .../apache/pulsar/broker/PulsarService.java | 10 ++++++---- .../broker/namespace/NamespaceService.java | 11 ++++++++--- .../broker/namespace/OwnershipCache.java | 18 +++++++++++++++++- 3 files changed, 31 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 31b7fa391f245..5236efacb4dc2 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -401,6 +401,10 @@ public void start() throws PulsarServerException { // Start load management service (even if load balancing is disabled) this.loadManager.set(LoadManager.create(this)); + // needs load management service and before start broker service, + this.startNamespaceService(); + schemaRegistryService = SchemaRegistryService.create(this); + this.defaultOffloader = createManagedLedgerOffloader( OffloadPolicies.create(this.getConfiguration().getProperties())); @@ -458,8 +462,6 @@ public Boolean get() { } this.webService.addStaticResources("/static", "/static"); - schemaRegistryService = SchemaRegistryService.create(this); - webService.start(); // Refresh addresses, since the port might have been dynamically assigned @@ -474,8 +476,8 @@ public Boolean get() { this.webSocketService.setLocalCluster(clusterData); } - // needs load management service - this.startNamespaceService(); + // Initialize namespace service, after service url assigned. Should init zk and refresh self owner info. + this.nsService.initialize(); // Start the leader election service startLeaderElectionService(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java index 62876f014561a..24d933d0f2b3f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/NamespaceService.java @@ -73,7 +73,6 @@ import java.net.URI; import java.net.URL; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -158,13 +157,19 @@ public NamespaceService(PulsarService pulsar) { host = pulsar.getAdvertisedAddress(); this.config = pulsar.getConfiguration(); this.loadManager = pulsar.getLoadManager(); - ServiceUnitZkUtils.initZK(pulsar.getLocalZkCache().getZooKeeper(), pulsar.getSafeBrokerServiceUrl()); this.bundleFactory = new NamespaceBundleFactory(pulsar, Hashing.crc32()); this.ownershipCache = new OwnershipCache(pulsar, bundleFactory, this); this.namespaceClients = new ConcurrentOpenHashMap<>(); this.bundleOwnershipListeners = new CopyOnWriteArrayList<>(); } + public void initialize() { + ServiceUnitZkUtils.initZK(pulsar.getLocalZkCache().getZooKeeper(), pulsar.getSafeBrokerServiceUrl()); + if (!getOwnershipCache().refreshSelfOwnerInfo()) { + throw new RuntimeException("Failed to refresh self owner info."); + } + } + public CompletableFuture> getBrokerServiceUrlAsync(TopicName topic, boolean authoritative) { return getBundleAsync(topic) @@ -424,7 +429,7 @@ private void searchForCandidateBroker(NamespaceBundle bundle, try { checkNotNull(candidateBroker); - if (pulsar.getSafeWebServiceAddress().equals(candidateBroker)) { + if (candidateBroker.equals(pulsar.getSafeWebServiceAddress())) { // invalidate namespace policies and try to load latest policies to avoid data-discrepancy if broker // doesn't receive watch on policies changes final String policyPath = AdminResource.path(POLICIES, bundle.getNamespaceObject().toString()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java index 6b6ee36242dfc..50e96fadf51fb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/OwnershipCache.java @@ -73,7 +73,7 @@ public class OwnershipCache { /** * The NamespaceEphemeralData objects that can be associated with the current owner */ - private final NamespaceEphemeralData selfOwnerInfo; + private NamespaceEphemeralData selfOwnerInfo; /** * The NamespaceEphemeralData objects that can be associated with the current owner, when the broker is disabled. @@ -111,6 +111,8 @@ public class OwnershipCache { */ private NamespaceService namespaceService; + private final PulsarService pulsar; + private class OwnedServiceUnitCacheLoader implements AsyncCacheLoader { @SuppressWarnings("deprecation") @@ -156,6 +158,7 @@ public CompletableFuture asyncLoad(String namespaceBundleZNode, Exe */ public OwnershipCache(PulsarService pulsar, NamespaceBundleFactory bundleFactory, NamespaceService namespaceService) { this.namespaceService = namespaceService; + this.pulsar = pulsar; this.ownerBrokerUrl = pulsar.getSafeBrokerServiceUrl(); this.ownerBrokerUrlTls = pulsar.getBrokerServiceUrlTls(); this.selfOwnerInfo = new NamespaceEphemeralData(ownerBrokerUrl, ownerBrokerUrlTls, @@ -211,6 +214,11 @@ public CompletableFuture tryAcquiringOwnership(Namespace CompletableFuture future = new CompletableFuture<>(); + if (!refreshSelfOwnerInfo()) { + future.completeExceptionally(new RuntimeException("Namespace service does not ready for acquiring ownership")); + return future; + } + LOG.info("Trying to acquire ownership of {}", bundle); // Doing a get() on the ownedBundlesCache will trigger an async ZK write to acquire the lock over the @@ -367,4 +375,12 @@ public void updateBundleState(NamespaceBundle bundle, boolean isActive) throws E public NamespaceEphemeralData getSelfOwnerInfo() { return selfOwnerInfo; } + + public synchronized boolean refreshSelfOwnerInfo() { + if (selfOwnerInfo.getNativeUrl() == null) { + this.selfOwnerInfo = new NamespaceEphemeralData(pulsar.getSafeBrokerServiceUrl(), pulsar.getBrokerServiceUrlTls(), + pulsar.getSafeWebServiceAddress(), pulsar.getWebServiceAddressTls(), false); + } + return selfOwnerInfo.getNativeUrl() != null; + } } From 67f8cf30d33f3cb7e8d9309cadb2d80626dd25bc Mon Sep 17 00:00:00 2001 From: k2la Date: Mon, 9 Mar 2020 16:45:37 +0900 Subject: [PATCH 72/74] [pulsar-client-cpp] Fix Redelivery of Messages on UnackedMessageTracker When Ack Messages . (#6498) ### Motivation Because of #6391 , acked messages were counted as unacked messages. Although messages from brokers were acknowledged, the following log was output. ``` 2020-03-06 19:44:51.790 INFO ConsumerImpl:174 | [persistent://public/default/t1, sub1, 0] Created consumer on broker [127.0.0.1:58860 -> 127.0.0.1:6650] my-message-0: Fri Mar 6 19:45:05 2020 my-message-1: Fri Mar 6 19:45:05 2020 my-message-2: Fri Mar 6 19:45:05 2020 2020-03-06 19:45:15.818 INFO UnAckedMessageTrackerEnabled:53 | [persistent://public/default/t1, sub1, 0] : 3 Messages were not acked within 10000 time ``` This behavior happened on master branch. --- .../lib/UnAckedMessageTrackerEnabled.cc | 14 +++++++++----- .../lib/UnAckedMessageTrackerEnabled.h | 3 +-- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.cc b/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.cc index 7894e64a874f6..9185dba70bb3b 100644 --- a/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.cc +++ b/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.cc @@ -90,8 +90,10 @@ UnAckedMessageTrackerEnabled::UnAckedMessageTrackerEnabled(long timeoutMs, long bool UnAckedMessageTrackerEnabled::add(const MessageId& m) { std::lock_guard acquire(lock_); if (messageIdPartitionMap.count(m) == 0) { - bool insert = messageIdPartitionMap.insert(std::make_pair(m, timePartitions.back())).second; - return insert && timePartitions.back().insert(m).second; + std::set& partition = timePartitions.back(); + bool emplace = messageIdPartitionMap.emplace(m, partition).second; + bool insert = partition.insert(m).second; + return emplace && insert; } return false; } @@ -104,7 +106,8 @@ bool UnAckedMessageTrackerEnabled::isEmpty() { bool UnAckedMessageTrackerEnabled::remove(const MessageId& m) { std::lock_guard acquire(lock_); bool removed = false; - std::map>::iterator exist = messageIdPartitionMap.find(m); + + std::map&>::iterator exist = messageIdPartitionMap.find(m); if (exist != messageIdPartitionMap.end()) { removed = exist->second.erase(m); } @@ -121,7 +124,7 @@ void UnAckedMessageTrackerEnabled::removeMessagesTill(const MessageId& msgId) { for (auto it = messageIdPartitionMap.begin(); it != messageIdPartitionMap.end(); it++) { MessageId msgIdInMap = it->first; if (msgIdInMap < msgId) { - std::map>::iterator exist = messageIdPartitionMap.find(msgId); + std::map&>::iterator exist = messageIdPartitionMap.find(msgId); if (exist != messageIdPartitionMap.end()) { exist->second.erase(msgId); } @@ -135,7 +138,8 @@ void UnAckedMessageTrackerEnabled::removeTopicMessage(const std::string& topic) for (auto it = messageIdPartitionMap.begin(); it != messageIdPartitionMap.end(); it++) { MessageId msgIdInMap = it->first; if (msgIdInMap.getTopicName().compare(topic) == 0) { - std::map>::iterator exist = messageIdPartitionMap.find(msgIdInMap); + std::map&>::iterator exist = + messageIdPartitionMap.find(msgIdInMap); if (exist != messageIdPartitionMap.end()) { exist->second.erase(msgIdInMap); } diff --git a/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.h b/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.h index c2b4012adb184..9195b30d70d69 100644 --- a/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.h +++ b/pulsar-client-cpp/lib/UnAckedMessageTrackerEnabled.h @@ -23,7 +23,6 @@ #include namespace pulsar { - class UnAckedMessageTrackerEnabled : public UnAckedMessageTrackerInterface { public: ~UnAckedMessageTrackerEnabled(); @@ -41,7 +40,7 @@ class UnAckedMessageTrackerEnabled : public UnAckedMessageTrackerInterface { void timeoutHandlerHelper(); bool isEmpty(); long size(); - std::map> messageIdPartitionMap; + std::map&> messageIdPartitionMap; std::deque> timePartitions; std::mutex lock_; DeadlineTimerPtr timer_; From de4c6a3cc7a68161e9bc4fd3dec09402ebe647ff Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Mon, 9 Mar 2020 00:46:01 -0700 Subject: [PATCH 73/74] [pulsar-proxy] fixing data-type of logging-level (#6476) ### Modification `ProxyConfig` has wrapper method for `proxyLogLevel` to present `Optional` data-type. after #3543 we can define config param as optional without creating wrapper methods. --- .../apache/pulsar/proxy/server/ProxyConfiguration.java | 9 +-------- .../org/apache/pulsar/proxy/server/ProxyService.java | 4 ++-- .../org/apache/pulsar/proxy/server/ProxyParserTest.java | 4 ++-- 3 files changed, 5 insertions(+), 12 deletions(-) diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java index 6a293a0411965..02c4549ec836d 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java @@ -153,7 +153,7 @@ public class ProxyConfiguration implements PulsarConfiguration { + " 1: Parse and log any tcp channel info and command info without message body" + " 2: Parse and log channel info, command info and message body" ) - private Integer proxyLogLevel = 0; + private Optional proxyLogLevel = Optional.ofNullable(0); @FieldContext( category = CATEGORY_SERVER, @@ -386,13 +386,6 @@ public Optional getServicePort() { return servicePort; } - public Optional getproxyLogLevel() { - return Optional.ofNullable(proxyLogLevel); - } - public void setProxyLogLevel(int proxyLogLevel) { - this.proxyLogLevel = proxyLogLevel; - } - public Optional getServicePortTls() { return servicePortTls; } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java index a4c505d5dcec5..a5776d53cef34 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java @@ -106,8 +106,8 @@ public ProxyService(ProxyConfiguration proxyConfig, this.lookupRequestSemaphore = new AtomicReference( new Semaphore(proxyConfig.getMaxConcurrentLookupRequests(), false)); - if (proxyConfig.getproxyLogLevel().isPresent()) { - ProxyService.proxyLogLevel = Integer.valueOf(proxyConfig.getproxyLogLevel().get()); + if (proxyConfig.getProxyLogLevel().isPresent()) { + ProxyService.proxyLogLevel = Integer.valueOf(proxyConfig.getProxyLogLevel().get()); } else { ProxyService.proxyLogLevel = 0; } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java index 3557441d80612..907dc787451a8 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java @@ -75,14 +75,14 @@ protected void setup() throws Exception { proxyConfig.setZookeeperServers(DUMMY_VALUE); proxyConfig.setConfigurationStoreServers(DUMMY_VALUE); //enable full parsing feature - proxyConfig.setProxyLogLevel(2); + proxyConfig.setProxyLogLevel(Optional.ofNullable(2)); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)))); doReturn(mockZooKeeperClientFactory).when(proxyService).getZooKeeperClientFactory(); Optional proxyLogLevel = Optional.of(2); - assertEquals( proxyLogLevel , proxyService.getConfiguration().getproxyLogLevel()); + assertEquals( proxyLogLevel , proxyService.getConfiguration().getProxyLogLevel()); proxyService.start(); } From 2936e176e21813d010289f00a9d28020fcabab80 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Mon, 9 Mar 2020 18:02:14 -0700 Subject: [PATCH 74/74] [pulsar-broker] recover zk-badversion while updating cursor metadata (#5604) fix test --- .../mledger/ManagedLedgerFactory.java | 8 +- .../mledger/impl/ManagedCursorImpl.java | 26 ++++- .../impl/ManagedLedgerFactoryImpl.java | 9 +- .../mledger/impl/ManagedLedgerImpl.java | 8 ++ .../mledger/impl/ManagedLedgerTest.java | 108 +++++++++++++++++- .../pulsar/broker/service/BrokerService.java | 11 +- ...sistentDispatcherFailoverConsumerTest.java | 5 +- .../broker/service/PersistentTopicTest.java | 10 +- .../pulsar/broker/service/ServerCnxTest.java | 18 +-- .../PersistentTransactionBufferTest.java | 7 +- .../testclient/ManagedLedgerWriter.java | 2 +- 11 files changed, 185 insertions(+), 27 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java index 49197c51242d5..914b3df67299b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java @@ -19,6 +19,9 @@ package org.apache.bookkeeper.mledger; import com.google.common.annotations.Beta; + +import java.util.function.Supplier; + import org.apache.bookkeeper.mledger.AsyncCallbacks.ManagedLedgerInfoCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenLedgerCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenReadOnlyCursorCallback; @@ -77,10 +80,13 @@ ManagedLedger open(String name, ManagedLedgerConfig config) * managed ledger configuration * @param callback * callback object + * @param mlOwnershipChecker + * checks ml-ownership in case updating ml-metadata fails due to ownership conflict * @param ctx * opaque context */ - void asyncOpen(String name, ManagedLedgerConfig config, OpenLedgerCallback callback, Object ctx); + void asyncOpen(String name, ManagedLedgerConfig config, OpenLedgerCallback callback, + Supplier mlOwnershipChecker, Object ctx); /** * Open a {@link ReadOnlyCursor} positioned to the earliest entry for the specified managed ledger diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 05f6f5c762133..411279dfd6819 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -1958,7 +1958,7 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { * @param properties * @param callback */ - private void persistPositionWhenClosing(PositionImpl position, Map properties, + void persistPositionWhenClosing(PositionImpl position, Map properties, final AsyncCallbacks.CloseCallback callback, final Object ctx) { if (shouldPersistUnackRangesToLedger()) { @@ -2053,6 +2053,30 @@ public void operationComplete(Void result, Stat stat) { @Override public void operationFailed(MetaStoreException e) { + if (e instanceof MetaStoreException.BadVersionException) { + log.warn("[{}] Failed to update cursor metadata for {} due to version conflict {}", + ledger.name, name, e.getMessage()); + // it means previous owner of the ml might have updated the version incorrectly. So, check + // the ownership and refresh the version again. + if (ledger.mlOwnershipChecker != null && ledger.mlOwnershipChecker.get()) { + ledger.getStore().asyncGetCursorInfo(ledger.getName(), name, + new MetaStoreCallback() { + @Override + public void operationComplete(ManagedCursorInfo info, Stat stat) { + cursorLedgerStat = stat; + } + + @Override + public void operationFailed(MetaStoreException e) { + if (log.isDebugEnabled()) { + log.debug( + "[{}] Failed to refresh cursor metadata-version for {} due to {}", + ledger.name, name, e.getMessage()); + } + } + }); + } + } callback.operationFailed(e); } }); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index b88bc7f3444f1..6f037ceb155a3 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -40,6 +40,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; @@ -280,7 +281,7 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { r.e = exception; latch.countDown(); } - }, null); + }, null, null); latch.await(); @@ -292,12 +293,12 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { @Override public void asyncOpen(String name, OpenLedgerCallback callback, Object ctx) { - asyncOpen(name, new ManagedLedgerConfig(), callback, ctx); + asyncOpen(name, new ManagedLedgerConfig(), callback, null, ctx); } @Override public void asyncOpen(final String name, final ManagedLedgerConfig config, final OpenLedgerCallback callback, - final Object ctx) { + Supplier mlOwnershipChecker, final Object ctx) { // If the ledger state is bad, remove it from the map. CompletableFuture existingFuture = ledgers.get(name); @@ -325,7 +326,7 @@ public void asyncOpen(final String name, final ManagedLedgerConfig config, final new EnsemblePlacementPolicyConfig(config.getBookKeeperEnsemblePlacementPolicyClassName(), config.getBookKeeperEnsemblePlacementPolicyProperties())), store, config, scheduledExecutor, - orderedExecutor, name); + orderedExecutor, name, mlOwnershipChecker); newledger.initialize(new ManagedLedgerInitializeLedgerCallback() { @Override public void initializeComplete() { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index a1d41b6c6a000..38ed3ea12995b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -65,6 +65,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.function.Predicate; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.bookkeeper.client.AsyncCallback.CreateCallback; @@ -188,6 +189,7 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback { private static final Random random = new Random(System.currentTimeMillis()); private long maximumRolloverTimeMs; + protected final Supplier mlOwnershipChecker; volatile PositionImpl lastConfirmedEntry; @@ -252,6 +254,11 @@ public enum PositionBound { public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, OrderedExecutor orderedExecutor, final String name) { + this(factory, bookKeeper, store, config, scheduledExecutor, orderedExecutor, name, null); + } + public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, + ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, OrderedExecutor orderedExecutor, + final String name, final Supplier mlOwnershipChecker) { this.factory = factory; this.bookKeeper = bookKeeper; this.config = config; @@ -275,6 +282,7 @@ public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper // Get the next rollover time. Add a random value upto 5% to avoid rollover multiple ledgers at the same time this.maximumRolloverTimeMs = (long) (config.getMaximumRolloverTimeMs() * (1 + random.nextDouble() * 5 / 100.0)); + this.mlOwnershipChecker = mlOwnershipChecker; } synchronized void initialize(final ManagedLedgerInitializeLedgerCallback callback, final Object ctx) { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index a31416f8dbd03..d38196317857c 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -96,11 +96,13 @@ import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.VoidCallback; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.commons.lang3.mutable.MutableObject; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition; @@ -116,6 +118,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; public class ManagedLedgerTest extends MockedBookKeeperTestCase { @@ -124,6 +127,11 @@ public class ManagedLedgerTest extends MockedBookKeeperTestCase { private static final Charset Encoding = Charsets.UTF_8; + @DataProvider(name = "checkOwnershipFlag") + public Object[][] checkOwnershipFlagProvider() { + return new Object[][] { { Boolean.TRUE }, { Boolean.FALSE } }; + } + @Test public void managedLedgerApi() throws Exception { ManagedLedger ledger = factory.open("my_test_ledger"); @@ -355,7 +363,7 @@ public void openCursorFailed(ManagedLedgerException exception, Object ctx) { public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { fail(exception.getMessage()); } - }, null); + }, null, null); counter.await(); @@ -1980,7 +1988,7 @@ public void testActiveDeactiveCursorWithDiscardEntriesFromCache() throws Excepti } // (3) Validate: cache should remove all entries read by both active cursors - log.info("expected, found : {}, {}", (5 * (totalInsertedEntries)), entryCache.getSize()); + log.info("expected, found : {}, {}", (5 * (totalInsertedEntries)), entryCache.getSize()); assertEquals((5 * totalInsertedEntries), entryCache.getSize()); final int remainingEntries = totalInsertedEntries - readEntries; @@ -2528,6 +2536,100 @@ public void avoidUseSameOpAddEntryBetweenDifferentLedger() throws Exception { } } + /** + * It verifies that managed-cursor can recover metadata-version if it fails to update due to version conflict. This + * test verifies that version recovery happens if checkOwnership supplier is passed while creating managed-ledger. + * + * @param checkOwnershipFlag + * @throws Exception + */ + @Test(dataProvider = "checkOwnershipFlag") + public void recoverMLWithBadVersion(boolean checkOwnershipFlag) throws Exception { + + ManagedLedgerFactoryConfig conf = new ManagedLedgerFactoryConfig(); + ManagedLedgerFactoryImpl factory1 = new ManagedLedgerFactoryImpl(bkc, zkc, conf); + ManagedLedgerFactoryImpl factory2 = new ManagedLedgerFactoryImpl(bkc, zkc, conf); + + final MutableObject ledger1 = new MutableObject<>(), ledger2 = new MutableObject<>(); + final MutableObject cursor1 = new MutableObject<>(), cursor2 = new MutableObject<>(); + + createLedger(factory1, ledger1, cursor1, checkOwnershipFlag); + ledger1.getValue().addEntry("test1".getBytes(Encoding)); + ledger1.getValue().addEntry("test2".getBytes(Encoding)); + Entry entry = cursor1.getValue().readEntries(1).get(0); + cursor1.getValue().delete(entry.getPosition()); + + createLedger(factory2, ledger2, cursor2, checkOwnershipFlag); + entry = cursor2.getValue().readEntries(1).get(0); + + // 1. closing cursor will change the zk-version + cursor1.getValue().close(); + + // 2. try to creatCursorLedger which should fail first time because of BadVersionException + // However, if checkOwnershipFlag is eanbled the managed-cursor will reover from that exception. + boolean isFailed = updateCusorMetadataByCreatingMetadataLedger(cursor2); + Assert.assertTrue(isFailed); + + isFailed = updateCusorMetadataByCreatingMetadataLedger(cursor2); + if (checkOwnershipFlag) { + Assert.assertFalse(isFailed); + } else { + Assert.assertTrue(isFailed); + } + + log.info("Test completed"); + } + + private boolean updateCusorMetadataByCreatingMetadataLedger(MutableObject cursor2) + throws InterruptedException { + MutableObject failed = new MutableObject<>(); + failed.setValue(false); + CountDownLatch createLedgerDoneLatch = new CountDownLatch(1); + cursor2.getValue().createNewMetadataLedger(new VoidCallback() { + + @Override + public void operationComplete() { + createLedgerDoneLatch.countDown(); + } + + @Override + public void operationFailed(ManagedLedgerException exception) { + failed.setValue(true); + createLedgerDoneLatch.countDown(); + } + + }); + createLedgerDoneLatch.await(); + return failed.getValue(); + } + + private void createLedger(ManagedLedgerFactoryImpl factory, MutableObject ledger1, + MutableObject cursor1, boolean checkOwnershipFlag) throws Exception { + CountDownLatch latch = new CountDownLatch(1); + factory.asyncOpen("my_test_ledger", new ManagedLedgerConfig(), new OpenLedgerCallback() { + @Override + public void openLedgerComplete(ManagedLedger ledger, Object ctx) { + ledger1.setValue(ledger); + ledger.asyncOpenCursor("test-cursor", new OpenCursorCallback() { + @Override + public void openCursorComplete(ManagedCursor cursor, Object ctx) { + cursor1.setValue((ManagedCursorImpl) cursor); + latch.countDown(); + } + + @Override + public void openCursorFailed(ManagedLedgerException exception, Object ctx) { + } + }, null); + } + + @Override + public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { + } + }, checkOwnershipFlag ? () -> true : null, null); + latch.await(); + } + private void setFieldValue(Class clazz, Object classObj, String fieldName, Object fieldValue) throws Exception { Field field = clazz.getDeclaredField(fieldName); field.setAccessible(true); @@ -2543,4 +2645,4 @@ public static void retryStrategically(Predicate predicate, int retryCount, Thread.sleep(intSleepTimeInMillis + (intSleepTimeInMillis * i)); } } -} +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 498f1ea504b5d..a050b47d350b9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -916,7 +916,7 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { topicFuture.completeExceptionally(new PersistenceException(exception)); } } - }, null); + }, () -> isTopicNsOwnedByBroker(topicName), null); }).exceptionally((exception) -> { log.warn("[{}] Failed to get topic configuration: {}", topic, exception.getMessage(), exception); @@ -1217,6 +1217,15 @@ public void monitorBacklogQuota() { }); } + public boolean isTopicNsOwnedByBroker(TopicName topicName) throws RuntimeException { + try { + return pulsar.getNamespaceService().isServiceUnitOwned(topicName); + } catch (Exception e) { + log.warn("Failed to check the ownership of the topic: {}, {}", topicName, e.getMessage()); + } + return false; + } + public void checkTopicNsOwnership(final String topic) throws RuntimeException { TopicName topicName = TopicName.get(topic); boolean ownedByThisInstance; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java index 62c8aff9cfa28..2b4976bc0713b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentDispatcherFailoverConsumerTest.java @@ -42,6 +42,7 @@ import java.lang.reflect.Field; import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.function.Supplier; import java.util.Collections; import java.util.List; import java.util.Optional; @@ -208,7 +209,7 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { return null; } }).when(mlFactoryMock).asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), - any(OpenLedgerCallback.class), any()); + any(OpenLedgerCallback.class), any(Supplier.class), any()); // call openLedgerFailed on ML factory asyncOpen doAnswer(new Answer() { @@ -219,7 +220,7 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { return null; } }).when(mlFactoryMock).asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), - any(OpenLedgerCallback.class), any()); + any(OpenLedgerCallback.class), any(Supplier.class), any()); // call addComplete on ledger asyncAddEntry doAnswer(new Answer() { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java index bc92ee492c230..d717a5a9e7b7a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java @@ -21,6 +21,7 @@ import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.createMockBookKeeper; import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.createMockZooKeeper; import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.eq; @@ -59,6 +60,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback; @@ -223,7 +225,7 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { return null; } }).when(mlFactoryMock).asyncOpen(anyString(), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), - any()); + any(Supplier.class), any()); CompletableFuture future = brokerService.getOrCreateTopic(topicName).thenAccept(topic -> { assertTrue(topic.toString().contains(topicName)); @@ -254,7 +256,7 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { return null; } }).when(mlFactoryMock).asyncOpen(anyString(), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), - any()); + any(Supplier.class), any()); CompletableFuture future = brokerService.getOrCreateTopic(jinxedTopicName); @@ -1095,7 +1097,7 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { return null; } }).when(mlFactoryMock).asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), - any(OpenLedgerCallback.class), any()); + any(OpenLedgerCallback.class), any(Supplier.class), any()); // call openLedgerFailed on ML factory asyncOpen doAnswer(new Answer() { @@ -1106,7 +1108,7 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { return null; } }).when(mlFactoryMock).asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), - any(OpenLedgerCallback.class), any()); + any(OpenLedgerCallback.class), any(Supplier.class), any()); // call addComplete on ledger asyncAddEntry doAnswer(new Answer() { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java index f01f771149a1c..ba4cf5bee2815 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java @@ -21,6 +21,7 @@ import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.createMockBookKeeper; import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.createMockZooKeeper; import static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.any; import static org.mockito.Mockito.matches; import static org.mockito.Mockito.doAnswer; @@ -50,6 +51,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback; @@ -767,7 +769,7 @@ public void testCreateProducerTimeout() throws Exception { }); return null; }).when(mlFactoryMock).asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), - any(OpenLedgerCallback.class), any()); + any(OpenLedgerCallback.class), any(Supplier.class), any()); // In a create producer timeout from client side we expect to see this sequence of commands : // 1. create producer @@ -824,7 +826,7 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { return null; } }).when(mlFactoryMock).asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), - any(OpenLedgerCallback.class), any()); + any(OpenLedgerCallback.class), any(Supplier.class), any()); // In a create producer timeout from client side we expect to see this sequence of commands : // 1. create producer @@ -902,7 +904,7 @@ public void testCreateProducerBookieTimeout() throws Exception { }); return null; }).when(mlFactoryMock).asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), - any(OpenLedgerCallback.class), any()); + any(OpenLedgerCallback.class), any(Supplier.class), any()); // In a create producer timeout from client side we expect to see this sequence of commands : // 1. create a failure producer which will timeout creation after 100msec @@ -973,7 +975,7 @@ public void testSubscribeTimeout() throws Exception { return null; }).when(mlFactoryMock).asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), - any(OpenLedgerCallback.class), any()); + any(OpenLedgerCallback.class), any(Supplier.class), any()); // In a subscribe timeout from client side we expect to see this sequence of commands : // 1. Subscribe @@ -1046,7 +1048,7 @@ public void testSubscribeBookieTimeout() throws Exception { }); return null; }).when(mlFactoryMock).asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), - any(OpenLedgerCallback.class), any()); + any(OpenLedgerCallback.class), any(Supplier.class), any()); CompletableFuture openTopicFail = new CompletableFuture<>(); doAnswer(invocationOnMock -> { @@ -1056,7 +1058,7 @@ public void testSubscribeBookieTimeout() throws Exception { }); return null; }).when(mlFactoryMock).asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), - any(OpenLedgerCallback.class), any()); + any(OpenLedgerCallback.class), any(Supplier.class), any()); // In a subscribe timeout from client side we expect to see this sequence of commands : // 1. Subscribe against failtopic which will fail after 100msec @@ -1455,7 +1457,7 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { return null; } }).when(mlFactoryMock).asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), - any(OpenLedgerCallback.class), any()); + any(OpenLedgerCallback.class), any(Supplier.class), any()); // call openLedgerFailed on ML factory asyncOpen doAnswer(new Answer() { @@ -1470,7 +1472,7 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { return null; } }).when(mlFactoryMock).asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), - any(OpenLedgerCallback.class), any()); + any(OpenLedgerCallback.class), any(Supplier.class), any()); // call addComplete on ledger asyncAddEntry doAnswer(new Answer() { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/PersistentTransactionBufferTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/PersistentTransactionBufferTest.java index 75781745e6e60..4bb508c925b4c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/PersistentTransactionBufferTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/PersistentTransactionBufferTest.java @@ -45,6 +45,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.bookkeeper.client.PulsarMockBookKeeper; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; @@ -246,7 +247,8 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { return null; } }).when(mlFactoryMock) - .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any()); + .asyncOpen(matches(".*success.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), + any(Supplier.class), any()); // call openLedgerFailed on ML factory asyncOpen doAnswer(new Answer() { @@ -257,7 +259,8 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { return null; } }).when(mlFactoryMock) - .asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), any()); + .asyncOpen(matches(".*fail.*"), any(ManagedLedgerConfig.class), any(OpenLedgerCallback.class), + any(Supplier.class), any()); // call addComplete on ledger asyncAddEntry doAnswer(new Answer() { diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ManagedLedgerWriter.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ManagedLedgerWriter.java index 00bdf0320fadb..7cb36330a5a8a 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ManagedLedgerWriter.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/ManagedLedgerWriter.java @@ -197,7 +197,7 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { future.completeExceptionally(exception); } - }, null); + }, null, null); } List managedLedgers = futures.stream().map(CompletableFuture::join).collect(Collectors.toList());