From fa1b14a21569afcc4ff9327625f82cd6f5ba3ecf Mon Sep 17 00:00:00 2001 From: Masahiro Sakamoto Date: Wed, 28 Jul 2021 16:38:17 +0900 Subject: [PATCH] [stats] Do not expose meaningless stats for publisher (#11454) Currently, publisher stats includes some fields that are meaningless for users. ``` count producerNameOffset producerNameLength addressOffset addressLength connectedSinceOffset connectedSinceLength clientVersionOffset clientVersionLength stringBuffer ``` These cause the size of the json data to grow and should not be exposed. Add the `@JsonIgnore` annotation to the above fields. This is a modification similar to https://github.com/apache/pulsar/pull/11005. (cherry picked from commit d3c44ba8d1b0fad7518077708f6211dd10925878) --- .../broker/stats/ConsumerStatsTest.java | 13 +- .../data/stats/PublisherStatsImpl.java | 11 ++ .../policies/data/stats/TopicStatsImpl.java | 2 + .../policies/data/PublisherStatsTest.java | 32 +++- site2/docs/getting-started-docker.md | 154 +++++++++++------- 5 files changed, 148 insertions(+), 64 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java index f9eaa1b1b511a..2f702ab89f645 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/ConsumerStatsTest.java @@ -38,6 +38,7 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -193,7 +194,10 @@ public void testConsumerStatsOutput() throws Exception { "lastAckedTimestamp", "lastConsumedTimestamp", "keyHashRanges", - "metadata"); + "metadata", + "address", + "connectedSince", + "clientVersion"); final String topicName = "persistent://prop/use/ns-abc/testConsumerStatsOutput"; final String subName = "my-subscription"; @@ -208,9 +212,10 @@ public void testConsumerStatsOutput() throws Exception { ObjectMapper mapper = ObjectMapperFactory.create(); JsonNode node = mapper.readTree(mapper.writer().writeValueAsString(stats.getSubscriptions() .get(subName).getConsumers().get(0))); - if (node.fieldNames().hasNext()) { - String field = node.fieldNames().next(); - Assert.assertTrue(allowedFields.contains(field)); + Iterator itr = node.fieldNames(); + while (itr.hasNext()) { + String field = itr.next(); + Assert.assertTrue(allowedFields.contains(field), field + " should not be exposed"); } consumer.close(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PublisherStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PublisherStatsImpl.java index 683b1eac0ab9f..887bae1722ca2 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PublisherStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/PublisherStatsImpl.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.common.policies.data.stats; +import com.fasterxml.jackson.annotation.JsonIgnore; import lombok.Data; import org.apache.pulsar.client.api.ProducerAccessMode; import org.apache.pulsar.common.policies.data.PublisherStats; @@ -28,6 +29,7 @@ */ @Data public class PublisherStatsImpl implements PublisherStats { + @JsonIgnore private int count; public ProducerAccessMode accessMode; @@ -48,25 +50,34 @@ public class PublisherStatsImpl implements PublisherStats { public long producerId; /** Producer name. */ + @JsonIgnore private int producerNameOffset = -1; + @JsonIgnore private int producerNameLength; /** Address of this publisher. */ + @JsonIgnore private int addressOffset = -1; + @JsonIgnore private int addressLength; /** Timestamp of connection. */ + @JsonIgnore private int connectedSinceOffset = -1; + @JsonIgnore private int connectedSinceLength; /** Client library version. */ + @JsonIgnore private int clientVersionOffset = -1; + @JsonIgnore private int clientVersionLength; /** * In order to prevent multiple string objects under stats: create a string-buffer that stores data for all string * place-holders. */ + @JsonIgnore private StringBuilder stringBuffer = new StringBuilder(); /** Metadata (key/value strings) associated with this publisher. */ diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java index 941c318b70195..ec97d6e35c851 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.common.policies.data.stats; +import com.fasterxml.jackson.annotation.JsonIgnore; import lombok.AccessLevel; import lombok.Data; import lombok.Getter; @@ -37,6 +38,7 @@ */ @Data public class TopicStatsImpl implements TopicStats { + @JsonIgnore private int count; /** Total rate of messages published on the topic (msg/s). */ diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PublisherStatsTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PublisherStatsTest.java index 6bc8771127bb9..9f34a9128e48f 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PublisherStatsTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/PublisherStatsTest.java @@ -20,14 +20,35 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Sets; +import java.util.Iterator; +import java.util.Set; import org.apache.pulsar.common.policies.data.stats.PublisherStatsImpl; +import org.apache.pulsar.common.util.ObjectMapperFactory; import org.testng.annotations.Test; public class PublisherStatsTest { @Test - public void testPublisherStats() { + public void testPublisherStats() throws Exception { + Set allowedFields = Sets.newHashSet( + "accessMode", + "msgRateIn", + "msgThroughputIn", + "averageMsgSize", + "chunkedMessageRate", + "producerId", + "metadata", + "address", + "connectedSince", + "clientVersion", + "producerName" + ); + PublisherStatsImpl stats = new PublisherStatsImpl(); assertNull(stats.getAddress()); assertNull(stats.getClientVersion()); @@ -53,6 +74,15 @@ public void testPublisherStats() { assertEquals(stats.getConnectedSince(), "connected"); assertEquals(stats.getAddress(), "address1"); assertEquals(stats.getClientVersion(), "version"); + + // Check if private fields are included in json + ObjectMapper mapper = ObjectMapperFactory.create(); + JsonNode node = mapper.readTree(mapper.writer().writeValueAsString(stats)); + Iterator itr = node.fieldNames(); + while (itr.hasNext()) { + String field = itr.next(); + assertTrue(allowedFields.contains(field), field + " should not be exposed"); + } stats.setAddress(null); assertNull(stats.getAddress()); diff --git a/site2/docs/getting-started-docker.md b/site2/docs/getting-started-docker.md index 56cff28df57a9..e5125ae7d21fc 100644 --- a/site2/docs/getting-started-docker.md +++ b/site2/docs/getting-started-docker.md @@ -15,44 +15,39 @@ and follow the instructions for your OS. * For MacOS, Linux, and Windows: ```shell - $ docker run -it \ - -p 6650:6650 \ - -p 8080:8080 \ - --mount source=pulsardata,target=/pulsar/data \ - --mount source=pulsarconf,target=/pulsar/conf \ - apachepulsar/pulsar:{{pulsar:version}} \ - bin/pulsar standalone + $ docker run -it -p 6650:6650 -p 8080:8080 --mount source=pulsardata,target=/pulsar/data --mount source=pulsarconf,target=/pulsar/conf apachepulsar/pulsar:{{pulsar:version}} bin/pulsar standalone ``` A few things to note about this command: - * The data, metadata, and configuration are persisted on Docker volumes in order to not start "fresh" every -time the container is restarted. For details on the volumes you can use `docker volume inspect ` - * For Docker on Windows make sure to configure it to use Linux containers +* The data, metadata, and configuration are persisted on Docker volumes in order to not start "fresh" every + time the container is restarted. For details on the volumes you can use `docker volume inspect ` +* For Docker on Windows make sure to configure it to use Linux containers If you start Pulsar successfully, you will see `INFO`-level log messages like this: ``` -2017-08-09 22:34:04,030 - INFO - [main:WebService@213] - Web Service started at http://127.0.0.1:8080 -2017-08-09 22:34:04,038 - INFO - [main:PulsarService@335] - messaging service is ready, bootstrap service on port=8080, broker url=pulsar://127.0.0.1:6650, cluster=standalone, configs=org.apache.pulsar.broker.ServiceConfiguration@4db60246 +08:18:30.970 [main] INFO org.apache.pulsar.broker.web.WebService - HTTP Service started at http://0.0.0.0:8080 +... +07:53:37.322 [main] INFO org.apache.pulsar.broker.PulsarService - messaging service is ready, bootstrap service port = 8080, broker url= pulsar://localhost:6650, cluster=standalone, configs=org.apache.pulsar.broker.ServiceConfiguration@98b63c1 ... ``` > #### Tip -> +> > When you start a local standalone cluster, a `public/default` namespace is created automatically. The namespace is used for development purposes. All Pulsar topics are managed within namespaces. For more information, see [Topics](concepts-messaging.md#topics). ## Use Pulsar in Docker -Pulsar offers client libraries for [Java](client-libraries-java.md), [Go](client-libraries-go.md), [Python](client-libraries-python.md) +Pulsar offers client libraries for [Java](client-libraries-java.md), [Go](client-libraries-go.md), [Python](client-libraries-python.md) and [C++](client-libraries-cpp.md). If you're running a local standalone cluster, you can use one of these root URLs to interact with your cluster: * `pulsar://localhost:6650` * `http://localhost:8080` -The following example will guide you get started with Pulsar quickly by using the [Python](client-libraries-python.md) +The following example will guide you get started with Pulsar quickly by using the [Python client API](client-libraries-python.md) client API. Install the Pulsar Python client library directly from [PyPI](https://pypi.org/project/pulsar-client/): @@ -111,50 +106,91 @@ The output is something like this: ```json { - "averageMsgSize": 0.0, - "msgRateIn": 0.0, - "msgRateOut": 0.0, - "msgThroughputIn": 0.0, - "msgThroughputOut": 0.0, - "publishers": [ - { - "address": "/172.17.0.1:35048", - "averageMsgSize": 0.0, - "clientVersion": "1.19.0-incubating", - "connectedSince": "2017-08-09 20:59:34.621+0000", - "msgRateIn": 0.0, - "msgThroughputIn": 0.0, - "producerId": 0, - "producerName": "standalone-0-1" - } - ], - "replication": {}, - "storageSize": 16, - "subscriptions": { - "my-sub": { - "blockedSubscriptionOnUnackedMsgs": false, - "consumers": [ + "msgRateIn": 0.0, + "msgThroughputIn": 0.0, + "msgRateOut": 1.8332950480217471, + "msgThroughputOut": 91.33142602871978, + "bytesInCounter": 7097, + "msgInCounter": 143, + "bytesOutCounter": 6607, + "msgOutCounter": 133, + "averageMsgSize": 0.0, + "msgChunkPublished": false, + "storageSize": 7097, + "backlogSize": 0, + "offloadedStorageSize": 0, + "publishers": [ { - "address": "/172.17.0.1:35064", - "availablePermits": 996, - "blockedConsumerOnUnackedMsgs": false, - "clientVersion": "1.19.0-incubating", - "connectedSince": "2017-08-09 21:05:39.222+0000", - "consumerName": "166111", - "msgRateOut": 0.0, - "msgRateRedeliver": 0.0, - "msgThroughputOut": 0.0, - "unackedMessages": 0 + "accessMode": "Shared", + "msgRateIn": 0.0, + "msgThroughputIn": 0.0, + "averageMsgSize": 0.0, + "chunkedMessageRate": 0.0, + "producerId": 0, + "metadata": {}, + "address": "/127.0.0.1:35604", + "connectedSince": "2021-07-04T09:05:43.04788Z", + "clientVersion": "2.8.0", + "producerName": "standalone-2-5" + } + ], + "waitingPublishers": 0, + "subscriptions": { + "my-sub": { + "msgRateOut": 1.8332950480217471, + "msgThroughputOut": 91.33142602871978, + "bytesOutCounter": 6607, + "msgOutCounter": 133, + "msgRateRedeliver": 0.0, + "chunkedMessageRate": 0, + "msgBacklog": 0, + "backlogSize": 0, + "msgBacklogNoDelayed": 0, + "blockedSubscriptionOnUnackedMsgs": false, + "msgDelayed": 0, + "unackedMessages": 0, + "type": "Exclusive", + "activeConsumerName": "3c544f1daa", + "msgRateExpired": 0.0, + "totalMsgExpired": 0, + "lastExpireTimestamp": 0, + "lastConsumedFlowTimestamp": 1625389101290, + "lastConsumedTimestamp": 1625389546070, + "lastAckedTimestamp": 1625389546162, + "lastMarkDeleteAdvancedTimestamp": 1625389546163, + "consumers": [ + { + "msgRateOut": 1.8332950480217471, + "msgThroughputOut": 91.33142602871978, + "bytesOutCounter": 6607, + "msgOutCounter": 133, + "msgRateRedeliver": 0.0, + "chunkedMessageRate": 0.0, + "consumerName": "3c544f1daa", + "availablePermits": 867, + "unackedMessages": 0, + "avgMessagesPerEntry": 6, + "blockedConsumerOnUnackedMsgs": false, + "lastAckedTimestamp": 1625389546162, + "lastConsumedTimestamp": 1625389546070, + "metadata": {}, + "address": "/127.0.0.1:35472", + "connectedSince": "2021-07-04T08:58:21.287682Z", + "clientVersion": "2.8.0" + } + ], + "isDurable": true, + "isReplicated": false, + "consumersAfterMarkDeletePosition": {}, + "nonContiguousDeletedMessagesRanges": 0, + "nonContiguousDeletedMessagesRangesSerializedSize": 0, + "durable": true, + "replicated": false } - ], - "msgBacklog": 0, - "msgRateExpired": 0.0, - "msgRateOut": 0.0, - "msgRateRedeliver": 0.0, - "msgThroughputOut": 0.0, - "type": "Exclusive", - "unackedMessages": 0 - } - } + }, + "replication": {}, + "deduplicationStatus": "Disabled", + "nonContiguousDeletedMessagesRanges": 0, + "nonContiguousDeletedMessagesRangesSerializedSize": 0 } -``` +``` \ No newline at end of file