diff --git a/.github/workflows/ci-cpp-build-windows.yaml b/.github/workflows/ci-cpp-build-windows.yaml index c524fdc4b0ca60..a287a4c3560e11 100644 --- a/.github/workflows/ci-cpp-build-windows.yaml +++ b/.github/workflows/ci-cpp-build-windows.yaml @@ -101,7 +101,7 @@ jobs: run: | if [ "$RUNNER_OS" == "Windows" ]; then cd pulsar-client-cpp && \ - cmake --build ./build + cmake --build ./build --config Release fi - name: Configure (dynamic library only) @@ -126,5 +126,5 @@ jobs: run: | if [ "$RUNNER_OS" == "Windows" ]; then cd pulsar-client-cpp && \ - cmake --build ./build-1 + cmake --build ./build-1 --config Release fi diff --git a/README.md b/README.md index 0eb77b67f04392..5c50afbef7fcf6 100644 --- a/README.md +++ b/README.md @@ -88,6 +88,7 @@ components in the Pulsar ecosystem, including connectors, adapters, and other la Requirements: * Java [JDK 11](https://adoptopenjdk.net/?variant=openjdk11) or [JDK 8](https://adoptopenjdk.net/?variant=openjdk8) * Maven 3.6.1+ + * zip Compile and install: 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 e3148b9986f086..3c9e3b492de8a9 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 @@ -2289,19 +2289,14 @@ private void maybeOffload(CompletableFuture finalPromise) { } private boolean hasLedgerRetentionExpired(long ledgerTimestamp) { - if (config.getRetentionTimeMillis() < 0) { - // Negative retention time equates to infinite retention - return false; - } - - long elapsedMs = clock.millis() - ledgerTimestamp; - return elapsedMs > config.getRetentionTimeMillis(); + return config.getRetentionTimeMillis() >= 0 + && clock.millis() - ledgerTimestamp > config.getRetentionTimeMillis(); } - private boolean isLedgerRetentionOverSizeQuota() { + private boolean isLedgerRetentionOverSizeQuota(long sizeToDelete) { // Handle the -1 size limit as "infinite" size quota return config.getRetentionSizeInMB() >= 0 - && TOTAL_SIZE_UPDATER.get(this) > config.getRetentionSizeInMB() * 1024 * 1024; + && TOTAL_SIZE_UPDATER.get(this) - sizeToDelete >= config.getRetentionSizeInMB() * MegaByte; } private boolean isOffloadedNeedsDelete(OffloadContext offload) { @@ -2369,11 +2364,32 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { if (log.isDebugEnabled()) { log.debug("[{}] Slowest consumer ledger id: {}", name, slowestReaderLedgerId); } + + long totalSizeToDelete = 0; + boolean retentionSizeQuotaMet = false; // skip ledger if retention constraint met for (LedgerInfo ls : ledgers.headMap(slowestReaderLedgerId, false).values()) { - boolean expired = hasLedgerRetentionExpired(ls.getTimestamp()); - boolean overRetentionQuota = isLedgerRetentionOverSizeQuota(); + // currentLedger can not be deleted + if (ls.getLedgerId() == currentLedger.getId()) { + if (log.isDebugEnabled()) { + log.debug("[{}] Ledger {} skipped for deletion as it is currently being written to", name, + ls.getLedgerId()); + } + break; + } + // if truncate, all ledgers besides currentLedger are going to be deleted + if (isTruncate){ + if (log.isDebugEnabled()) { + log.debug("[{}] Ledger {} will be truncated with ts {}", + name, ls.getLedgerId(), ls.getTimestamp()); + } + ledgersToDelete.add(ls); + continue; + } + totalSizeToDelete += ls.getSize(); + boolean overRetentionQuota = isLedgerRetentionOverSizeQuota(totalSizeToDelete); + boolean expired = hasLedgerRetentionExpired(ls.getTimestamp()); if (log.isDebugEnabled()) { log.debug( "[{}] Checking ledger {} -- time-old: {} sec -- " @@ -2381,21 +2397,23 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { name, ls.getLedgerId(), (clock.millis() - ls.getTimestamp()) / 1000.0, expired, overRetentionQuota, currentLedger.getId()); } - if (ls.getLedgerId() == currentLedger.getId()) { - log.debug("[{}] Ledger {} skipped for deletion as it is currently being written to", name, - ls.getLedgerId()); - break; - } else if (expired || isTruncate) { - log.debug("[{}] Ledger {} has expired or be truncated, expired is {}, isTruncate is {}, ts {}", name, ls.getLedgerId(), expired, isTruncate, ls.getTimestamp()); - ledgersToDelete.add(ls); - } else if (overRetentionQuota || isTruncate) { - log.debug("[{}] Ledger {} is over quota or be truncated, overRetentionQuota is {}, isTruncate is {}", name, ls.getLedgerId(), overRetentionQuota, isTruncate); + + if (expired || overRetentionQuota) { + if (log.isDebugEnabled()) { + log.debug("[{}] Ledger {} has expired or over quota, expired is: {}, ts: {}, " + + "overRetentionQuota is: {}, ledge size: {}", + name, ls.getLedgerId(), expired, ls.getTimestamp(), overRetentionQuota, ls.getSize()); + } ledgersToDelete.add(ls); } else { - log.debug("[{}] Ledger {} not deleted. Neither expired nor over-quota", name, ls.getLedgerId()); + // once retention constraint has been met, skip check + if (log.isDebugEnabled()) { + log.debug("[{}] Ledger {} not deleted. Neither expired nor over-quota", name, ls.getLedgerId()); + } break; } } + for (LedgerInfo ls : ledgers.values()) { if (isOffloadedNeedsDelete(ls.getOffloadContext()) && !ledgersToDelete.contains(ls)) { log.debug("[{}] Ledger {} has been offloaded, bookkeeper ledger needs to be deleted", name, 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 62b44b8c091365..25c1725bdb4cc5 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 @@ -44,6 +44,7 @@ import java.nio.charset.Charset; import java.security.GeneralSecurityException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -1970,6 +1971,49 @@ public void testInfiniteRetention() throws Exception { assertTrue(ml.getTotalSize() > "shortmessage".getBytes().length); } + @Test + public void testRetentionSize() throws Exception { + final int retentionSizeInMB = 5; + final int totalMessage = 10; + + // message size is 1MB + final int messageSize = 1048576; + char[] data = new char[messageSize]; + Arrays.fill(data, 'a'); + byte [] message = new String(data).getBytes(Encoding); + + @Cleanup("shutdown") + ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setRetentionSizeInMB(retentionSizeInMB); + config.setMaxEntriesPerLedger(1); + config.setRetentionTime(1, TimeUnit.HOURS); + + + ManagedLedgerImpl ml = (ManagedLedgerImpl) factory.open("retention_size_ledger", config); + ManagedCursor c1 = ml.openCursor("c1"); + Position position = null; + for (int i = 0; i < totalMessage; i++) { + position = ml.addEntry(message); + } + // all ledgers are not delete yet since no entry has been acked for c1 + assertEquals(ml.getLedgersInfoAsList().size(), totalMessage); + + List entryList = c1.readEntries(totalMessage); + if (null != position) { + c1.markDelete(position); + } + entryList.forEach(entry -> { + log.info("Read entry position {}:{}", entry.getLedgerId(), entry.getEntryId()); + entry.release(); + }); + + Awaitility.await().untilAsserted(() -> { + assertTrue(ml.getTotalSize() <= retentionSizeInMB * 1024 * 1024); + assertEquals(ml.getLedgersInfoAsList().size(), 5); + }); + } + @Test public void testTimestampOnWorkingLedger() throws Exception { ManagedLedgerConfig conf = new ManagedLedgerConfig(); diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java index 8016bcef314728..9f756fcce01dd7 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BaseResources.java @@ -27,8 +27,8 @@ import java.util.function.Function; import lombok.Getter; import org.apache.pulsar.metadata.api.MetadataCache; +import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreException; -import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; /** * Base class for all configuration resources to access configurations from metadata-store. @@ -39,18 +39,18 @@ public class BaseResources { @Getter - private final MetadataStoreExtended store; + private final MetadataStore store; @Getter private final MetadataCache cache; private int operationTimeoutSec; - public BaseResources(MetadataStoreExtended store, Class clazz, int operationTimeoutSec) { + public BaseResources(MetadataStore store, Class clazz, int operationTimeoutSec) { this.store = store; this.cache = store.getMetadataCache(clazz); this.operationTimeoutSec = operationTimeoutSec; } - public BaseResources(MetadataStoreExtended store, TypeReference typeRef, int operationTimeoutSec) { + public BaseResources(MetadataStore store, TypeReference typeRef, int operationTimeoutSec) { this.store = store; this.cache = store.getMetadataCache(typeRef); this.operationTimeoutSec = operationTimeoutSec; diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BookieResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BookieResources.java index 219099999db907..2b911ac04c93c2 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BookieResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/BookieResources.java @@ -22,12 +22,12 @@ import java.util.concurrent.CompletableFuture; import java.util.function.Function; import org.apache.pulsar.common.policies.data.BookiesRackConfiguration; -import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping; public class BookieResources extends BaseResources { - public BookieResources(MetadataStoreExtended store, int operationTimeoutSec) { + public BookieResources(MetadataStore store, int operationTimeoutSec) { super(store, BookiesRackConfiguration.class, operationTimeoutSec); } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ClusterResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ClusterResources.java index a92ecf16f76d75..c5bfde772611d1 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ClusterResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ClusterResources.java @@ -21,12 +21,10 @@ import java.util.HashSet; import java.util.Set; import lombok.Getter; - import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.apache.pulsar.common.policies.data.FailureDomainImpl; +import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreException; -import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; public class ClusterResources extends BaseResources { @@ -34,7 +32,7 @@ public class ClusterResources extends BaseResources { @Getter private FailureDomainResources failureDomainResources; - public ClusterResources(MetadataStoreExtended store, int operationTimeoutSec) { + public ClusterResources(MetadataStore store, int operationTimeoutSec) { super(store, ClusterData.class, operationTimeoutSec); this.failureDomainResources = new FailureDomainResources(store, FailureDomainImpl.class, operationTimeoutSec); } @@ -46,7 +44,7 @@ public Set list() throws MetadataStoreException { public static class FailureDomainResources extends BaseResources { public static final String FAILURE_DOMAIN = "failureDomain"; - public FailureDomainResources(MetadataStoreExtended store, Class clazz, + public FailureDomainResources(MetadataStore store, Class clazz, int operationTimeoutSec) { super(store, clazz, operationTimeoutSec); } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/DynamicConfigurationResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/DynamicConfigurationResources.java index c95410207ef1fa..ce6e7979947eca 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/DynamicConfigurationResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/DynamicConfigurationResources.java @@ -20,12 +20,11 @@ import com.fasterxml.jackson.core.type.TypeReference; import java.util.Map; - -import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.api.MetadataStore; public class DynamicConfigurationResources extends BaseResources> { - public DynamicConfigurationResources(MetadataStoreExtended store, int operationTimeoutSec) { + public DynamicConfigurationResources(MetadataStore store, int operationTimeoutSec) { super(store, new TypeReference>() { }, operationTimeoutSec); } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LoadManagerReportResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LoadManagerReportResources.java index d3176bbc6b7628..acc5e23869d12a 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LoadManagerReportResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LoadManagerReportResources.java @@ -18,12 +18,12 @@ */ package org.apache.pulsar.broker.resources; -import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport; public class LoadManagerReportResources extends BaseResources { - public LoadManagerReportResources(MetadataStoreExtended configurationStore, int operationTimeoutSec) { + public LoadManagerReportResources(MetadataStore configurationStore, int operationTimeoutSec) { super(configurationStore, LoadManagerReport.class, operationTimeoutSec); } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LocalPoliciesResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LocalPoliciesResources.java index ebaa38e6b10eb0..dda042b527f19c 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LocalPoliciesResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/LocalPoliciesResources.java @@ -19,11 +19,11 @@ package org.apache.pulsar.broker.resources; import org.apache.pulsar.common.policies.data.LocalPolicies; -import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.api.MetadataStore; public class LocalPoliciesResources extends BaseResources { - public LocalPoliciesResources(MetadataStoreExtended configurationStore, int operationTimeoutSec) { + public LocalPoliciesResources(MetadataStore configurationStore, int operationTimeoutSec) { super(configurationStore, LocalPolicies.class, operationTimeoutSec); } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java index 58d493ee171351..0479bb8571c62b 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java @@ -27,16 +27,16 @@ import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicies; +import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreException; -import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; @Getter public class NamespaceResources extends BaseResources { private IsolationPolicyResources isolationPolicies; private PartitionedTopicResources partitionedTopicResources; - private MetadataStoreExtended configurationStore; + private MetadataStore configurationStore; - public NamespaceResources(MetadataStoreExtended configurationStore, int operationTimeoutSec) { + public NamespaceResources(MetadataStore configurationStore, int operationTimeoutSec) { super(configurationStore, Policies.class, operationTimeoutSec); this.configurationStore = configurationStore; isolationPolicies = new IsolationPolicyResources(configurationStore, operationTimeoutSec); @@ -44,7 +44,7 @@ public NamespaceResources(MetadataStoreExtended configurationStore, int operatio } public static class IsolationPolicyResources extends BaseResources> { - public IsolationPolicyResources(MetadataStoreExtended store, int operationTimeoutSec) { + public IsolationPolicyResources(MetadataStore store, int operationTimeoutSec) { super(store, new TypeReference>() { }, operationTimeoutSec); } @@ -56,7 +56,7 @@ public Optional getPolicies(String path) throws Meta } public static class PartitionedTopicResources extends BaseResources { - public PartitionedTopicResources(MetadataStoreExtended configurationStore, int operationTimeoutSec) { + public PartitionedTopicResources(MetadataStore configurationStore, int operationTimeoutSec) { super(configurationStore, PartitionedTopicMetadata.class, operationTimeoutSec); } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java index 1e82e78b7d8043..a8b96a250ea39c 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java @@ -20,6 +20,7 @@ import java.util.Optional; +import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreConfig; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; @@ -39,14 +40,15 @@ public class PulsarResources { private LocalPoliciesResources localPolicies; private LoadManagerReportResources loadReportResources; private BookieResources bookieResources; + private TopicResources topicResources; - private Optional localMetadataStore; - private Optional configurationMetadataStore; + private Optional localMetadataStore; + private Optional configurationMetadataStore; - public PulsarResources(MetadataStoreExtended localMetadataStore, MetadataStoreExtended configurationMetadataStore) { + public PulsarResources(MetadataStore localMetadataStore, MetadataStore configurationMetadataStore) { this(localMetadataStore, configurationMetadataStore, DEFAULT_OPERATION_TIMEOUT_SEC); } - public PulsarResources(MetadataStoreExtended localMetadataStore, MetadataStoreExtended configurationMetadataStore, + public PulsarResources(MetadataStore localMetadataStore, MetadataStore configurationMetadataStore, int operationTimeoutSec) { if (configurationMetadataStore != null) { tenantResources = new TenantResources(configurationMetadataStore, operationTimeoutSec); @@ -59,6 +61,7 @@ public PulsarResources(MetadataStoreExtended localMetadataStore, MetadataStoreEx localPolicies = new LocalPoliciesResources(localMetadataStore, operationTimeoutSec); loadReportResources = new LoadManagerReportResources(localMetadataStore, operationTimeoutSec); bookieResources = new BookieResources(localMetadataStore, operationTimeoutSec); + topicResources = new TopicResources(localMetadataStore); } this.localMetadataStore = Optional.ofNullable(localMetadataStore); this.configurationMetadataStore = Optional.ofNullable(configurationMetadataStore); diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ResourceGroupResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ResourceGroupResources.java index 7d6bf6ff4c5de2..643b00355fb221 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ResourceGroupResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/ResourceGroupResources.java @@ -19,10 +19,10 @@ package org.apache.pulsar.broker.resources; import org.apache.pulsar.common.policies.data.ResourceGroup; -import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.api.MetadataStore; public class ResourceGroupResources extends BaseResources { - public ResourceGroupResources(MetadataStoreExtended store, int operationTimeoutSec) { + public ResourceGroupResources(MetadataStore store, int operationTimeoutSec) { super(store, ResourceGroup.class, operationTimeoutSec); } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TenantResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TenantResources.java index 127332e1332ea8..86b0397b6a0231 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TenantResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TenantResources.java @@ -19,11 +19,10 @@ package org.apache.pulsar.broker.resources; import org.apache.pulsar.common.policies.data.TenantInfo; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; -import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.api.MetadataStore; public class TenantResources extends BaseResources { - public TenantResources(MetadataStoreExtended store, int operationTimeoutSec) { + public TenantResources(MetadataStore store, int operationTimeoutSec) { super(store, TenantInfo.class, operationTimeoutSec); } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java new file mode 100644 index 00000000000000..37125340fbde6f --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java @@ -0,0 +1,69 @@ +/** + * 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.resources; + +import static org.apache.pulsar.common.util.Codec.decode; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.metadata.api.MetadataStore; + +public class TopicResources { + private static final String MANAGED_LEDGER_PATH = "/managed-ledgers"; + + private final MetadataStore store; + + TopicResources(MetadataStore store) { + this.store = store; + } + + public CompletableFuture> getExistingPartitions(TopicName topic) { + String topicPartitionPath = MANAGED_LEDGER_PATH + "/" + topic.getNamespace() + "/" + + topic.getDomain(); + return store.getChildren(topicPartitionPath).thenApply(topics -> + topics.stream() + .map(s -> String.format("%s://%s/%s", + topic.getDomain().value(), topic.getNamespace(), decode(s))) + .collect(Collectors.toList()) + ); + } + + public CompletableFuture persistentTopicExists(TopicName topic) { + String path = MANAGED_LEDGER_PATH + "/" + topic.getPersistenceNamingEncoding();; + return store.exists(path); + } + + public CompletableFuture clearNamespacePersistence(NamespaceName ns) { + String path = MANAGED_LEDGER_PATH + "/" + ns; + return store.delete(path, Optional.empty()); + } + + public CompletableFuture clearDomainPersistence(NamespaceName ns) { + String path = MANAGED_LEDGER_PATH + "/" + ns + "/persistent"; + return store.delete(path, Optional.empty()); + } + + public CompletableFuture clearTennantPersistence(String tenant) { + String path = MANAGED_LEDGER_PATH + "/" + tenant; + return store.delete(path, Optional.empty()); + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarBrokerStarter.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarBrokerStarter.java index 8d29684697937e..6ec8e310192bf1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarBrokerStarter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarBrokerStarter.java @@ -26,6 +26,7 @@ import static org.apache.pulsar.common.configuration.PulsarConfigurationLoader.isComplete; import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; +import com.beust.jcommander.Parameters; import com.google.common.annotations.VisibleForTesting; import java.io.File; import java.io.FileInputStream; @@ -51,6 +52,7 @@ import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.naming.NamespaceBundleSplitAlgorithm; import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.util.CmdGenerateDocs; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerService; import org.apache.pulsar.functions.worker.service.WorkerServiceLoader; @@ -72,6 +74,7 @@ private static ServiceConfiguration loadConfig(String configFile) throws Excepti } @VisibleForTesting + @Parameters(commandDescription = "Options") private static class StarterArguments { @Parameter(names = {"-c", "--broker-conf"}, description = "Configuration file for Broker") private String brokerConfigFile = @@ -97,6 +100,9 @@ private static class StarterArguments { @Parameter(names = {"-h", "--help"}, description = "Show this help message") private boolean help = false; + + @Parameter(names = {"-g", "--generate-docs"}, description = "Generate docs") + private boolean generateDocs = false; } private static ServerConfiguration readBookieConfFile(String bookieConfigFile) throws IllegalArgumentException { @@ -134,7 +140,7 @@ private static class BrokerStarter { private final WorkerService functionsWorkerService; private final WorkerConfig workerConfig; - BrokerStarter(String[] args) throws Exception{ + BrokerStarter(String[] args) throws Exception { StarterArguments starterArguments = new StarterArguments(); JCommander jcommander = new JCommander(starterArguments); jcommander.setProgramName("PulsarBrokerStarter"); @@ -146,6 +152,13 @@ private static class BrokerStarter { System.exit(-1); } + if (starterArguments.generateDocs) { + CmdGenerateDocs cmd = new CmdGenerateDocs("pulsar"); + cmd.addCommand("broker", starterArguments); + cmd.run(null); + System.exit(-1); + } + // init broker config if (isBlank(starterArguments.brokerConfigFile)) { jcommander.usage(); 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 c73c72db8301a3..6e5894804aa612 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 @@ -38,7 +38,6 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.cache.LocalZooKeeperCacheService; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.broker.web.PulsarWebResource; @@ -64,15 +63,10 @@ import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.MetadataStoreException.AlreadyExistsException; import org.apache.pulsar.metadata.api.MetadataStoreException.BadVersionException; import org.apache.pulsar.metadata.api.MetadataStoreException.NotFoundException; -import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; -import org.apache.pulsar.zookeeper.ZooKeeperCache; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooDefs; -import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,28 +74,11 @@ public abstract class AdminResource extends PulsarWebResource { private static final Logger log = LoggerFactory.getLogger(AdminResource.class); public static final String POLICIES_READONLY_FLAG_PATH = "/admin/flags/policies-readonly"; public static final String PARTITIONED_TOPIC_PATH_ZNODE = "partitioned-topics"; - private static final String MANAGED_LEDGER_PATH_ZNODE = "/managed-ledgers"; protected BookKeeper bookKeeper() { return pulsar().getBookKeeperClient(); } - protected ZooKeeper localZk() { - return pulsar().getZkClient(); - } - - protected ZooKeeperCache localZkCache() { - return pulsar().getLocalZkCache(); - } - - protected LocalZooKeeperCacheService localCacheService() { - return pulsar().getLocalZkCacheService(); - } - - protected void localZKCreate(String path, byte[] content) throws Exception { - localZk().create(path, content, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - } - /** * Get the domain of the topic (whether it's persistent or non-persistent). */ @@ -154,7 +131,7 @@ public void validatePoliciesReadOnlyAccess() { arePoliciesReadOnly = pulsar().getPulsarResources().getNamespaceResources() .exists(POLICIES_READONLY_FLAG_PATH); } catch (Exception e) { - log.warn("Unable to fetch contents of [{}] from global zookeeper", POLICIES_READONLY_FLAG_PATH, e); + log.warn("Unable to fetch contents of [{}] from configuration store", POLICIES_READONLY_FLAG_PATH, e); throw new RestException(e); } @@ -180,7 +157,7 @@ protected CompletableFuture tryCreatePartitionsAsync(int numPartitions) { private CompletableFuture tryCreatePartitionAsync(final int partition, CompletableFuture reuseFuture) { CompletableFuture result = reuseFuture == null ? new CompletableFuture<>() : reuseFuture; - Optional localStore = getPulsarResources().getLocalMetadataStore(); + Optional localStore = getPulsarResources().getLocalMetadataStore(); if (!localStore.isPresent()) { result.completeExceptionally(new IllegalStateException("metadata store not initialized")); return result; @@ -600,25 +577,13 @@ protected List getPartitionedTopicList(TopicDomain topicDomain) { } protected List getTopicPartitionList(TopicDomain topicDomain) { - List topicPartitions = Lists.newArrayList(); - try { - String topicPartitionPath = joinPath(MANAGED_LEDGER_PATH_ZNODE, - namespaceName.toString(), topicDomain.value()); - List topics = localZk().getChildren(topicPartitionPath, false); - topicPartitions = topics.stream() - .map(s -> String.format("%s://%s/%s", topicDomain.value(), namespaceName.toString(), decode(s))) - .collect(Collectors.toList()); - } catch (KeeperException.NoNodeException e) { - // NoNode means there are no topics in this domain for this namespace + return getPulsarResources().getTopicResources().getExistingPartitions(topicName).join(); } catch (Exception e) { log.error("[{}] Failed to get topic partition list for namespace {}", clientAppId(), namespaceName.toString(), e); throw new RestException(e); } - - topicPartitions.sort(null); - return topicPartitions; } protected void internalCreatePartitionedTopic(AsyncResponse asyncResponse, int numPartitions, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java index 76cd60209e689b..6ccbc8c330a112 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/ClustersBase.java @@ -739,9 +739,9 @@ private void filterAndUnloadMatchedNameSpaces(AsyncResponse asyncResponse, tenants.forEach(tenant -> namespaces.getNamespacesAsync(tenant) .whenComplete((nss, e) -> { int leftTenantsToHandle = tenantsNumber.decrementAndGet(); - if (ex != null) { + if (e != null) { log.error("[{}] Failed to get namespaces for tenant {} when setNamespaceIsolationPolicy.", - clientAppId(), tenant, ex); + clientAppId(), tenant, e); if (leftTenantsToHandle == 0) { unloadMatchedNamespacesList(asyncResponse, nssToUnload, namespaces); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java index ec9de23e6233fd..1251968392c31e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/NamespacesBase.java @@ -458,6 +458,15 @@ protected void internalDeleteNamespaceForcefully(AsyncResponse asyncResponse, bo deleteRecursive(namespaceResources(), globalPartitionedPath); } + try { + pulsar().getPulsarResources().getTopicResources().clearDomainPersistence(namespaceName).get(); + pulsar().getPulsarResources().getTopicResources().clearNamespacePersistence(namespaceName).get(); + } catch (ExecutionException | InterruptedException e) { + // warn level log here since this failure has no side effect besides left a un-used metadata + // and also will not affect the re-creation of namespace + log.warn("[{}] Failed to remove managed-ledger for {}", clientAppId(), namespaceName, e); + } + // we have successfully removed all the ownership for the namespace, the policies znode can be deleted // now final String globalZkPolicyPath = path(POLICIES, namespaceName.toString()); @@ -780,11 +789,11 @@ protected void internalSetNamespaceMessageTTL(Integer messageTTL) { }); } - protected void internalSetSubscriptionExpirationTime(int expirationTime) { + protected void internalSetSubscriptionExpirationTime(Integer expirationTime) { validateNamespacePolicyOperation(namespaceName, PolicyName.SUBSCRIPTION_EXPIRATION_TIME, PolicyOperation.WRITE); validatePoliciesReadOnlyAccess(); - if (expirationTime < 0) { + if (expirationTime != null && expirationTime < 0) { throw new RestException(Status.PRECONDITION_FAILED, "Invalid value for subscription expiration time"); } updatePolicies(path(POLICIES, namespaceName.toString()), (policies) -> { 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 a3c4c3e30ce8de..4c137ea5ecc024 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 @@ -90,6 +90,7 @@ import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; +import org.apache.pulsar.common.api.proto.BrokerEntryMetadata; import org.apache.pulsar.common.api.proto.CommandSubscribe.InitialPosition; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.KeyValue; @@ -2494,6 +2495,7 @@ private Response generateResponseWithEntry(Entry entry) throws IOException { PositionImpl pos = (PositionImpl) entry.getPosition(); ByteBuf metadataAndPayload = entry.getDataBuffer(); + BrokerEntryMetadata brokerEntryMetadata = Commands.peekBrokerEntryMetadataIfExist(metadataAndPayload); MessageMetadata metadata = Commands.parseMessageMetadata(metadataAndPayload); ResponseBuilder responseBuilder = Response.ok(); @@ -2501,12 +2503,24 @@ private Response generateResponseWithEntry(Entry entry) throws IOException { for (KeyValue keyValue : metadata.getPropertiesList()) { responseBuilder.header("X-Pulsar-PROPERTY-" + keyValue.getKey(), keyValue.getValue()); } + if (brokerEntryMetadata != null) { + if (brokerEntryMetadata.hasBrokerTimestamp()) { + responseBuilder.header("X-Pulsar-Broker-Entry-METADATA-timestamp", + DateFormatter.format(brokerEntryMetadata.getBrokerTimestamp())); + } + if (brokerEntryMetadata.hasIndex()) { + responseBuilder.header("X-Pulsar-Broker-Entry-METADATA-index", brokerEntryMetadata.getIndex()); + } + } if (metadata.hasPublishTime()) { responseBuilder.header("X-Pulsar-publish-time", DateFormatter.format(metadata.getPublishTime())); } if (metadata.hasEventTime()) { responseBuilder.header("X-Pulsar-event-time", DateFormatter.format(metadata.getEventTime())); } + if (metadata.hasDeliverAtTime()) { + responseBuilder.header("X-Pulsar-deliver-at-time", DateFormatter.format(metadata.getDeliverAtTime())); + } if (metadata.hasNumMessagesInBatch()) { responseBuilder.header("X-Pulsar-num-batch-message", metadata.getNumMessagesInBatch()); responseBuilder.header("X-Pulsar-batch-size", metadataAndPayload.readableBytes() 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 d81da0a4153e4f..fc63fdc8d574f8 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 @@ -28,6 +28,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import javax.ws.rs.DELETE; import javax.ws.rs.DefaultValue; @@ -327,6 +328,15 @@ protected void internalDeleteTenantForcefully(AsyncResponse asyncResponse, Strin } return null; } + + + try { + pulsar().getPulsarResources().getTopicResources().clearTennantPersistence(tenant).get(); + } catch (ExecutionException | InterruptedException e) { + // warn level log here since this failure has no side effect besides left a un-used metadata + // and also will not affect the re-creation of tenant + log.warn("[{}] Failed to remove managed-ledger for {}", clientAppId(), tenant, e); + } // delete tenant normally internalDeleteTenant(asyncResponse, tenant); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java index 3e7067ba92cdde..529b8ce0cbe62a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/Namespaces.java @@ -358,7 +358,7 @@ public void setNamespaceMessageTTL(@PathParam("property") String property, @Path @ApiOperation(hidden = true, value = "Get the subscription expiration time for the namespace") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) - public int getSubscriptionExpirationTime(@PathParam("property") String property, + public Integer getSubscriptionExpirationTime(@PathParam("property") String property, @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { validateAdminAccessForTenant(property); validateNamespaceName(property, cluster, namespace); @@ -379,6 +379,16 @@ public void setSubscriptionExpirationTime(@PathParam("property") String property internalSetSubscriptionExpirationTime(expirationTime); } + @DELETE + @Path("/{property}/{cluster}/{namespace}/subscriptionExpirationTime") + @ApiOperation(hidden = true, value = "Remove subscription expiration time for namespace") + @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) + public void removeSubscriptionExpirationTime(@PathParam("property") String property, + @PathParam("cluster") String cluster, @PathParam("namespace") String namespace) { + validateNamespaceName(property, cluster, namespace); + internalSetSubscriptionExpirationTime(null); + } @POST @Path("/{property}/{cluster}/{namespace}/antiAffinity") diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java index 4609377913f9ba..9a1f684c425147 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Namespaces.java @@ -314,7 +314,7 @@ public void removeNamespaceMessageTTL(@PathParam("tenant") String tenant, @ApiOperation(value = "Get the subscription expiration time for the namespace") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) - public int getSubscriptionExpirationTime(@PathParam("tenant") String tenant, + public Integer getSubscriptionExpirationTime(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { validateAdminAccessForTenant(tenant); validateNamespaceName(tenant, namespace); @@ -338,6 +338,17 @@ public void setSubscriptionExpirationTime(@PathParam("tenant") String tenant, internalSetSubscriptionExpirationTime(expirationTime); } + @DELETE + @Path("/{tenant}/{namespace}/subscriptionExpirationTime") + @ApiOperation(value = "Remove subscription expiration time for namespace") + @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist")}) + public void removeSubscriptionExpirationTime(@PathParam("tenant") String tenant, + @PathParam("namespace") String namespace) { + validateNamespaceName(tenant, namespace); + internalSetSubscriptionExpirationTime(null); + } + @GET @Path("/{tenant}/{namespace}/deduplication") @ApiOperation(value = "Get broker side deduplication for all topics in a namespace") diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java index 5adb852413f7f7..223cf81ccdb560 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java @@ -43,41 +43,74 @@ class ProtocolHandlerWithClassLoader implements ProtocolHandler { @Override public String protocolName() { - return handler.protocolName(); + try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + return handler.protocolName(); + } } @Override public boolean accept(String protocol) { - return handler.accept(protocol); + try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + return handler.accept(protocol); + } } @Override public void initialize(ServiceConfiguration conf) throws Exception { - handler.initialize(conf); + try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + handler.initialize(conf); + } } @Override public String getProtocolDataToAdvertise() { - return handler.getProtocolDataToAdvertise(); + try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + return handler.getProtocolDataToAdvertise(); + } } @Override public void start(BrokerService service) { - handler.start(service); + try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + handler.start(service); + } } @Override public Map> newChannelInitializers() { - return handler.newChannelInitializers(); + try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + return handler.newChannelInitializers(); + } } @Override public void close() { - handler.close(); + try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + handler.close(); + } + try { classLoader.close(); } catch (IOException e) { log.warn("Failed to close the protocol handler class loader", e); } } + + /** + * Help to switch the class loader of current thread to the NarClassLoader, and change it back when it's done. + * With the help of try-with-resources statement, the code would be cleaner than using try finally every time. + */ + private static class ClassLoaderSwitcher implements AutoCloseable { + private final ClassLoader prevClassLoader; + + ClassLoaderSwitcher(ClassLoader classLoader) { + prevClassLoader = Thread.currentThread().getContextClassLoader(); + Thread.currentThread().setContextClassLoader(classLoader); + } + + @Override + public void close() { + Thread.currentThread().setContextClassLoader(prevClassLoader); + } + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java index e73daaa4210d4c..690a5984b4828b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractDispatcherSingleActiveConsumer.java @@ -55,6 +55,8 @@ public abstract class AbstractDispatcherSingleActiveConsumer extends AbstractBas AtomicIntegerFieldUpdater.newUpdater(AbstractDispatcherSingleActiveConsumer.class, "isClosed"); private volatile int isClosed = FALSE; + protected boolean isFirstRead = true; + public AbstractDispatcherSingleActiveConsumer(SubType subscriptionType, int partitionIndex, String topicName, Subscription subscription, ServiceConfiguration serviceConfig) { @@ -159,6 +161,10 @@ public synchronized void addConsumer(Consumer consumer) throws BrokerServiceExce isKeyHashRangeFiltered = false; } + if (consumers.isEmpty()) { + isFirstRead = true; + } + consumers.add(consumer); if (!pickAndScheduleActiveConsumer()) { 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 e390dd15b5521d..c1e17159eb7385 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 @@ -968,37 +968,36 @@ public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { } private CompletableFuture> createNonPersistentTopic(String topic) { - CompletableFuture> topicFuture = futureWithDeadline(); - if (!pulsar.getConfiguration().isEnableNonPersistentTopics()) { if (log.isDebugEnabled()) { log.debug("Broker is unable to load non-persistent topic {}", topic); } - topicFuture.completeExceptionally( + return FutureUtil.failedFuture( new NotAllowedException("Broker is not unable to load non-persistent topic")); - return topicFuture; } final long topicCreateTimeMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()); NonPersistentTopic nonPersistentTopic = new NonPersistentTopic(topic, this); - CompletableFuture replicationFuture = nonPersistentTopic.checkReplication(); - replicationFuture.thenRun(() -> { + + CompletableFuture> future = nonPersistentTopic.initialize() + .thenCompose(__ -> nonPersistentTopic.checkReplication()) + .thenApply(__ -> { log.info("Created topic {}", nonPersistentTopic); long topicLoadLatencyMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()) - topicCreateTimeMs; pulsarStats.recordTopicLoadTimeValue(topic, topicLoadLatencyMs); addTopicToStatsMaps(TopicName.get(topic), nonPersistentTopic); - topicFuture.complete(Optional.of(nonPersistentTopic)); + return Optional.of(nonPersistentTopic); }); - replicationFuture.exceptionally((ex) -> { + + future.exceptionally((ex) -> { log.warn("Replication check failed. Removing topic from topics list {}, {}", topic, ex); nonPersistentTopic.stopReplProducers().whenComplete((v, exception) -> { - pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); - topicFuture.completeExceptionally(ex); + pulsar.getExecutor().execute(() -> topics.remove(topic, future)); }); return null; }); - return topicFuture; + return future; } private CompletableFuture futureWithDeadline() { @@ -1223,7 +1222,9 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { preCreateSubForCompaction = ((SystemTopic) persistentTopic) .preCreateSubForCompactionIfNeeded(); } - CompletableFuture replicationFuture = persistentTopic.checkReplication(); + CompletableFuture replicationFuture = persistentTopic + .initialize() + .thenCompose(__ -> persistentTopic.checkReplication()); FutureUtil.waitForAll(Lists.newArrayList(preCreateSubForCompaction, replicationFuture)) .thenCompose(v -> { // Also check dedup status diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index 320139ec68f6d2..7feebc38faddb4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -41,6 +41,7 @@ import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.common.api.proto.CommandAck; import org.apache.pulsar.common.api.proto.CommandAck.AckType; @@ -124,12 +125,13 @@ public class Consumer { private boolean preciseDispatcherFlowControl; private PositionImpl readPositionWhenJoining; private final String clientAddress; // IP address only, no port number included + private final MessageId startMessageId; public Consumer(Subscription subscription, SubType subType, String topicName, long consumerId, int priorityLevel, String consumerName, int maxUnackedMessages, TransportCnx cnx, String appId, Map metadata, boolean readCompacted, InitialPosition subscriptionInitialPosition, - KeySharedMeta keySharedMeta) { + KeySharedMeta keySharedMeta, MessageId startMessageId) { this.subscription = subscription; this.subType = subType; @@ -149,6 +151,10 @@ public Consumer(Subscription subscription, SubType subType, String topicName, lo this.bytesOutCounter = new LongAdder(); this.msgOutCounter = new LongAdder(); this.appId = appId; + + // Ensure we start from compacted view + this.startMessageId = (readCompacted && startMessageId == null) ? MessageId.earliest : startMessageId; + this.preciseDispatcherFlowControl = cnx.isPreciseDispatcherFlowControl(); PERMITS_RECEIVED_WHILE_CONSUMER_BLOCKED_UPDATER.set(this, 0); MESSAGE_PERMITS_UPDATER.set(this, 0); @@ -842,6 +848,10 @@ private int getStickyKeyHash(Entry entry) { byte[] stickyKey = Commands.peekStickyKey(entry.getDataBuffer(), topicName, subscription.getName()); return StickyKeyConsumerSelector.makeStickyKeyHash(stickyKey); } + + public MessageId getStartMessageId() { + return startMessageId; + } private static final Logger log = LoggerFactory.getLogger(Consumer.class); } 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 d1e4506fb6422c..e0cc873d415924 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 @@ -96,6 +96,8 @@ default long getNumberOfMessages() { } } + CompletableFuture initialize(); + void publishMessage(ByteBuf headersAndPayload, PublishContext callback); /** 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 9fd4ce5e8d7614..2751d1e2a01967 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 @@ -81,9 +81,9 @@ import org.apache.pulsar.common.protocol.schema.SchemaData; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; +import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats; import org.apache.pulsar.utils.StatsOutputStream; -import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -134,28 +134,31 @@ public void reset() { public NonPersistentTopic(String topic, BrokerService brokerService) { super(topic, brokerService); + this.subscriptions = new ConcurrentOpenHashMap<>(16, 1); this.replicators = new ConcurrentOpenHashMap<>(16, 1); this.isFenced = false; + } - try { - Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace())) - .orElseThrow(() -> new KeeperException.NoNodeException()); - isEncryptionRequired = policies.encryption_required; - isAllowAutoUpdateSchema = policies.is_allow_auto_update_schema; - if (policies.inactive_topic_policies != null) { - inactiveTopicPolicies = policies.inactive_topic_policies; - } - setSchemaCompatibilityStrategy(policies); - - schemaValidationEnforced = policies.schema_validation_enforced; + public CompletableFuture initialize() { + return brokerService.pulsar().getPulsarResources().getNamespaceResources() + .getAsync(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace())) + .thenAccept(optPolicies -> { + if (!optPolicies.isPresent()) { + log.warn("[{}] Policies not present {} and isEncryptionRequired will be set to false", topic); + isEncryptionRequired = false; + } else { + Policies policies = optPolicies.get(); + isEncryptionRequired = policies.encryption_required; + isAllowAutoUpdateSchema = policies.is_allow_auto_update_schema; + if (policies.inactive_topic_policies != null) { + inactiveTopicPolicies = policies.inactive_topic_policies; + } + setSchemaCompatibilityStrategy(policies); - } catch (Exception e) { - log.warn("[{}] Error getting policies {} and isEncryptionRequired will be set to false", topic, - e.getMessage()); - isEncryptionRequired = false; - } + schemaValidationEnforced = policies.schema_validation_enforced; + } + }); } @Override @@ -268,7 +271,7 @@ public CompletableFuture subscribe(final TransportCnx cnx, String subs NonPersistentSubscription subscription = subscriptions.computeIfAbsent(subscriptionName, name -> new NonPersistentSubscription(this, subscriptionName)); Consumer consumer = new Consumer(subscription, subType, topic, consumerId, priorityLevel, consumerName, 0, - cnx, cnx.getAuthRole(), metadata, readCompacted, initialPosition, keySharedMeta); + cnx, cnx.getAuthRole(), metadata, readCompacted, initialPosition, keySharedMeta, MessageId.latest); addConsumerToSubscription(subscription, consumer).thenRun(() -> { if (!cnx.isActive()) { try { @@ -482,53 +485,57 @@ public CompletableFuture checkReplication() { log.debug("[{}] Checking replication status", name); } - Policies policies = null; - try { - policies = brokerService.pulsar().getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, name.getNamespace())) - .orElseThrow(() -> new KeeperException.NoNodeException()); - } catch (Exception e) { - CompletableFuture future = new CompletableFuture<>(); - future.completeExceptionally(new ServerMetadataException(e)); - return future; - } + return brokerService.pulsar().getPulsarResources().getNamespaceResources() + .getAsync(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace())) + .thenCompose(optPolicies -> { + if (!optPolicies.isPresent()) { + return FutureUtil.failedFuture( + new ServerMetadataException( + new MetadataStoreException.NotFoundException())); + } - Set configuredClusters; - if (policies.replication_clusters != null) { - configuredClusters = policies.replication_clusters; - } else { - configuredClusters = Collections.emptySet(); - } + Policies policies = optPolicies.get(); + Set configuredClusters; + if (policies.replication_clusters != null) { + configuredClusters = policies.replication_clusters; + } else { + configuredClusters = Collections.emptySet(); + } - String localCluster = brokerService.pulsar().getConfiguration().getClusterName(); + String localCluster = brokerService.pulsar().getConfiguration().getClusterName(); - List> futures = Lists.newArrayList(); + List> futures = Lists.newArrayList(); - // Check for missing replicators - for (String cluster : configuredClusters) { - if (cluster.equals(localCluster)) { - continue; - } + // Check for missing replicators + for (String cluster : configuredClusters) { + if (cluster.equals(localCluster)) { + continue; + } + + if (!replicators.containsKey(cluster)) { + if (!startReplicator(cluster)) { + // it happens when global topic is a partitioned topic and replicator can't start on + // original + // non partitioned-topic (topic without partition prefix) + return FutureUtil + .failedFuture(new NamingException( + topic + " failed to start replicator for " + cluster)); + } + } + } + + // Check for replicators to be stopped + replicators.forEach((cluster, replicator) -> { + if (!cluster.equals(localCluster)) { + if (!configuredClusters.contains(cluster)) { + futures.add(removeReplicator(cluster)); + } + } + }); + return FutureUtil.waitForAll(futures); + }); - if (!replicators.containsKey(cluster)) { - if (!startReplicator(cluster)) { - // it happens when global topic is a partitioned topic and replicator can't start on original - // non partitioned-topic (topic without partition prefix) - return FutureUtil - .failedFuture(new NamingException(topic + " failed to start replicator for " + cluster)); - } - } - } - // Check for replicators to be stopped - replicators.forEach((cluster, replicator) -> { - if (!cluster.equals(localCluster)) { - if (!configuredClusters.contains(cluster)) { - futures.add(removeReplicator(cluster)); - } - } - }); - return FutureUtil.waitForAll(futures); } boolean startReplicator(String remoteCluster) { @@ -886,13 +893,12 @@ public void checkInactiveSubscriptions() { try { Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache() .get(AdminResource.path(POLICIES, name.getNamespace())) - .orElseThrow(KeeperException.NoNodeException::new); + .orElseThrow(MetadataStoreException.NotFoundException::new); final int defaultExpirationTime = brokerService.pulsar().getConfiguration() .getSubscriptionExpirationTimeMinutes(); + final Integer nsExpirationTime = policies.subscription_expiration_time_minutes; final long expirationTimeMillis = TimeUnit.MINUTES - .toMillis((policies.subscription_expiration_time_minutes <= 0 && defaultExpirationTime > 0) - ? defaultExpirationTime - : policies.subscription_expiration_time_minutes); + .toMillis(nsExpirationTime == null ? defaultExpirationTime : nsExpirationTime); if (expirationTimeMillis > 0) { subscriptions.forEach((subName, sub) -> { if (sub.getDispatcher() != null 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 2bd94fff51c50d..4bae5b02a40f04 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 @@ -153,6 +153,7 @@ public synchronized void internalReadEntriesComplete(final List entries, } havePendingRead = false; + isFirstRead = false; if (readBatchSize < serviceConfig.getDispatcherMaxReadBatchSize()) { int newReadBatchSize = Math.min(readBatchSize * 2, serviceConfig.getDispatcherMaxReadBatchSize()); @@ -338,7 +339,8 @@ protected void readMoreEntries(Consumer consumer) { } havePendingRead = true; if (consumer.readCompacted()) { - topic.getCompactedTopic().asyncReadEntriesOrWait(cursor, messagesToRead, this, consumer); + topic.getCompactedTopic().asyncReadEntriesOrWait(cursor, messagesToRead, isFirstRead, + this, consumer); } else { cursor.asyncReadEntriesOrWait(messagesToRead, bytesToRead, this, consumer, topic.getMaxReadPosition()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStreamingDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStreamingDispatcherSingleActiveConsumer.java index e213b049154132..2e922324f18735 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStreamingDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStreamingDispatcherSingleActiveConsumer.java @@ -121,6 +121,8 @@ public synchronized void internalReadEntryComplete(Entry entry, PendingReadEntry havePendingRead = false; } + isFirstRead = false; + if (readBatchSize < serviceConfig.getDispatcherMaxReadBatchSize()) { int newReadBatchSize = Math.min(readBatchSize * 2, serviceConfig.getDispatcherMaxReadBatchSize()); if (log.isDebugEnabled()) { @@ -197,7 +199,8 @@ protected void readMoreEntries(Consumer consumer) { havePendingRead = true; if (consumer.readCompacted()) { - topic.getCompactedTopic().asyncReadEntriesOrWait(cursor, messagesToRead, this, consumer); + topic.getCompactedTopic().asyncReadEntriesOrWait(cursor, messagesToRead, isFirstRead, + this, consumer); } else { streamingEntryReader.asyncReadEntries(messagesToRead, bytesToRead, consumer); } 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 112c933a8d1e16..0f0d9670a9d3f0 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 @@ -147,9 +147,9 @@ import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.compaction.CompactedTopic; import org.apache.pulsar.compaction.CompactedTopicImpl; +import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats; import org.apache.pulsar.utils.StatsOutputStream; -import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -277,37 +277,13 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS } else { final String subscriptionName = Codec.decode(cursor.getName()); subscriptions.put(subscriptionName, createPersistentSubscription(subscriptionName, cursor, - PersistentSubscription.isCursorFromReplicatedSubscription(cursor))); + PersistentSubscription.isCursorFromReplicatedSubscription(cursor))); // subscription-cursor gets activated by default: deactivate as there is no active subscription right // now subscriptions.get(subscriptionName).deactivateCursor(); } } this.messageDeduplication = new MessageDeduplication(brokerService.pulsar(), this, ledger); - - try { - Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace())) - .orElseThrow(() -> new KeeperException.NoNodeException()); - this.isEncryptionRequired = policies.encryption_required; - - setSchemaCompatibilityStrategy(policies); - isAllowAutoUpdateSchema = policies.is_allow_auto_update_schema; - - schemaValidationEnforced = policies.schema_validation_enforced; - if (policies.inactive_topic_policies != null) { - inactiveTopicPolicies = policies.inactive_topic_policies; - } - updateUnackedMessagesAppliedOnSubscription(policies); - updateUnackedMessagesExceededOnConsumer(policies); - } catch (Exception e) { - log.warn("[{}] Error getting policies {} and isEncryptionRequired will be set to false", - topic, e.getMessage()); - isEncryptionRequired = false; - updateUnackedMessagesAppliedOnSubscription(null); - updateUnackedMessagesExceededOnConsumer(null); - } - if (ledger.getProperties().containsKey(TOPIC_EPOCH_PROPERTY_NAME)) { topicEpoch = Optional.of(Long.parseLong(ledger.getProperties().get(TOPIC_EPOCH_PROPERTY_NAME))); } @@ -325,6 +301,40 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry()); } + @Override + public CompletableFuture initialize() { + return brokerService.pulsar().getPulsarResources().getNamespaceResources() + .getAsync(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace())) + .thenAccept(optPolicies -> { + if (!optPolicies.isPresent()) { + isEncryptionRequired = false; + updateUnackedMessagesAppliedOnSubscription(null); + updateUnackedMessagesExceededOnConsumer(null); + return; + } + + Policies policies = optPolicies.get(); + this.isEncryptionRequired = policies.encryption_required; + + setSchemaCompatibilityStrategy(policies); + isAllowAutoUpdateSchema = policies.is_allow_auto_update_schema; + + schemaValidationEnforced = policies.schema_validation_enforced; + if (policies.inactive_topic_policies != null) { + inactiveTopicPolicies = policies.inactive_topic_policies; + } + updateUnackedMessagesAppliedOnSubscription(policies); + updateUnackedMessagesExceededOnConsumer(policies); + }).exceptionally(ex -> { + log.warn("[{}] Error getting policies {} and isEncryptionRequired will be set to false", + topic, ex.getMessage()); + isEncryptionRequired = false; + updateUnackedMessagesAppliedOnSubscription(null); + updateUnackedMessagesExceededOnConsumer(null); + return null; + }); + } + // for testing purposes @VisibleForTesting PersistentTopic(String topic, BrokerService brokerService, ManagedLedger ledger, @@ -533,12 +543,11 @@ public synchronized void addFailed(ManagedLedgerException exception, Object ctx) @Override public CompletableFuture> addProducer(Producer producer, CompletableFuture producerQueuedFuture) { - return super.addProducer(producer, producerQueuedFuture).thenApply(topicEpoch -> { + return super.addProducer(producer, producerQueuedFuture).thenCompose(topicEpoch -> { messageDeduplication.producerAdded(producer.getProducerName()); // Start replication producers if not already - startReplProducers(); - return topicEpoch; + return startReplProducers().thenApply(__ -> topicEpoch); }); } @@ -579,26 +588,30 @@ private boolean hasRemoteProducers() { return foundRemote.get(); } - public void startReplProducers() { + public CompletableFuture startReplProducers() { // read repl-cluster from policies to avoid restart of replicator which are in process of disconnect and close - try { - Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace())) - .orElseThrow(() -> new KeeperException.NoNodeException()); - if (policies.replication_clusters != null) { - Set configuredClusters = Sets.newTreeSet(policies.replication_clusters); - replicators.forEach((region, replicator) -> { - if (configuredClusters.contains(region)) { - replicator.startProducer(); + return brokerService.pulsar().getPulsarResources().getNamespaceResources() + .getAsync(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace())) + .thenAccept(optPolicies -> { + if (optPolicies.isPresent()) { + if (optPolicies.get().replication_clusters != null) { + Set configuredClusters = Sets.newTreeSet(optPolicies.get().replication_clusters); + replicators.forEach((region, replicator) -> { + if (configuredClusters.contains(region)) { + replicator.startProducer(); + } + }); + } + } else { + replicators.forEach((region, replicator) -> replicator.startProducer()); } - }); - } - } catch (Exception e) { + }).exceptionally(ex -> { if (log.isDebugEnabled()) { - log.debug("[{}] Error getting policies while starting repl-producers {}", topic, e.getMessage()); + log.debug("[{}] Error getting policies while starting repl-producers {}", topic, ex.getMessage()); } replicators.forEach((region, replicator) -> replicator.startProducer()); - } + return null; + }); } public CompletableFuture stopReplProducers() { @@ -718,7 +731,7 @@ public CompletableFuture subscribe(final TransportCnx cnx, String subs CompletableFuture future = subscriptionFuture.thenCompose(subscription -> { Consumer consumer = new Consumer(subscription, subType, topic, consumerId, priorityLevel, consumerName, maxUnackedMessages, cnx, cnx.getAuthRole(), metadata, - readCompacted, initialPosition, keySharedMeta); + readCompacted, initialPosition, keySharedMeta, startMessageId); return addConsumerToSubscription(subscription, consumer).thenCompose(v -> { checkBackloggedCursors(); if (!cnx.isActive()) { @@ -1072,7 +1085,7 @@ public void deleteLedgerComplete(Object ctx) { @Override public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { - if (exception.getCause() instanceof KeeperException.NoNodeException) { + if (exception.getCause() instanceof MetadataStoreException.NotFoundException) { log.info("[{}] Topic is already deleted {}", topic, exception.getMessage()); deleteLedgerComplete(ctx); } else { @@ -1233,86 +1246,84 @@ public CompletableFuture checkReplication() { log.debug("[{}] Checking replication status", name); } - Policies policies = null; - try { - policies = brokerService.pulsar().getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, name.getNamespace())) - .orElseThrow(() -> new KeeperException.NoNodeException()); - } catch (Exception e) { - CompletableFuture future = new CompletableFuture<>(); - future.completeExceptionally(new ServerMetadataException(e)); - return future; - } - //Ignore current broker's config for messageTTL for replication. - final int newMessageTTLinSeconds; - try { - newMessageTTLinSeconds = getMessageTTL(); - } catch (Exception e) { - return FutureUtil.failedFuture(new ServerMetadataException(e)); - } + CompletableFuture policiesFuture = brokerService.pulsar().getPulsarResources() + .getNamespaceResources() + .getAsync(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace())) + .thenCompose(optPolicies -> { + if (!optPolicies.isPresent()) { + return FutureUtil.failedFuture( + new ServerMetadataException( + new MetadataStoreException.NotFoundException())); + } - Set configuredClusters; - if (policies.replication_clusters != null) { - configuredClusters = Sets.newTreeSet(policies.replication_clusters); - } else { - configuredClusters = Collections.emptySet(); - } + return CompletableFuture.completedFuture(optPolicies.get()); + }); - String localCluster = brokerService.pulsar().getConfiguration().getClusterName(); + CompletableFuture ttlFuture = getMessageTTL(); - // if local cluster is removed from global namespace cluster-list : then delete topic forcefully because pulsar - // doesn't serve global topic without local repl-cluster configured. - if (TopicName.get(topic).isGlobal() && !configuredClusters.contains(localCluster)) { - log.info("Deleting topic [{}] because local cluster is not part of global namespace repl list {}", - topic, configuredClusters); - return deleteForcefully(); - } + return CompletableFuture.allOf(policiesFuture, ttlFuture) + .thenCompose(__ -> { + Policies policies = policiesFuture.join(); + int newMessageTTLinSeconds = ttlFuture.join(); - List> futures = Lists.newArrayList(); + Set configuredClusters; + if (policies.replication_clusters != null) { + configuredClusters = Sets.newTreeSet(policies.replication_clusters); + } else { + configuredClusters = Collections.emptySet(); + } - // Check for missing replicators - for (String cluster : configuredClusters) { - if (cluster.equals(localCluster)) { - continue; - } + String localCluster = brokerService.pulsar().getConfiguration().getClusterName(); - if (!replicators.containsKey(cluster)) { - futures.add(startReplicator(cluster)); - } - } + // if local cluster is removed from global namespace cluster-list : then delete topic forcefully + // because pulsar doesn't serve global topic without local repl-cluster configured. + if (TopicName.get(topic).isGlobal() && !configuredClusters.contains(localCluster)) { + log.info("Deleting topic [{}] because local cluster is not part of " + + " global namespace repl list {}", topic, configuredClusters); + return deleteForcefully(); + } - // Check for replicators to be stopped - replicators.forEach((cluster, replicator) -> { - // Update message TTL - ((PersistentReplicator) replicator).updateMessageTTL(newMessageTTLinSeconds); + List> futures = Lists.newArrayList(); - if (!cluster.equals(localCluster)) { - if (!configuredClusters.contains(cluster)) { - futures.add(removeReplicator(cluster)); - } - } + // Check for missing replicators + for (String cluster : configuredClusters) { + if (cluster.equals(localCluster)) { + continue; + } - }); + if (!replicators.containsKey(cluster)) { + futures.add(startReplicator(cluster)); + } + } - return FutureUtil.waitForAll(futures); + // Check for replicators to be stopped + replicators.forEach((cluster, replicator) -> { + // Update message TTL + ((PersistentReplicator) replicator).updateMessageTTL(newMessageTTLinSeconds); + + if (!cluster.equals(localCluster)) { + if (!configuredClusters.contains(cluster)) { + futures.add(removeReplicator(cluster)); + } + } + + }); + + return FutureUtil.waitForAll(futures); + }); } @Override public void checkMessageExpiry() { - try { + getMessageTTL().thenAccept(messageTtlInSeconds -> { //If topic level policy or message ttl is not set, fall back to namespace level config. - int messageTtlInSeconds = getMessageTTL(); if (messageTtlInSeconds != 0) { subscriptions.forEach((__, sub) -> sub.expireMessages(messageTtlInSeconds)); replicators.forEach((__, replicator) -> ((PersistentReplicator) replicator).expireMessages(messageTtlInSeconds)); } - } catch (Exception e) { - if (log.isDebugEnabled()) { - log.debug("[{}] Error getting policies", topic); - } - } + }); } @Override @@ -1329,7 +1340,7 @@ public void checkCompaction() { if (compactionThreshold == null) { Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache() .get(AdminResource.path(POLICIES, name.getNamespace())) - .orElseThrow(() -> new KeeperException.NoNodeException()); + .orElseThrow(() -> new MetadataStoreException.NotFoundException()); compactionThreshold = policies.compaction_threshold; } if (compactionThreshold == null) { @@ -2139,13 +2150,12 @@ public void checkInactiveSubscriptions() { try { Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache() .get(AdminResource.path(POLICIES, name.getNamespace())) - .orElseThrow(() -> new KeeperException.NoNodeException()); + .orElseThrow(() -> new MetadataStoreException.NotFoundException()); final int defaultExpirationTime = brokerService.pulsar().getConfiguration() .getSubscriptionExpirationTimeMinutes(); + final Integer nsExpirationTime = policies.subscription_expiration_time_minutes; final long expirationTimeMillis = TimeUnit.MINUTES - .toMillis((policies.subscription_expiration_time_minutes <= 0 && defaultExpirationTime > 0) - ? defaultExpirationTime - : policies.subscription_expiration_time_minutes); + .toMillis(nsExpirationTime == null ? defaultExpirationTime : nsExpirationTime); if (expirationTimeMillis > 0) { subscriptions.forEach((subName, sub) -> { if (sub.dispatcher != null && sub.dispatcher.isConsumerConnected() || sub.isReplicated()) { @@ -2655,21 +2665,26 @@ public synchronized OffloadProcessStatus offloadStatus() { * Get message TTL for this topic. * @return Message TTL in second. */ - private int getMessageTTL() throws Exception { + private CompletableFuture getMessageTTL() { //Return Topic level message TTL if exist. If topic level policy or message ttl is not set, //fall back to namespace level message ttl then message ttl set for current broker. Optional messageTtl = getTopicPolicies().map(TopicPolicies::getMessageTTLInSeconds); if (messageTtl.isPresent()) { - return messageTtl.get(); + return CompletableFuture.completedFuture(messageTtl.get()); } TopicName name = TopicName.get(topic); - Policies policies = brokerService.pulsar().getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, name.getNamespace())) - .orElseThrow(KeeperException.NoNodeException::new); - if (policies.message_ttl_in_seconds != null) { - return policies.message_ttl_in_seconds; - } - return brokerService.getPulsar().getConfiguration().getTtlDurationDefaultInSeconds(); + + return brokerService.pulsar().getPulsarResources().getNamespaceResources() + .getAsync(AdminResource.path(POLICIES, TopicName.get(topic).getNamespace())) + .thenApply(optPolicies -> { + if (optPolicies.isPresent()) { + if (optPolicies.get().message_ttl_in_seconds != null) { + return optPolicies.get().message_ttl_in_seconds; + } + } + + return brokerService.getPulsar().getConfiguration().getTtlDurationDefaultInSeconds(); + }); } private static final Logger log = LoggerFactory.getLogger(PersistentTopic.class); 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 c88063f37e29d9..05f1b15f530e0d 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 @@ -88,6 +88,9 @@ void updateStats(TopicStats stats) { replStats.msgThroughputIn += as.msgThroughputIn; replStats.msgThroughputOut += as.msgThroughputOut; replStats.replicationBacklog += as.replicationBacklog; + replStats.msgRateExpired += as.msgRateExpired; + replStats.connectedCount += as.connectedCount; + replStats.replicationDelayInSeconds += as.replicationDelayInSeconds; }); stats.subscriptionStats.forEach((n, as) -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedReplicationStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedReplicationStats.java index b132e7e43776d5..ca92d557143e50 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedReplicationStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedReplicationStats.java @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.stats.prometheus; public class AggregatedReplicationStats { + + /** Total rate of messages received from the remote cluster (msg/s). */ public double msgRateIn; /** Total throughput received from the remote cluster. bytes/s */ @@ -30,9 +32,16 @@ public class AggregatedReplicationStats { /** Total throughput delivered to the replication-subscriber. bytes/s */ public double msgThroughputOut; - /** - * Number of messages pending to be replicated to remote cluster. - */ + /** Total rate of messages expired (msg/s). */ + public double msgRateExpired; + + /** Number of messages pending to be replicated to remote cluster. */ public long replicationBacklog; + /** The count of replication-subscriber up and running to replicate to remote cluster. */ + public long connectedCount; + + /** Time in seconds from the time a message was produced to the time when it is about to be replicated. */ + public long replicationDelayInSeconds; + } 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 034e83e67af658..18c4f97f40c385 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 @@ -210,6 +210,11 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include aggReplStats.msgRateOut += replStats.msgRateOut; aggReplStats.msgThroughputOut += replStats.msgThroughputOut; aggReplStats.replicationBacklog += replStats.replicationBacklog; + aggReplStats.msgRateIn += replStats.msgRateIn; + aggReplStats.msgThroughputIn += replStats.msgThroughputIn; + aggReplStats.msgRateExpired += replStats.msgRateExpired; + aggReplStats.connectedCount += replStats.connected ? 1 : 0; + aggReplStats.replicationDelayInSeconds += replStats.replicationDelayInSeconds; }); } @@ -327,6 +332,12 @@ private static void printNamespaceStats(SimpleTextOutputStream stream, String cl replStats.msgThroughputOut); metricWithRemoteCluster(stream, cluster, namespace, "pulsar_replication_backlog", remoteCluster, replStats.replicationBacklog); + metricWithRemoteCluster(stream, cluster, namespace, "pulsar_replication_connected_count", remoteCluster, + replStats.connectedCount); + metricWithRemoteCluster(stream, cluster, namespace, "pulsar_replication_rate_expired", remoteCluster, + replStats.msgRateExpired); + metricWithRemoteCluster(stream, cluster, namespace, "pulsar_replication_delay_in_seconds", + remoteCluster, replStats.replicationDelayInSeconds); }); } } 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 557d7ff352116a..0649235a36960f 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 @@ -237,6 +237,12 @@ static void printTopicStats(SimpleTextOutputStream stream, String cluster, Strin replStats.msgThroughputOut); metricWithRemoteCluster(stream, cluster, namespace, topic, "pulsar_replication_backlog", remoteCluster, replStats.replicationBacklog); + metricWithRemoteCluster(stream, cluster, namespace, topic, "pulsar_replication_connected_count", + remoteCluster, replStats.connectedCount); + metricWithRemoteCluster(stream, cluster, namespace, topic, "pulsar_replication_rate_expired", + remoteCluster, replStats.msgRateExpired); + metricWithRemoteCluster(stream, cluster, namespace, topic, "pulsar_replication_delay_in_seconds", + remoteCluster, replStats.replicationDelayInSeconds); }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/tools/BrokerTool.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/tools/BrokerTool.java index a0f7ee24f888af..b710f22aec018a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/tools/BrokerTool.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/tools/BrokerTool.java @@ -29,19 +29,23 @@ public class BrokerTool { public static final String NAME = "broker-tool"; - public static void main(String[] args) { + public static int run(String[] args) { CliSpec.Builder specBuilder = CliSpec.newBuilder() .withName(NAME) .withUsage(NAME + " [flags] [commands]") .withDescription(NAME + " is used for operations on a specific broker") .withFlags(new CliFlags()) .withConsole(System.out) - .addCommand(new LoadReportCommand()); + .addCommand(new LoadReportCommand()) + .addCommand(new GenerateDocsCommand()); CliSpec spec = specBuilder.build(); - int retCode = Cli.runCli(spec, args); - Runtime.getRuntime().exit(retCode); + return Cli.runCli(spec, args); } + public static void main(String[] args) { + int retCode = run(args); + Runtime.getRuntime().exit(retCode); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/tools/GenerateDocsCommand.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/tools/GenerateDocsCommand.java new file mode 100644 index 00000000000000..afc570fd307678 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/tools/GenerateDocsCommand.java @@ -0,0 +1,80 @@ +/** + * 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.tools; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import java.util.ArrayList; +import java.util.List; +import org.apache.bookkeeper.tools.framework.Cli; +import org.apache.bookkeeper.tools.framework.CliCommand; +import org.apache.bookkeeper.tools.framework.CliFlags; +import org.apache.bookkeeper.tools.framework.CliSpec; +import org.apache.pulsar.common.util.CmdGenerateDocs; + +/** + * The command to generate documents of broker-tool. + */ +public class GenerateDocsCommand extends CliCommand { + + private static final String NAME = "gen-doc"; + private static final String DESC = "Generate documents of broker-tool"; + + /** + * The CLI flags of gen docs command. + */ + protected static class GenDocFlags extends CliFlags { + @Parameter( + names = {"-n", "--command-names"}, + description = "List of command names" + ) + private List commandNames = new ArrayList<>(); + } + + public GenerateDocsCommand() { + super(CliSpec.newBuilder() + .withName(NAME) + .withDescription(DESC) + .withFlags(new GenDocFlags()) + .build()); + } + + @Override + public Boolean apply(CliFlags globalFlags, String[] args) { + CliSpec newSpec = CliSpec.newBuilder(spec) + .withRunFunc(cmdFlags -> apply(cmdFlags)) + .build(); + return 0 == Cli.runCli(newSpec, args); + } + + private boolean apply(GenerateDocsCommand.GenDocFlags flags) { + CmdGenerateDocs cmd = new CmdGenerateDocs("pulsar"); + JCommander commander = new JCommander(); + commander.addCommand("load-report", new LoadReportCommand.Flags()); + cmd.addCommand("broker-tool", commander); + if (flags.commandNames.isEmpty()) { + cmd.run(null); + } else { + ArrayList args = new ArrayList(flags.commandNames); + args.add(0, "-n"); + cmd.run(args.toArray(new String[0])); + } + return true; + } +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/api/RawReader.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/api/RawReader.java index 415c3dce6d88c7..f74157a9389143 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/client/api/RawReader.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/api/RawReader.java @@ -34,7 +34,7 @@ public interface RawReader { static CompletableFuture create(PulsarClient client, String topic, String subscription) { CompletableFuture> future = new CompletableFuture<>(); RawReader r = new RawReaderImpl((PulsarClientImpl) client, topic, subscription, future); - return future.thenCompose((consumer) -> r.seekAsync(MessageId.earliest)).thenApply((ignore) -> r); + return future.thenCompose(x -> x.seekAsync(MessageId.earliest)).thenApply(__ -> r); } /** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java index 88b8e5826d9b94..4922852bda465f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java @@ -22,9 +22,13 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; +import org.apache.pulsar.broker.service.Consumer; public interface CompactedTopic { CompletableFuture newCompactedLedger(Position p, long compactedLedgerId); - void asyncReadEntriesOrWait(ManagedCursor cursor, int numberOfEntriesToRead, - ReadEntriesCallback callback, Object ctx); + void asyncReadEntriesOrWait(ManagedCursor cursor, + int numberOfEntriesToRead, + boolean isFirstRead, + ReadEntriesCallback callback, + Consumer consumer); } 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 c4646edf8dca02..b061a424b131cf 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 @@ -42,6 +42,8 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.RawMessage; import org.apache.pulsar.client.impl.RawMessageImpl; import org.apache.pulsar.common.api.proto.MessageIdData; @@ -81,13 +83,20 @@ public CompletableFuture newCompactedLedger(Position p, long compactedLedgerI } @Override - public void asyncReadEntriesOrWait(ManagedCursor cursor, int numberOfEntriesToRead, - ReadEntriesCallback callback, Object ctx) { + public void asyncReadEntriesOrWait(ManagedCursor cursor, + int numberOfEntriesToRead, + boolean isFirstRead, + ReadEntriesCallback callback, Consumer consumer) { synchronized (this) { - PositionImpl cursorPosition = (PositionImpl) cursor.getReadPosition(); + PositionImpl cursorPosition; + if (isFirstRead && MessageId.earliest.equals(consumer.getStartMessageId())){ + cursorPosition = PositionImpl.earliest; + } else { + cursorPosition = (PositionImpl) cursor.getReadPosition(); + } if (compactionHorizon == null || compactionHorizon.compareTo(cursorPosition) < 0) { - cursor.asyncReadEntriesOrWait(numberOfEntriesToRead, callback, ctx, PositionImpl.latest); + cursor.asyncReadEntriesOrWait(numberOfEntriesToRead, callback, consumer, PositionImpl.latest); } else { compactedTopicContext.thenCompose( (context) -> findStartPoint(cursorPosition, context.ledger.getLastAddConfirmed(), context.cache) @@ -96,11 +105,11 @@ public void asyncReadEntriesOrWait(ManagedCursor cursor, int numberOfEntriesToRe // the cursor just needs to be set to the compaction horizon if (startPoint == COMPACT_LEDGER_EMPTY) { cursor.seek(compactionHorizon.getNext()); - callback.readEntriesComplete(Collections.emptyList(), ctx); + callback.readEntriesComplete(Collections.emptyList(), consumer); return CompletableFuture.completedFuture(null); } if (startPoint == NEWER_THAN_COMPACTED && compactionHorizon.compareTo(cursorPosition) < 0) { - cursor.asyncReadEntriesOrWait(numberOfEntriesToRead, callback, ctx, + cursor.asyncReadEntriesOrWait(numberOfEntriesToRead, callback, consumer, PositionImpl.latest); return CompletableFuture.completedFuture(null); } else { @@ -108,23 +117,23 @@ public void asyncReadEntriesOrWait(ManagedCursor cursor, int numberOfEntriesToRe startPoint + numberOfEntriesToRead); if (startPoint == NEWER_THAN_COMPACTED) { cursor.seek(compactionHorizon.getNext()); - callback.readEntriesComplete(Collections.emptyList(), ctx); + callback.readEntriesComplete(Collections.emptyList(), consumer); return CompletableFuture.completedFuture(null); } return readEntries(context.ledger, startPoint, endPoint) .thenAccept((entries) -> { Entry lastEntry = entries.get(entries.size() - 1); cursor.seek(lastEntry.getPosition().getNext()); - callback.readEntriesComplete(entries, ctx); + callback.readEntriesComplete(entries, consumer); }); } })) .exceptionally((exception) -> { if (exception.getCause() instanceof NoSuchElementException) { cursor.seek(compactionHorizon.getNext()); - callback.readEntriesComplete(Collections.emptyList(), ctx); + callback.readEntriesComplete(Collections.emptyList(), consumer); } else { - callback.readEntriesFailed(new ManagedLedgerException(exception), ctx); + callback.readEntriesFailed(new ManagedLedgerException(exception), consumer); } return null; }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java index ee2b374e56b928..ff01a1d9f89869 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactorTool.java @@ -39,6 +39,7 @@ import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.apache.pulsar.common.util.CmdGenerateDocs; import org.apache.pulsar.common.util.netty.EventLoopUtil; import org.apache.pulsar.zookeeper.ZooKeeperClientFactory; import org.apache.pulsar.zookeeper.ZookeeperBkClientFactoryImpl; @@ -57,6 +58,9 @@ private static class Arguments { @Parameter(names = {"-h", "--help"}, description = "Show this help message") private boolean help = false; + + @Parameter(names = {"-g", "--generate-docs"}, description = "Generate docs") + private boolean generateDocs = false; } public static void main(String[] args) throws Exception { @@ -71,6 +75,13 @@ public static void main(String[] args) throws Exception { System.exit(-1); } + if (arguments.generateDocs) { + CmdGenerateDocs cmd = new CmdGenerateDocs("pulsar"); + cmd.addCommand("compact-topic", arguments); + cmd.run(null); + System.exit(-1); + } + // init broker config ServiceConfiguration brokerConfig; if (isBlank(arguments.brokerConfigFile)) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtils.java index ee2c384646fac1..bbf9d9fcf765d1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtils.java @@ -44,6 +44,7 @@ import javax.crypto.SecretKey; import lombok.Cleanup; import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; +import org.apache.pulsar.common.util.CmdGenerateDocs; import org.apache.pulsar.common.util.RelativeTimeUtil; public class TokensCliUtils { @@ -303,6 +304,8 @@ public static void main(String[] args) throws Exception { CommandValidateToken commandValidateToken = new CommandValidateToken(); jcommander.addCommand("validate", commandValidateToken); + jcommander.addCommand("gen-doc", new Object()); + try { jcommander.parse(args); @@ -329,6 +332,10 @@ public static void main(String[] args) throws Exception { commandShowToken.run(); } else if (cmd.equals("validate")) { commandValidateToken.run(); + } else if (cmd.equals("gen-doc")) { + CmdGenerateDocs genDocCmd = new CmdGenerateDocs("pulsar"); + genDocCmd.addCommand("tokens", jcommander); + genDocCmd.run(null); } else { System.err.println("Invalid command: " + cmd); System.exit(1); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/PulsarBrokerStarterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/PulsarBrokerStarterTest.java index 37877f43ac8f88..29658abf677b3f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/PulsarBrokerStarterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/PulsarBrokerStarterTest.java @@ -22,17 +22,23 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; - +import com.beust.jcommander.Parameter; import com.google.common.collect.Sets; +import java.io.ByteArrayOutputStream; import java.io.File; import java.io.FileNotFoundException; import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStreamWriter; +import java.io.PrintStream; import java.io.PrintWriter; +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; +import java.util.Arrays; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.common.util.CmdGenerateDocs; import org.testng.annotations.Test; @Test(groups = "broker") @@ -347,4 +353,38 @@ public void testMainEnableRunBookieThroughBrokerConfig() throws Exception { // code should reach here. } } + + @Test + public void testMainGenerateDocs() throws Exception { + PrintStream oldStream = System.out; + try { + ByteArrayOutputStream baoStream = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baoStream)); + + Class argumentsClass = Class.forName("org.apache.pulsar.PulsarBrokerStarter$StarterArguments"); + Constructor constructor = argumentsClass.getDeclaredConstructor(); + constructor.setAccessible(true); + Object obj = constructor.newInstance(); + + CmdGenerateDocs cmd = new CmdGenerateDocs("pulsar"); + cmd.addCommand("broker", obj); + cmd.run(null); + + String message = baoStream.toString(); + + Field[] fields = argumentsClass.getDeclaredFields(); + for (Field field : fields) { + boolean fieldHasAnno = field.isAnnotationPresent(Parameter.class); + if (fieldHasAnno) { + Parameter fieldAnno = field.getAnnotation(Parameter.class); + String[] names = fieldAnno.names(); + String nameStr = Arrays.asList(names).toString(); + nameStr = nameStr.substring(1, nameStr.length() - 1); + assertTrue(message.indexOf(nameStr) > 0); + } + } + } finally { + System.setOut(oldStream); + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java index 533a2a5afe91d0..3e855234e538a3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java @@ -87,7 +87,6 @@ protected void setup() throws Exception { persistentTopics.setServletContext(new MockServletContext()); persistentTopics.setPulsar(pulsar); - doReturn(mockZooKeeper).when(persistentTopics).localZk(); doReturn(false).when(persistentTopics).isRequestHttps(); doReturn(null).when(persistentTopics).originalPrincipal(); doReturn("test").when(persistentTopics).clientAppId(); 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 f16d2a661a380e..997a93411a652a 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 @@ -1219,6 +1219,9 @@ public void testDeleteTenantForcefully() throws Exception { assertFalse(admin.tenants().getTenants().contains(tenant)); }); + final String managedLedgerPathForTenant = "/managed-ledgers/" + tenant; + assertFalse(pulsar.getLocalMetadataStore().exists(managedLedgerPathForTenant).join()); + admin.tenants().createTenant(tenant, new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test"))); assertTrue(admin.tenants().getTenants().contains(tenant)); @@ -1227,6 +1230,57 @@ public void testDeleteTenantForcefully() throws Exception { // reset back to false pulsar.getConfiguration().setForceDeleteTenantAllowed(false); pulsar.getConfiguration().setForceDeleteNamespaceAllowed(false); + + } + + @Test + public void testDeleteNamespaceForcefully() throws Exception { + // allow forced deletion of namespaces + pulsar.getConfiguration().setForceDeleteNamespaceAllowed(true); + + String tenant = "my-tenant"; + assertFalse(admin.tenants().getTenants().contains(tenant)); + + // create tenant + admin.tenants().createTenant(tenant, + new TenantInfoImpl(Sets.newHashSet("role1", "role2"), Sets.newHashSet("test"))); + + assertTrue(admin.tenants().getTenants().contains(tenant)); + + // create namespace + String namespace = tenant + "/my-ns"; + admin.namespaces().createNamespace("my-tenant/my-ns", Sets.newHashSet("test")); + + assertEquals(admin.namespaces().getNamespaces(tenant), Lists.newArrayList("my-tenant/my-ns")); + + // create topic + String topic = namespace + "/my-topic"; + admin.topics().createPartitionedTopic(topic, 10); + + assertFalse(admin.topics().getList(namespace).isEmpty()); + + try { + admin.namespaces().deleteNamespace(namespace, false); + fail("should have failed due to namespace not empty"); + } catch (PulsarAdminException e) { + // Expected: cannot delete non-empty tenant + } + + // delete namespace forcefully + admin.namespaces().deleteNamespace(namespace, true); + assertFalse(admin.namespaces().getNamespaces(tenant).contains(namespace)); + assertTrue(admin.namespaces().getNamespaces(tenant).isEmpty()); + + final String managedLedgerPath = "/managed-ledgers/" + namespace; + final String persistentDomain = managedLedgerPath + "/" + TopicDomain.persistent.value(); + final String nonPersistentDomain = managedLedgerPath + "/" + TopicDomain.non_persistent.value(); + + assertFalse(pulsar.getLocalMetadataStore().exists(managedLedgerPath).join()); + assertFalse(pulsar.getLocalMetadataStore().exists(persistentDomain).join()); + assertFalse(pulsar.getLocalMetadataStore().exists(nonPersistentDomain).join()); + + // reset back to false + pulsar.getConfiguration().setForceDeleteNamespaceAllowed(false); } @Test @@ -2768,19 +2822,27 @@ public void testTopicStatsLastExpireTimestampForSubscription() throws PulsarAdmi public void testSubscriptionExpiry() throws Exception { final String namespace1 = "prop-xyz/sub-gc1"; final String namespace2 = "prop-xyz/sub-gc2"; + final String namespace3 = "prop-xyz/sub-gc3"; final String topic1 = "persistent://" + namespace1 + "/testSubscriptionExpiry"; final String topic2 = "persistent://" + namespace2 + "/testSubscriptionExpiry"; + final String topic3 = "persistent://" + namespace3 + "/testSubscriptionExpiry"; final String sub = "sub1"; admin.namespaces().createNamespace(namespace1, Sets.newHashSet("test")); admin.namespaces().createNamespace(namespace2, Sets.newHashSet("test")); + admin.namespaces().createNamespace(namespace3, Sets.newHashSet("test")); admin.topics().createSubscription(topic1, sub, MessageId.latest); admin.topics().createSubscription(topic2, sub, MessageId.latest); + admin.topics().createSubscription(topic3, sub, MessageId.latest); admin.namespaces().setSubscriptionExpirationTime(namespace1, 0); admin.namespaces().setSubscriptionExpirationTime(namespace2, 1); + admin.namespaces().setSubscriptionExpirationTime(namespace3, 1); + admin.namespaces().removeSubscriptionExpirationTime(namespace3); + + Assert.assertEquals((int) admin.namespaces().getSubscriptionExpirationTime(namespace1), 0); + Assert.assertEquals((int) admin.namespaces().getSubscriptionExpirationTime(namespace2), 1); + Assert.assertNull(admin.namespaces().getSubscriptionExpirationTime(namespace3)); - Assert.assertEquals(admin.namespaces().getSubscriptionExpirationTime(namespace1), 0); - Assert.assertEquals(admin.namespaces().getSubscriptionExpirationTime(namespace2), 1); Thread.sleep(60000); for (int i = 0; i < 60; i++) { if (admin.topics().getSubscriptions(topic2).size() == 0) { @@ -2790,11 +2852,14 @@ public void testSubscriptionExpiry() throws Exception { } Assert.assertEquals(admin.topics().getSubscriptions(topic1).size(), 1); Assert.assertEquals(admin.topics().getSubscriptions(topic2).size(), 0); + Assert.assertEquals(admin.topics().getSubscriptions(topic3).size(), 1); admin.topics().delete(topic1); admin.topics().delete(topic2); + admin.topics().delete(topic3); admin.namespaces().deleteNamespace(namespace1); admin.namespaces().deleteNamespace(namespace2); + admin.namespaces().deleteNamespace(namespace3); } @Test 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 80fd835bb722e7..cf8b17efd49eab 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 @@ -150,7 +150,6 @@ public void setup() throws Exception { namespaces = spy(new Namespaces()); namespaces.setServletContext(new MockServletContext()); namespaces.setPulsar(pulsar); - doReturn(mockZooKeeper).when(namespaces).localZk(); doReturn("test").when(namespaces).clientAppId(); doReturn(Sets.newTreeSet(Lists.newArrayList("use", "usw", "usc", "global"))).when(namespaces).clusters(); doNothing().when(namespaces).validateAdminAccessForTenant("my-tenant"); @@ -168,7 +167,6 @@ public void setup() throws Exception { persistentTopics = spy(new PersistentTopics()); persistentTopics.setServletContext(new MockServletContext()); persistentTopics.setPulsar(pulsar); - doReturn(mockZooKeeper).when(persistentTopics).localZk(); doReturn("test").when(persistentTopics).clientAppId(); doReturn("persistent").when(persistentTopics).domain(); doReturn(Sets.newTreeSet(Lists.newArrayList("use", "usw", "usc"))).when(persistentTopics).clusters(); 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 afae4764a613b9..b2eda82ac31b52 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 @@ -161,7 +161,6 @@ public void setup() throws Exception { namespaces = spy(new Namespaces()); namespaces.setServletContext(new MockServletContext()); namespaces.setPulsar(pulsar); - doReturn(mockZooKeeper).when(namespaces).localZk(); doReturn(false).when(namespaces).isRequestHttps(); doReturn("test").when(namespaces).clientAppId(); doReturn(null).when(namespaces).originalPrincipal(); 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 d61b86e573ca9b..9c8b596b58b23a 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 @@ -114,7 +114,6 @@ protected void setup() throws Exception { persistentTopics = spy(new PersistentTopics()); persistentTopics.setServletContext(new MockServletContext()); persistentTopics.setPulsar(pulsar); - doReturn(mockZooKeeper).when(persistentTopics).localZk(); doReturn(false).when(persistentTopics).isRequestHttps(); doReturn(null).when(persistentTopics).originalPrincipal(); doReturn("test").when(persistentTopics).clientAppId(); @@ -125,7 +124,6 @@ protected void setup() throws Exception { nonPersistentTopic = spy(new NonPersistentTopics()); nonPersistentTopic.setServletContext(new MockServletContext()); nonPersistentTopic.setPulsar(pulsar); - doReturn(mockZooKeeper).when(nonPersistentTopic).localZk(); doReturn(false).when(nonPersistentTopic).isRequestHttps(); doReturn(null).when(nonPersistentTopic).originalPrincipal(); doReturn("test").when(nonPersistentTopic).clientAppId(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ResourceGroupsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ResourceGroupsTest.java index 40d82ca450e5da..b6510a18995bf4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ResourceGroupsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/ResourceGroupsTest.java @@ -55,7 +55,6 @@ protected void setup() throws Exception { resourcegroups = spy(new ResourceGroups()); resourcegroups.setServletContext(new MockServletContext()); resourcegroups.setPulsar(pulsar); - doReturn(mockZooKeeper).when(resourcegroups).localZk(); doReturn(false).when(resourcegroups).isRequestHttps(); doReturn("test").when(resourcegroups).clientAppId(); doReturn(null).when(resourcegroups).originalPrincipal(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicMessageTTLTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicMessageTTLTest.java index afe1144712d2c3..4a2c728bc07895 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicMessageTTLTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicMessageTTLTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.admin; import com.google.common.collect.Sets; +import java.util.concurrent.CompletableFuture; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -162,22 +163,22 @@ public void testDifferentLevelPolicyPriority() throws Exception { Integer namespaceMessageTTL = admin.namespaces().getNamespaceMessageTTL(myNamespace); Assert.assertNull(namespaceMessageTTL); - Assert.assertEquals(method.invoke(persistentTopic), 3600); + Assert.assertEquals((int) ((CompletableFuture) method.invoke(persistentTopic)).join(), 3600); admin.namespaces().setNamespaceMessageTTL(myNamespace, 10); Awaitility.await().untilAsserted(() -> Assert.assertEquals(admin.namespaces().getNamespaceMessageTTL(myNamespace).intValue(), 10)); - Assert.assertEquals((int)method.invoke(persistentTopic), 10); + Assert.assertEquals((int) ((CompletableFuture) method.invoke(persistentTopic)).join(), 10); admin.namespaces().setNamespaceMessageTTL(myNamespace, 0); Awaitility.await().untilAsserted(() -> Assert.assertEquals(admin.namespaces().getNamespaceMessageTTL(myNamespace).intValue(), 0)); - Assert.assertEquals((int)method.invoke(persistentTopic), 0); + Assert.assertEquals((int) ((CompletableFuture) method.invoke(persistentTopic)).join(), 0); admin.namespaces().removeNamespaceMessageTTL(myNamespace); Awaitility.await().untilAsserted(() -> Assert.assertNull(admin.namespaces().getNamespaceMessageTTL(myNamespace))); - Assert.assertEquals((int)method.invoke(persistentTopic), 3600); + Assert.assertEquals((int) ((CompletableFuture) method.invoke(persistentTopic)).join(), 3600); } @Test(timeOut = 20000) @@ -219,7 +220,7 @@ public void testDifferentLevelPolicyApplied() throws Exception { admin.topics().removeMessageTTL(topicName); Awaitility.await().untilAsserted(() -> Assert.assertEquals(admin.topics().getMessageTTL(topicName, true).intValue(), 3600)); - Assert.assertEquals((int)method.invoke(persistentTopic), 3600); + Assert.assertEquals((int) ((CompletableFuture)method.invoke(persistentTopic)).join(), 3600); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java index 8001f4d8ac0563..1667de2acc0bde 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java @@ -187,6 +187,7 @@ public void testSplitMapWithRefreshedStatMap() throws Exception { NamespaceBundle originalBundle = bundles.findBundle(topicName); PersistentTopic topic = new PersistentTopic(topicName.toString(), ledger, pulsar.getBrokerService()); + topic.initialize().join(); Method method = pulsar.getBrokerService().getClass().getDeclaredMethod("addTopicToStatsMaps", TopicName.class, Topic.class); method.setAccessible(true); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoaderTest.java index 0704d63ea62288..42d26d53f27c6f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoaderTest.java @@ -25,8 +25,14 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.expectThrows; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.socket.SocketChannel; +import java.net.InetSocketAddress; +import java.util.Map; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.common.nar.NarClassLoader; @@ -68,4 +74,79 @@ public void testWrapper() throws Exception { verify(h, times(1)).getProtocolDataToAdvertise(); } + public void testClassLoaderSwitcher() throws Exception { + NarClassLoader loader = mock(NarClassLoader.class); + + String protocol = "test-protocol"; + + ProtocolHandler h = new ProtocolHandler() { + @Override + public String protocolName() { + assertEquals(Thread.currentThread().getContextClassLoader(), loader); + return protocol; + } + + @Override + public boolean accept(String protocol) { + assertEquals(Thread.currentThread().getContextClassLoader(), loader); + return true; + } + + @Override + public void initialize(ServiceConfiguration conf) throws Exception { + assertEquals(Thread.currentThread().getContextClassLoader(), loader); + throw new Exception("test exception"); + } + + @Override + public String getProtocolDataToAdvertise() { + assertEquals(Thread.currentThread().getContextClassLoader(), loader); + return "test-protocol-data"; + } + + @Override + public void start(BrokerService service) { + assertEquals(Thread.currentThread().getContextClassLoader(), loader); + } + + @Override + public Map> newChannelInitializers() { + assertEquals(Thread.currentThread().getContextClassLoader(), loader); + return null; + } + + @Override + public void close() { + assertEquals(Thread.currentThread().getContextClassLoader(), loader); + } + }; + ProtocolHandlerWithClassLoader wrapper = new ProtocolHandlerWithClassLoader(h, loader); + + ClassLoader curClassLoader = Thread.currentThread().getContextClassLoader(); + + assertEquals(wrapper.protocolName(), protocol); + assertEquals(Thread.currentThread().getContextClassLoader(), curClassLoader); + + assertTrue(wrapper.accept(protocol)); + assertEquals(Thread.currentThread().getContextClassLoader(), curClassLoader); + + + ServiceConfiguration conf = new ServiceConfiguration(); + expectThrows(Exception.class, () -> wrapper.initialize(conf)); + assertEquals(Thread.currentThread().getContextClassLoader(), curClassLoader); + + assertEquals(wrapper.getProtocolDataToAdvertise(), "test-protocol-data"); + assertEquals(Thread.currentThread().getContextClassLoader(), curClassLoader); + + BrokerService service = mock(BrokerService.class); + wrapper.start(service); + assertEquals(Thread.currentThread().getContextClassLoader(), curClassLoader); + + + assertNull(wrapper.newChannelInitializers()); + assertEquals(Thread.currentThread().getContextClassLoader(), curClassLoader); + + wrapper.close(); + assertEquals(Thread.currentThread().getContextClassLoader(), curClassLoader); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java index e7d98a829934b9..52c8375fa78a34 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java @@ -25,6 +25,9 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.impl.MessageImpl; +import org.apache.pulsar.common.api.proto.BrokerEntryMetadata; import org.assertj.core.util.Sets; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -38,6 +41,7 @@ @Test(groups = "broker") public class BrokerEntryMetadataE2ETest extends BrokerTestBase { + @DataProvider(name = "subscriptionTypes") public static Object[] subscriptionTypes() { return new Object[] { @@ -97,17 +101,98 @@ public void testProduceAndConsume(SubscriptionType subType) throws Exception { public void testPeekMessage() throws Exception { final String topic = newTopicName(); final String subscription = "my-sub"; + final long eventTime= 200; + final long deliverAtTime = 300; @Cleanup Producer producer = pulsarClient.newProducer() .topic(topic) .create(); - producer.newMessage().value("hello".getBytes()).send(); + + long sendTime = System.currentTimeMillis(); + producer.newMessage() + .eventTime(eventTime) + .deliverAt(deliverAtTime) + .value("hello".getBytes()) + .send(); admin.topics().createSubscription(topic, subscription, MessageId.earliest); final List> messages = admin.topics().peekMessages(topic, subscription, 1); Assert.assertEquals(messages.size(), 1); - Assert.assertEquals(messages.get(0).getData(), "hello".getBytes()); + MessageImpl message = (MessageImpl) messages.get(0); + Assert.assertEquals(message.getData(), "hello".getBytes()); + Assert.assertEquals(message.getEventTime(), eventTime); + Assert.assertEquals(message.getDeliverAtTime(), deliverAtTime); + Assert.assertTrue(message.getPublishTime() >= sendTime); + + BrokerEntryMetadata entryMetadata = message.getBrokerEntryMetadata(); + Assert.assertEquals(entryMetadata.getIndex(), 0); + Assert.assertTrue(entryMetadata.getBrokerTimestamp() >= sendTime); + } + + @Test(timeOut = 20000) + public void testGetMessageById() throws Exception { + final String topic = newTopicName(); + final String subscription = "my-sub"; + final long eventTime= 200; + final long deliverAtTime = 300; + + @Cleanup + Producer producer = pulsarClient.newProducer() + .topic(topic) + .create(); + + long sendTime = System.currentTimeMillis(); + MessageIdImpl messageId = (MessageIdImpl) producer.newMessage() + .eventTime(eventTime) + .deliverAt(deliverAtTime) + .value("hello".getBytes()) + .send(); + + admin.topics().createSubscription(topic, subscription, MessageId.earliest); + MessageImpl message = (MessageImpl) admin.topics() + .getMessageById(topic, messageId.getLedgerId(), messageId.getEntryId()); + Assert.assertEquals(message.getData(), "hello".getBytes()); + Assert.assertEquals(message.getEventTime(), eventTime); + Assert.assertEquals(message.getDeliverAtTime(), deliverAtTime); + Assert.assertTrue(message.getPublishTime() >= sendTime); + + BrokerEntryMetadata entryMetadata = message.getBrokerEntryMetadata(); + Assert.assertEquals(entryMetadata.getIndex(), 0); + Assert.assertTrue(entryMetadata.getBrokerTimestamp() >= sendTime); + } + + + @Test(timeOut = 20000) + public void testExamineMessage() throws Exception { + final String topic = newTopicName(); + final String subscription = "my-sub"; + final long eventTime= 200; + final long deliverAtTime = 300; + + @Cleanup + Producer producer = pulsarClient.newProducer() + .topic(topic) + .create(); + + long sendTime = System.currentTimeMillis(); + producer.newMessage() + .eventTime(eventTime) + .deliverAt(deliverAtTime) + .value("hello".getBytes()) + .send(); + + admin.topics().createSubscription(topic, subscription, MessageId.earliest); + MessageImpl message = + (MessageImpl) admin.topics().examineMessage(topic, "earliest", 1); + Assert.assertEquals(message.getData(), "hello".getBytes()); + Assert.assertEquals(message.getEventTime(), eventTime); + Assert.assertEquals(message.getDeliverAtTime(), deliverAtTime); + Assert.assertTrue(message.getPublishTime() >= sendTime); + + BrokerEntryMetadata entryMetadata = message.getBrokerEntryMetadata(); + Assert.assertEquals(entryMetadata.getIndex(), 0); + Assert.assertTrue(entryMetadata.getBrokerTimestamp() >= sendTime); } @Test(timeOut = 20000) 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 6cccfee8e0659c..692ea221710a73 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 @@ -73,6 +73,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.transaction.TransactionTestBase; +import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.common.api.proto.BaseCommand; import org.apache.pulsar.common.api.proto.CommandActiveConsumerChange; import org.apache.pulsar.common.api.proto.CommandSubscribe.InitialPosition; @@ -299,7 +300,7 @@ public void testConsumerGroupChangesWithOldNewConsumers() throws Exception { // 2. Add old consumer Consumer consumer1 = new Consumer(sub, SubType.Exclusive, topic.getName(), 1 /* consumer id */, 0, - "Cons1"/* consumer name */, 50000, serverCnxWithOldVersion, "myrole-1", Collections.emptyMap(), false, InitialPosition.Latest, null); + "Cons1"/* consumer name */, 50000, serverCnxWithOldVersion, "myrole-1", Collections.emptyMap(), false, InitialPosition.Latest, null, MessageId.latest); pdfc.addConsumer(consumer1); List consumers = pdfc.getConsumers(); assertSame(consumers.get(0).consumerName(), consumer1.consumerName()); @@ -310,7 +311,7 @@ public void testConsumerGroupChangesWithOldNewConsumers() throws Exception { // 3. Add new consumer Consumer consumer2 = new Consumer(sub, SubType.Exclusive, topic.getName(), 2 /* consumer id */, 0, - "Cons2"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), false, InitialPosition.Latest, null); + "Cons2"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), false, InitialPosition.Latest, null, MessageId.latest); pdfc.addConsumer(consumer2); consumers = pdfc.getConsumers(); assertSame(consumers.get(0).consumerName(), consumer1.consumerName()); @@ -339,7 +340,7 @@ public void testAddRemoveConsumer() throws Exception { // 2. Add consumer Consumer consumer1 = spy(new Consumer(sub, SubType.Exclusive, topic.getName(), 1 /* consumer id */, 0, "Cons1"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), - false /* read compacted */, InitialPosition.Latest, null)); + false /* read compacted */, InitialPosition.Latest, null, MessageId.latest)); pdfc.addConsumer(consumer1); List consumers = pdfc.getConsumers(); assertSame(consumers.get(0).consumerName(), consumer1.consumerName()); @@ -363,7 +364,7 @@ public void testAddRemoveConsumer() throws Exception { // 5. Add another consumer which does not change active consumer Consumer consumer2 = spy(new Consumer(sub, SubType.Exclusive, topic.getName(), 2 /* consumer id */, 0, "Cons2"/* consumer name */, - 50000, serverCnx, "myrole-1", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null)); + 50000, serverCnx, "myrole-1", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null, MessageId.latest)); pdfc.addConsumer(consumer2); consumers = pdfc.getConsumers(); assertSame(pdfc.getActiveConsumer().consumerName(), consumer1.consumerName()); @@ -377,7 +378,7 @@ public void testAddRemoveConsumer() throws Exception { // 6. Add a consumer which changes active consumer Consumer consumer0 = spy(new Consumer(sub, SubType.Exclusive, topic.getName(), 0 /* consumer id */, 0, "Cons0"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), - false /* read compacted */, InitialPosition.Latest, null)); + false /* read compacted */, InitialPosition.Latest, null, MessageId.latest)); pdfc.addConsumer(consumer0); consumers = pdfc.getConsumers(); assertSame(pdfc.getActiveConsumer().consumerName(), consumer0.consumerName()); @@ -460,7 +461,7 @@ public void testAddRemoveConsumerNonPartitionedTopic() throws Exception { // 2. Add a consumer Consumer consumer1 = spy(new Consumer(sub, SubType.Failover, topic.getName(), 1 /* consumer id */, 1, "Cons1"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), - false /* read compacted */, InitialPosition.Latest, null)); + false /* read compacted */, InitialPosition.Latest, null, MessageId.latest)); pdfc.addConsumer(consumer1); List consumers = pdfc.getConsumers(); assertEquals(1, consumers.size()); @@ -469,7 +470,7 @@ public void testAddRemoveConsumerNonPartitionedTopic() throws Exception { // 3. Add a consumer with same priority level and consumer name is smaller in lexicographic order. Consumer consumer2 = spy(new Consumer(sub, SubType.Failover, topic.getName(), 2 /* consumer id */, 1, "Cons2"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), - false /* read compacted */, InitialPosition.Latest, null)); + false /* read compacted */, InitialPosition.Latest, null, MessageId.latest)); pdfc.addConsumer(consumer2); // 4. Verify active consumer doesn't change @@ -482,7 +483,7 @@ public void testAddRemoveConsumerNonPartitionedTopic() throws Exception { // 5. Add another consumer which has higher priority level Consumer consumer3 = spy(new Consumer(sub, SubType.Failover, topic.getName(), 3 /* consumer id */, 0, "Cons3"/* consumer name */, - 50000, serverCnx, "myrole-1", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null)); + 50000, serverCnx, "myrole-1", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null, MessageId.latest)); pdfc.addConsumer(consumer3); consumers = pdfc.getConsumers(); assertEquals(3, consumers.size()); @@ -672,7 +673,7 @@ private Consumer getNextConsumer(PersistentDispatcherMultipleConsumers dispatche private Consumer createConsumer(int priority, int permit, boolean blocked, int id) throws Exception { Consumer consumer = new Consumer(null, SubType.Shared, "test-topic", id, priority, ""+id, 5000, - serverCnx, "appId", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null); + serverCnx, "appId", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); try { consumer.flowPermits(permit); } catch (Exception e) { 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 9de327754e3b71..97c2ae4d338312 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 @@ -93,6 +93,7 @@ import org.apache.pulsar.broker.cache.ConfigurationCacheService; import org.apache.pulsar.broker.cache.LocalZooKeeperCacheService; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentReplicator; import org.apache.pulsar.broker.service.persistent.CompactorSubscription; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; @@ -100,6 +101,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.PulsarClientImpl; @@ -122,6 +124,8 @@ import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.compaction.CompactedTopic; import org.apache.pulsar.compaction.Compactor; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.apache.pulsar.zookeeper.ZooKeeperCache; import org.apache.pulsar.zookeeper.ZooKeeperDataCache; import org.apache.zookeeper.ZooKeeper; @@ -148,6 +152,7 @@ public class PersistentTopicTest extends MockedBookKeeperTestCase { private BrokerService brokerService; private ManagedLedgerFactory mlFactoryMock; private ServerCnx serverCnx; + private MetadataStore store; private ManagedLedger ledgerMock; private ManagedCursor cursorMock; private ConfigurationCacheService configCacheService; @@ -203,6 +208,10 @@ public void setup() throws Exception { brokerService = spy(new BrokerService(pulsar, eventLoopGroup)); doReturn(brokerService).when(pulsar).getBrokerService(); + store = new ZKMetadataStore(mockZk); + PulsarResources pulsarResources = spy(new PulsarResources(store, store)); + doReturn(pulsarResources).when(pulsar).getPulsarResources(); + serverCnx = spy(new ServerCnx(pulsar)); doReturn(true).when(serverCnx).isActive(); doReturn(true).when(serverCnx).isWritable(); @@ -223,6 +232,7 @@ public void setup() throws Exception { @AfterMethod(alwaysRun = true) public void teardown() throws Exception { + metadataStore.close(); brokerService.getTopics().clear(); brokerService.close(); //to clear pulsarStats try { @@ -715,7 +725,7 @@ public void testChangeSubscriptionType() throws Exception { Consumer consumer = new Consumer(sub, SubType.Exclusive, topic.getName(), 1, 0, "Cons1", 50000, serverCnx, "myrole-1", Collections.emptyMap(), false, InitialPosition.Latest, - new KeySharedMeta().setKeySharedMode(KeySharedMode.AUTO_SPLIT)); + new KeySharedMeta().setKeySharedMode(KeySharedMode.AUTO_SPLIT), MessageId.latest); sub.addConsumer(consumer); consumer.close(); @@ -726,7 +736,7 @@ public void testChangeSubscriptionType() throws Exception { consumer = new Consumer(sub, subType, topic.getName(), 1, 0, "Cons1", 50000, serverCnx, "myrole-1", Collections.emptyMap(), false, InitialPosition.Latest, - new KeySharedMeta().setKeySharedMode(KeySharedMode.AUTO_SPLIT)); + new KeySharedMeta().setKeySharedMode(KeySharedMode.AUTO_SPLIT), MessageId.latest); sub.addConsumer(consumer); assertTrue(sub.getDispatcher().isConsumerConnected()); @@ -749,7 +759,7 @@ public void testAddRemoveConsumer() throws Exception { // 1. simple add consumer Consumer consumer = new Consumer(sub, SubType.Exclusive, topic.getName(), 1 /* consumer id */, 0, "Cons1"/* consumer name */, - 50000, serverCnx, "myrole-1", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null); + 50000, serverCnx, "myrole-1", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); sub.addConsumer(consumer); assertTrue(sub.getDispatcher().isConsumerConnected()); @@ -782,7 +792,7 @@ public void testAddRemoveConsumerDurableCursor() throws Exception { PersistentSubscription sub = new PersistentSubscription(topic, "non-durable-sub", cursorMock, false); Consumer consumer = new Consumer(sub, SubType.Exclusive, topic.getName(), 1, 0, "Cons1", 50000, serverCnx, - "myrole-1", Collections.emptyMap(), false, InitialPosition.Latest, null); + "myrole-1", Collections.emptyMap(), false, InitialPosition.Latest, null, MessageId.latest); sub.addConsumer(consumer); assertFalse(sub.getDispatcher().isClosed()); @@ -818,14 +828,14 @@ private void testMaxConsumersShared() throws Exception { // 1. add consumer1 Consumer consumer = new Consumer(sub, SubType.Shared, topic.getName(), 1 /* consumer id */, 0, "Cons1"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), - false /* read compacted */, InitialPosition.Latest, null); + false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); addConsumerToSubscription.invoke(topic, sub, consumer); assertEquals(sub.getConsumers().size(), 1); // 2. add consumer2 Consumer consumer2 = new Consumer(sub, SubType.Shared, topic.getName(), 2 /* consumer id */, 0, "Cons2"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), - false /* read compacted */, InitialPosition.Latest, null); + false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); addConsumerToSubscription.invoke(topic, sub, consumer2); assertEquals(sub.getConsumers().size(), 2); @@ -833,7 +843,7 @@ private void testMaxConsumersShared() throws Exception { try { Consumer consumer3 = new Consumer(sub, SubType.Shared, topic.getName(), 3 /* consumer id */, 0, "Cons3"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), - false /* read compacted */, InitialPosition.Latest, null); + false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); ((CompletableFuture) addConsumerToSubscription.invoke(topic, sub, consumer3)).get(); fail("should have failed"); } catch (ExecutionException e) { @@ -846,7 +856,7 @@ private void testMaxConsumersShared() throws Exception { // 4. add consumer4 to sub2 Consumer consumer4 = new Consumer(sub2, SubType.Shared, topic.getName(), 4 /* consumer id */, 0, "Cons4"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), - false /* read compacted */, InitialPosition.Latest, null); + false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); addConsumerToSubscription.invoke(topic, sub2, consumer4); assertEquals(sub2.getConsumers().size(), 1); @@ -857,7 +867,7 @@ private void testMaxConsumersShared() throws Exception { try { Consumer consumer5 = new Consumer(sub2, SubType.Shared, topic.getName(), 5 /* consumer id */, 0, "Cons5"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), - false /* read compacted */, InitialPosition.Latest, null); + false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); ((CompletableFuture) addConsumerToSubscription.invoke(topic, sub2, consumer5)).get(); fail("should have failed"); } catch (ExecutionException e) { @@ -913,14 +923,14 @@ private void testMaxConsumersFailover() throws Exception { // 1. add consumer1 Consumer consumer = new Consumer(sub, SubType.Failover, topic.getName(), 1 /* consumer id */, 0, "Cons1"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), - false /* read compacted */, InitialPosition.Latest, null); + false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); addConsumerToSubscription.invoke(topic, sub, consumer); assertEquals(sub.getConsumers().size(), 1); // 2. add consumer2 Consumer consumer2 = new Consumer(sub, SubType.Failover, topic.getName(), 2 /* consumer id */, 0, "Cons2"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), - false /* read compacted */, InitialPosition.Latest, null); + false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); addConsumerToSubscription.invoke(topic, sub, consumer2); assertEquals(sub.getConsumers().size(), 2); @@ -928,7 +938,7 @@ private void testMaxConsumersFailover() throws Exception { try { Consumer consumer3 = new Consumer(sub, SubType.Failover, topic.getName(), 3 /* consumer id */, 0, "Cons3"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), - false /* read compacted */, InitialPosition.Latest, null); + false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); ((CompletableFuture) addConsumerToSubscription.invoke(topic, sub, consumer3)).get(); fail("should have failed"); } catch (ExecutionException e) { @@ -941,7 +951,7 @@ private void testMaxConsumersFailover() throws Exception { // 4. add consumer4 to sub2 Consumer consumer4 = new Consumer(sub2, SubType.Failover, topic.getName(), 4 /* consumer id */, 0, "Cons4"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), - false /* read compacted */, InitialPosition.Latest, null); + false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); addConsumerToSubscription.invoke(topic, sub2, consumer4); assertEquals(sub2.getConsumers().size(), 1); @@ -952,7 +962,7 @@ private void testMaxConsumersFailover() throws Exception { try { Consumer consumer5 = new Consumer(sub2, SubType.Failover, topic.getName(), 5 /* consumer id */, 0, "Cons5"/* consumer name */, 50000, serverCnx, "myrole-1", Collections.emptyMap(), - false /* read compacted */, InitialPosition.Latest, null); + false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); ((CompletableFuture) addConsumerToSubscription.invoke(topic, sub2, consumer5)).get(); fail("should have failed"); } catch (ExecutionException e) { @@ -1000,7 +1010,7 @@ private Consumer getMockedConsumerWithSpecificAddress(Topic topic, Subscription doReturn(new PulsarCommandSenderImpl(null, cnx)).when(cnx).getCommandSender(); return new Consumer(sub, SubType.Shared, topic.getName(), consumerId, 0, consumerNameBase + consumerId, 50000, - cnx, role, Collections.emptyMap(), false, InitialPosition.Latest, null); + cnx, role, Collections.emptyMap(), false, InitialPosition.Latest, null, MessageId.latest); } @Test @@ -1108,7 +1118,7 @@ public void testUbsubscribeRaceConditions() throws Exception { PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); PersistentSubscription sub = new PersistentSubscription(topic, "sub-1", cursorMock, false); Consumer consumer1 = new Consumer(sub, SubType.Exclusive, topic.getName(), 1 /* consumer id */, 0, "Cons1"/* consumer name */, - 50000, serverCnx, "myrole-1", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null); + 50000, serverCnx, "myrole-1", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); sub.addConsumer(consumer1); doAnswer(new Answer() { @@ -1132,7 +1142,7 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { Thread.sleep(10); /* delay to ensure that the ubsubscribe gets executed first */ sub.addConsumer(new Consumer(sub, SubType.Exclusive, topic.getName(), 2 /* consumer id */, 0, "Cons2"/* consumer name */, 50000, serverCnx, - "myrole-1", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null)).get(); + "myrole-1", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null, MessageId.latest)).get(); fail(); } catch (Exception e) { assertTrue(e.getCause() instanceof BrokerServiceException.SubscriptionFencedException); @@ -1735,6 +1745,7 @@ public void testAtomicReplicationRemoval() throws Exception { doReturn(new ArrayList<>()).when(ledgerMock).getCursors(); PersistentTopic topic = new PersistentTopic(globalTopicName, ledgerMock, brokerService); + topic.initialize().join(); String remoteReplicatorName = topic.getReplicatorPrefix() + "." + remoteCluster; ConcurrentOpenHashMap replicatorMap = topic.getReplicators(); @@ -1756,13 +1767,10 @@ public void testAtomicReplicationRemoval() throws Exception { // step-2 now, policies doesn't have removed replication cluster so, it should not invoke "startProducer" of the // replicator - when(pulsar.getConfigurationCache().policiesCache() - .get(AdminResource.path(POLICIES, TopicName.get(globalTopicName).getNamespace()))) - .thenReturn(Optional.of(new Policies())); // try to start replicator again - topic.startReplProducers(); + topic.startReplProducers().join(); // verify: replicator.startProducer is not invoked - verify(replicator, Mockito.times(0)).startProducer(); + verify(replicator, Mockito.times(1)).startProducer(); // step-3 : complete the callback to remove replicator from the list ArgumentCaptor captor = ArgumentCaptor.forClass(DeleteCursorCallback.class); @@ -1934,21 +1942,21 @@ public void testBacklogCursor() throws Exception { ManagedCursor cursor1 = ledger.openCursor("c1"); PersistentSubscription sub1 = new PersistentSubscription(topic, "sub-1", cursor1, false); Consumer consumer1 = new Consumer(sub1, SubType.Exclusive, topic.getName(), 1 /* consumer id */, 0, "Cons1"/* consumer name */, - 50000, serverCnx, "myrole-1", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null); + 50000, serverCnx, "myrole-1", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); topic.getSubscriptions().put(Codec.decode(cursor1.getName()), sub1); sub1.addConsumer(consumer1); // Open cursor2, add it into activeCursor-container and add it into subscription consumer list ManagedCursor cursor2 = ledger.openCursor("c2"); PersistentSubscription sub2 = new PersistentSubscription(topic, "sub-2", cursor2, false); Consumer consumer2 = new Consumer(sub2, SubType.Exclusive, topic.getName(), 2 /* consumer id */, 0, "Cons2"/* consumer name */, - 50000, serverCnx, "myrole-2", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null); + 50000, serverCnx, "myrole-2", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); topic.getSubscriptions().put(Codec.decode(cursor2.getName()), sub2); sub2.addConsumer(consumer2); // Open cursor3, add it into activeCursor-container and do not add it into subscription consumer list ManagedCursor cursor3 = ledger.openCursor("c3"); PersistentSubscription sub3 = new PersistentSubscription(topic, "sub-3", cursor3, false); Consumer consumer3 = new Consumer(sub2, SubType.Exclusive, topic.getName(), 3 /* consumer id */, 0, "Cons2"/* consumer name */, - 50000, serverCnx, "myrole-3", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null); + 50000, serverCnx, "myrole-3", Collections.emptyMap(), false /* read compacted */, InitialPosition.Latest, null, MessageId.latest); topic.getSubscriptions().put(Codec.decode(cursor3.getName()), sub3); // Case1: cursors are active as haven't started deactivateBacklogCursor scan @@ -2058,7 +2066,7 @@ public void testCheckInactiveSubscriptions() throws Exception { addConsumerToSubscription.setAccessible(true); Consumer consumer = new Consumer(nonDeletableSubscription1, SubType.Shared, topic.getName(), 1, 0, "consumer1", - 50000, serverCnx, "app1", Collections.emptyMap(), false, InitialPosition.Latest, null); + 50000, serverCnx, "app1", Collections.emptyMap(), false, InitialPosition.Latest, null, MessageId.latest); addConsumerToSubscription.invoke(topic, nonDeletableSubscription1, consumer); when(pulsar.getConfigurationCache().policiesCache() 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 5fe18f761ccb8e..6036723472ad77 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,15 +21,11 @@ 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 io.netty.channel.ChannelHandler; -import io.netty.channel.EventLoopGroup; -import org.awaitility.Awaitility; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.matches; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.matches; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; @@ -37,15 +33,14 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; - import com.google.common.collect.Maps; - import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandler; +import io.netty.channel.EventLoopGroup; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.handler.codec.LengthFieldBasedFrameDecoder; - import java.io.IOException; import java.lang.reflect.Field; import java.nio.charset.StandardCharsets; @@ -55,16 +50,14 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; - +import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCursorCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenCursorCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenLedgerCallback; -import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; @@ -84,20 +77,18 @@ import org.apache.pulsar.broker.cache.LocalZooKeeperCacheService; import org.apache.pulsar.broker.intercept.BrokerInterceptor; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.resources.NamespaceResources; +import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException; import org.apache.pulsar.broker.service.ServerCnx.State; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.schema.DefaultSchemaRegistryService; import org.apache.pulsar.broker.service.utils.ClientChannelHelper; import org.apache.pulsar.common.api.AuthData; -import org.apache.pulsar.common.protocol.ByteBufPair; -import org.apache.pulsar.common.protocol.Commands; -import org.apache.pulsar.common.protocol.Commands.ChecksumType; -import org.apache.pulsar.common.protocol.PulsarHandler; import org.apache.pulsar.common.api.proto.AuthMethod; import org.apache.pulsar.common.api.proto.BaseCommand; +import org.apache.pulsar.common.api.proto.BaseCommand.Type; import org.apache.pulsar.common.api.proto.CommandAck.AckType; -import org.apache.pulsar.common.api.proto.CommandConnect; import org.apache.pulsar.common.api.proto.CommandConnected; import org.apache.pulsar.common.api.proto.CommandError; import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse; @@ -107,18 +98,23 @@ import org.apache.pulsar.common.api.proto.CommandSubscribe.InitialPosition; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.CommandSuccess; -import org.apache.pulsar.common.api.proto.EncryptionKeys; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.api.proto.ProtocolVersion; import org.apache.pulsar.common.api.proto.ServerError; -import org.apache.pulsar.common.api.proto.BaseCommand.Type; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.protocol.ByteBufPair; +import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.protocol.Commands.ChecksumType; +import org.apache.pulsar.common.protocol.PulsarHandler; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.apache.pulsar.zookeeper.ZooKeeperCache; import org.apache.pulsar.zookeeper.ZooKeeperDataCache; import org.apache.zookeeper.ZooKeeper; +import org.awaitility.Awaitility; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -136,7 +132,9 @@ public class ServerCnxTest { private ManagedLedgerFactory mlFactoryMock; private ClientChannelHelper clientChannelHelper; private PulsarService pulsar; + private MetadataStoreExtended store; private ConfigurationCacheService configCacheService; + private NamespaceResources namespaceResources; protected NamespaceService namespaceService; private final int currentProtocolVersion = ProtocolVersion.values()[ProtocolVersion.values().length - 1] .getValue(); @@ -180,6 +178,8 @@ public void setup() throws Exception { doReturn(createMockBookKeeper(executor)) .when(pulsar).getBookKeeperClient(); + store = new ZKMetadataStore(mockZk); + configCacheService = mock(ConfigurationCacheService.class); ZooKeeperDataCache zkDataCache = mock(ZooKeeperDataCache.class); doReturn(Optional.empty()).when(zkDataCache).get(any()); @@ -198,6 +198,11 @@ public void setup() throws Exception { doReturn(brokerService).when(pulsar).getBrokerService(); doReturn(executor).when(pulsar).getOrderedExecutor(); + PulsarResources pulsarResources = spy(new PulsarResources(store, store)); + namespaceResources = spy(new NamespaceResources(store, 30)); + doReturn(namespaceResources).when(pulsarResources).getNamespaceResources(); + doReturn(pulsarResources).when(pulsar).getPulsarResources(); + namespaceService = mock(NamespaceService.class); doReturn(CompletableFuture.completedFuture(null)).when(namespaceService).getBundleAsync(any()); doReturn(namespaceService).when(pulsar).getNamespaceService(); @@ -222,6 +227,7 @@ public void teardown() throws Exception { brokerService.close(); executor.shutdownNow(); eventLoopGroup.shutdownGracefully().get(); + store.close(); } @Test(timeOut = 30000) @@ -1045,7 +1051,8 @@ public void testSubscribeTimeout() throws Exception { assertEquals(((CommandError) response).getRequestId(), 5); // We should receive response for 1st producer, since it was not cancelled by the close - assertFalse(channel.outboundMessages().isEmpty()); + Awaitility.await().untilAsserted(() -> assertFalse(channel.outboundMessages().isEmpty())); + assertTrue(channel.isActive()); response = getResponse(); assertEquals(response.getClass(), CommandSuccess.class); @@ -1334,16 +1341,14 @@ public void testProducerFailureOnEncryptionRequiredTopic() throws Exception { setChannelConnected(); // Set encryption_required to true - ZooKeeperDataCache zkDataCache = mock(ZooKeeperDataCache.class); Policies policies = mock(Policies.class); policies.encryption_required = true; policies.topicDispatchRate = Maps.newHashMap(); // add `clusterDispatchRate` otherwise there will be a NPE // `org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.getPoliciesDispatchRate` policies.clusterDispatchRate = Maps.newHashMap(); - doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace())); - doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace())); - doReturn(zkDataCache).when(configCacheService).policiesCache(); + doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(namespaceResources) + .getAsync(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace())); // test failure case: unencrypted producer cannot connect ByteBuf clientCommand = Commands.newProducer(encryptionRequiredTopicName, 2 /* producer id */, 2 /* request id */, @@ -1441,16 +1446,13 @@ public void testSendFailureOnEncryptionRequiredTopic() throws Exception { setChannelConnected(); // Set encryption_required to true - ZooKeeperDataCache zkDataCache = mock(ZooKeeperDataCache.class); Policies policies = mock(Policies.class); policies.encryption_required = true; policies.topicDispatchRate = Maps.newHashMap(); // add `clusterDispatchRate` otherwise there will be a NPE // `org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.getPoliciesDispatchRate` policies.clusterDispatchRate = Maps.newHashMap(); - doReturn(Optional.of(policies)).when(zkDataCache).get(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace())); - doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(zkDataCache).getAsync(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace())); - doReturn(zkDataCache).when(configCacheService).policiesCache(); + doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(namespaceResources).getAsync(AdminResource.path(POLICIES, TopicName.get(encryptionRequiredTopicName).getNamespace())); ByteBuf clientCommand = Commands.newProducer(encryptionRequiredTopicName, 1 /* producer id */, 1 /* request id */, "prod-name", true, Collections.emptyMap()); 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 5fec7a65277318..2d0a5f938df5d9 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 @@ -49,6 +49,9 @@ public void testSimpleAggregation() { replStats1.msgRateOut = 2.0; replStats1.msgThroughputOut = 256.0; replStats1.replicationBacklog = 1; + replStats1.connectedCount = 0; + replStats1.msgRateExpired = 3.0; + replStats1.replicationDelayInSeconds = 20; topicStats1.replicationStats.put(namespace, replStats1); AggregatedSubscriptionStats subStats1 = new AggregatedSubscriptionStats(); @@ -77,6 +80,9 @@ public void testSimpleAggregation() { replStats2.msgRateOut = 10.5; replStats2.msgThroughputOut = 1536.0; replStats2.replicationBacklog = 99; + replStats2.connectedCount = 1; + replStats2.msgRateExpired = 3.0; + replStats2.replicationDelayInSeconds = 20; topicStats2.replicationStats.put(namespace, replStats2); AggregatedSubscriptionStats subStats2 = new AggregatedSubscriptionStats(); @@ -110,6 +116,9 @@ public void testSimpleAggregation() { assertEquals(nsReplStats.msgRateOut, 12.5); assertEquals(nsReplStats.msgThroughputOut, 1792.0); assertEquals(nsReplStats.replicationBacklog, 100); + assertEquals(nsReplStats.connectedCount, 1); + assertEquals(nsReplStats.msgRateExpired, 6.0); + assertEquals(nsReplStats.replicationDelayInSeconds, 40); AggregatedSubscriptionStats nsSubStats = nsStats.subscriptionStats.get(namespace); assertNotNull(nsSubStats); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/tools/BrokerToolTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/tools/BrokerToolTest.java new file mode 100644 index 00000000000000..9899bb36a875cb --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/tools/BrokerToolTest.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.broker.tools; + +import static org.testng.Assert.assertTrue; +import com.beust.jcommander.Parameter; +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.lang.reflect.Field; +import java.util.Arrays; +import org.testng.annotations.Test; + +/** + * Broker Tool Tests. + */ +public class BrokerToolTest { + + /** + * Test broker-tool generate docs + * + * @throws Exception + */ + @Test + public void testGenerateDocs() throws Exception { + PrintStream oldStream = System.out; + try { + ByteArrayOutputStream baoStream = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baoStream)); + + BrokerTool.run(new String[]{"gen-doc"}); + + String message = baoStream.toString(); + + Class argumentsClass = Class.forName("org.apache.pulsar.broker.tools.LoadReportCommand$Flags"); + Field[] fields = argumentsClass.getDeclaredFields(); + for (Field field : fields) { + boolean fieldHasAnno = field.isAnnotationPresent(Parameter.class); + if (fieldHasAnno) { + Parameter fieldAnno = field.getAnnotation(Parameter.class); + String[] names = fieldAnno.names(); + String nameStr = Arrays.asList(names).toString(); + nameStr = nameStr.substring(1, nameStr.length() - 1); + assertTrue(message.indexOf(nameStr) > 0); + } + } + } finally { + System.setOut(oldStream); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java new file mode 100644 index 00000000000000..a73d1f5a5d171c --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java @@ -0,0 +1,229 @@ +/** + * 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.compaction; + +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import io.swagger.models.auth.In; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Random; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.api.OpenBuilder; +import org.apache.bookkeeper.mledger.ManagedLedgerInfo; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.CryptoKeyReader; +import org.apache.pulsar.client.api.EncryptionKeyInfo; +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.ProducerBuilder; +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.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.impl.BatchMessageIdImpl; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class CompactionRetentionTest extends MockedPulsarServiceBaseTest { + private ScheduledExecutorService compactionScheduler; + private BookKeeper bk; + + @BeforeMethod + @Override + public void setup() throws Exception { + conf.setManagedLedgerMinLedgerRolloverTimeMinutes(0); + conf.setManagedLedgerMaxEntriesPerLedger(2); + super.internalSetup(); + + admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); + admin.tenants().createTenant("my-tenant", + new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("use"))); + admin.namespaces().createNamespace("my-tenant/use/my-ns"); + + compactionScheduler = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder().setNameFormat("compaction-%d").setDaemon(true).build()); + bk = pulsar.getBookKeeperClientFactory().create(this.conf, null, null, Optional.empty(), null); + } + + @AfterMethod(alwaysRun = true) + @Override + public void cleanup() throws Exception { + super.internalCleanup(); + + if (compactionScheduler != null) { + compactionScheduler.shutdownNow(); + } + } + + /** + * Compaction should retain expired keys in the compacted view + */ + @Test + public void testCompaction() throws Exception { + String topic = "persistent://my-tenant/use/my-ns/my-topic-" + System.nanoTime(); + + Set keys = Sets.newHashSet("a", "b", "c"); + Set keysToExpire = Sets.newHashSet("x1", "x2"); + Set allKeys = new HashSet<>(); + allKeys.addAll(keys); + allKeys.addAll(keysToExpire); + + ObjectMapper mapper = new ObjectMapper(); + mapper.configure(SerializationFeature.INDENT_OUTPUT, true); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .create(); + + Compactor compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); + compactor.compact(topic).join(); + + log.info(" ---- X 1: {}", mapper.writeValueAsString( + admin.topics().getInternalStats(topic, false))); + + int round = 1; + + for (String key : allKeys) { + producer.newMessage() + .key(key) + .value(round) + .send(); + } + + log.info(" ---- X 2: {}", mapper.writeValueAsString( + admin.topics().getInternalStats(topic, false))); + + validateMessages(pulsarClient, true, topic, round, allKeys); + + compactor.compact(topic).join(); + + log.info(" ---- X 3: {}", mapper.writeValueAsString( + admin.topics().getInternalStats(topic, false))); + + validateMessages(pulsarClient, true, topic, round, allKeys); + + round = 2; + + for (String key : allKeys) { + producer.newMessage() + .key(key) + .value(round) + .send(); + } + + compactor.compact(topic).join(); + + validateMessages(pulsarClient, true, topic, round, allKeys); + + // Now explicitly remove the expiring keys + for (String key : keysToExpire) { + producer.newMessage() + .key(key) + .send(); + } + + compactor.compact(topic).join(); + + log.info(" ---- X 4: {}", mapper.writeValueAsString( + admin.topics().getInternalStats(topic, false))); + + validateMessages(pulsarClient, true, topic, round, keys); + + // In the raw topic there should be no messages + validateMessages(pulsarClient, false, topic, round, Collections.emptySet()); + } + + private void validateMessages(PulsarClient client, boolean readCompacted, String topic, int round, Set expectedKeys) + throws Exception { + @Cleanup + Reader reader = client.newReader(Schema.INT32) + .topic(topic) + .startMessageId(MessageId.earliest) + .readCompacted(readCompacted) + .create(); + + Map receivedValues = new HashMap<>(); + + while (true) { + Message msg = reader.readNext(1, TimeUnit.SECONDS); + if (msg == null) { + break; + } + + Integer value = msg.size() > 0 ? msg.getValue() : null; + log.info("Received: {} -- value: {}", msg.getKey(), value); + if (value != null) { + receivedValues.put(msg.getKey(), value); + } + } + + Map expectedReceivedValues = new HashMap<>(); + expectedKeys.forEach(k -> expectedReceivedValues.put(k, round)); + + log.info("Received values: {}", receivedValues); + log.info("Expected values: {}", expectedReceivedValues); + assertEquals(receivedValues, expectedReceivedValues); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorToolTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorToolTest.java new file mode 100644 index 00000000000000..795cf2b7f7cd7e --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorToolTest.java @@ -0,0 +1,75 @@ +/** + * 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.compaction; + +import static org.testng.Assert.assertTrue; +import com.beust.jcommander.Parameter; +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.util.Arrays; +import org.apache.pulsar.common.util.CmdGenerateDocs; +import org.testng.annotations.Test; + +/** + * CompactorTool Tests. + */ +public class CompactorToolTest { + + /** + * Test broker-tool generate docs + * + * @throws Exception + */ + @Test + public void testGenerateDocs() throws Exception { + PrintStream oldStream = System.out; + try { + ByteArrayOutputStream baoStream = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baoStream)); + + Class argumentsClass = Class.forName("org.apache.pulsar.compaction.CompactorTool$Arguments"); + + Constructor constructor = argumentsClass.getDeclaredConstructor(); + constructor.setAccessible(true); + Object obj = constructor.newInstance(); + + CmdGenerateDocs cmd = new CmdGenerateDocs("pulsar"); + cmd.addCommand("compact-topic", obj); + cmd.run(null); + + String message = baoStream.toString(); + + Field[] fields = argumentsClass.getDeclaredFields(); + for (Field field : fields) { + boolean fieldHasAnno = field.isAnnotationPresent(Parameter.class); + if (fieldHasAnno) { + Parameter fieldAnno = field.getAnnotation(Parameter.class); + String[] names = fieldAnno.names(); + String nameStr = Arrays.asList(names).toString(); + nameStr = nameStr.substring(1, nameStr.length() - 1); + assertTrue(message.indexOf(nameStr) > 0); + } + } + } finally { + System.setOut(oldStream); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSinkE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSinkE2ETest.java index b921266a60c102..4c9604805603d0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSinkE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarSinkE2ETest.java @@ -50,6 +50,7 @@ import org.apache.pulsar.functions.LocalRunner; import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.worker.PulsarFunctionTestUtils; +import org.awaitility.Awaitility; import org.testng.annotations.Test; import com.google.common.collect.Lists; @@ -74,12 +75,19 @@ public void testReadCompactedSink() throws Exception { final int messageNum = 20; final int maxKeys = 10; // 1 Setup producer + @Cleanup Producer producer = pulsarClient.newProducer(Schema.STRING) .topic(sourceTopic) .enableBatching(false) .messageRoutingMode(MessageRoutingMode.SinglePartition) .create(); - pulsarClient.newConsumer().topic(sourceTopic).subscriptionName(subscriptionName).readCompacted(true).subscribe().close(); + pulsarClient.newConsumer() + .topic(sourceTopic) + .subscriptionName(subscriptionName) + .readCompacted(true) + .subscribe() + .close(); + // 2 Send messages and record the expected values after compaction Map expected = new HashMap<>(); for (int j = 0; j < messageNum; j++) { @@ -107,18 +115,12 @@ public void testReadCompactedSink() throws Exception { admin.sink().createSinkWithUrl(sinkConfig, jarFilePathUrl); // 5 Sink should only read compacted value,so we will only receive compacted messages - retryStrategically((test) -> { - try { - String prometheusMetrics = PulsarFunctionTestUtils.getPrometheusMetrics(pulsar.getListenPortHTTP().get()); - Map metrics = PulsarFunctionTestUtils.parseMetrics(prometheusMetrics); - PulsarFunctionTestUtils.Metric m = metrics.get("pulsar_sink_received_total"); - return m.value == (double) maxKeys; - } catch (Exception e) { - return false; - } - }, 50, 1000); - - producer.close(); + Awaitility.await().ignoreExceptions().untilAsserted(() -> { + String prometheusMetrics = PulsarFunctionTestUtils.getPrometheusMetrics(pulsar.getListenPortHTTP().get()); + Map metrics = PulsarFunctionTestUtils.parseMetrics(prometheusMetrics); + PulsarFunctionTestUtils.Metric m = metrics.get("pulsar_sink_received_total"); + assertEquals(m.value, maxKeys); + }); } @Test(timeOut = 30000) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtilsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtilsTest.java new file mode 100644 index 00000000000000..bca20941fcd3dd --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/utils/auth/tokens/TokensCliUtilsTest.java @@ -0,0 +1,84 @@ +/** + * 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.utils.auth.tokens; + +import static org.testng.Assert.assertTrue; +import com.beust.jcommander.Parameter; +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.lang.reflect.Field; +import java.util.Arrays; +import org.testng.annotations.Test; + +/** + * TokensCliUtils Tests. + */ +public class TokensCliUtilsTest { + + /** + * Test tokens generate docs. + * + * @throws Exception + */ + @Test + public void testGenerateDocs() throws Exception { + PrintStream oldStream = System.out; + try { + ByteArrayOutputStream baoStream = new ByteArrayOutputStream(); + System.setOut(new PrintStream(baoStream)); + + TokensCliUtils.main(new String[]{"gen-doc"}); + + String message = baoStream.toString(); + + String[] innerClassList = { + TokensCliUtils.CommandCreateSecretKey.class.getName(), + TokensCliUtils.CommandCreateKeyPair.class.getName(), + TokensCliUtils.CommandCreateToken.class.getName(), + TokensCliUtils.CommandShowToken.class.getName(), + TokensCliUtils.CommandValidateToken.class.getName() + }; + + for (String name : innerClassList) { + assertInnerClass(name, message); + } + + } finally { + System.setOut(oldStream); + } + } + + private void assertInnerClass(String className, String message) throws Exception { + Class argumentsClass = Class.forName(className); + Field[] fields = argumentsClass.getDeclaredFields(); + for (Field field : fields) { + boolean fieldHasAnno = field.isAnnotationPresent(Parameter.class); + if (fieldHasAnno) { + Parameter fieldAnno = field.getAnnotation(Parameter.class); + String[] names = fieldAnno.names(); + if (names.length < 1) { + continue; + } + String nameStr = Arrays.asList(names).toString(); + nameStr = nameStr.substring(1, nameStr.length() - 1); + assertTrue(message.indexOf(nameStr) > 0); + } + } + } +} diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java index 556418093fe222..6e66d3bc935167 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Namespaces.java @@ -910,7 +910,7 @@ CompletableFuture grantPermissionOnSubscriptionAsync( * @throws PulsarAdminException * Unexpected error */ - int getSubscriptionExpirationTime(String namespace) throws PulsarAdminException; + Integer getSubscriptionExpirationTime(String namespace) throws PulsarAdminException; /** * Get the subscription expiration time for a namespace asynchronously. @@ -965,6 +965,29 @@ CompletableFuture grantPermissionOnSubscriptionAsync( */ CompletableFuture setSubscriptionExpirationTimeAsync(String namespace, int expirationTime); + /** + * Remove the subscription expiration time for a namespace. + * + * @param namespace + * Namespace name + * + * @throws NotAuthorizedException + * Don't have admin permission + * @throws NotFoundException + * Namespace does not exist + * @throws PulsarAdminException + * Unexpected error + */ + void removeSubscriptionExpirationTime(String namespace) throws PulsarAdminException; + + /** + * Remove the subscription expiration time for a namespace asynchronously. + * + * @param namespace + * Namespace name + */ + CompletableFuture removeSubscriptionExpirationTimeAsync(String namespace); + /** * Set anti-affinity group name for a namespace. *

diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java index b2376f82afe9a6..ff773f6f8670fc 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java @@ -60,7 +60,7 @@ public class Policies { @SuppressWarnings("checkstyle:MemberName") public Integer message_ttl_in_seconds = null; @SuppressWarnings("checkstyle:MemberName") - public int subscription_expiration_time_minutes = 0; + public Integer subscription_expiration_time_minutes = null; @SuppressWarnings("checkstyle:MemberName") public RetentionPolicies retention_policies = null; public boolean deleted = false; diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java index ccf7890559b80a..eaecfbe01166c2 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/NamespacesImpl.java @@ -698,7 +698,7 @@ public CompletableFuture removeNamespaceMessageTTLAsync(String namespace) } @Override - public int getSubscriptionExpirationTime(String namespace) throws PulsarAdminException { + public Integer getSubscriptionExpirationTime(String namespace) throws PulsarAdminException { try { return getSubscriptionExpirationTimeAsync(namespace).get(this.readTimeoutMs, TimeUnit.MILLISECONDS); } catch (ExecutionException e) { @@ -753,6 +753,27 @@ public CompletableFuture setSubscriptionExpirationTimeAsync(String namespa return asyncPostRequest(path, Entity.entity(expirationTime, MediaType.APPLICATION_JSON)); } + @Override + public void removeSubscriptionExpirationTime(String namespace) throws PulsarAdminException { + try { + removeSubscriptionExpirationTimeAsync(namespace).get(this.readTimeoutMs, TimeUnit.MILLISECONDS); + } catch (ExecutionException e) { + throw (PulsarAdminException) e.getCause(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new PulsarAdminException(e); + } catch (TimeoutException e) { + throw new PulsarAdminException.TimeoutException(e); + } + } + + @Override + public CompletableFuture removeSubscriptionExpirationTimeAsync(String namespace) { + NamespaceName ns = NamespaceName.get(namespace); + WebTarget path = namespacePath(ns, "subscriptionExpirationTime"); + return asyncDeleteRequest(path); + } + @Override public void setNamespaceAntiAffinityGroup(String namespace, String namespaceAntiAffinityGroup) throws PulsarAdminException { 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 d122ded01996b6..f611358f44ba44 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 @@ -58,6 +58,7 @@ import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.ResetCursorData; +import org.apache.pulsar.common.api.proto.BrokerEntryMetadata; import org.apache.pulsar.common.api.proto.KeyValue; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.api.proto.SingleMessageMetadata; @@ -84,6 +85,7 @@ import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.Codec; +import org.apache.pulsar.common.util.DateFormatter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,6 +97,10 @@ public class TopicsImpl extends BaseResource implements Topics { private static final String BATCH_SIZE_HEADER = "X-Pulsar-batch-size"; private static final String MESSAGE_ID = "X-Pulsar-Message-ID"; private static final String PUBLISH_TIME = "X-Pulsar-publish-time"; + private static final String EVENT_TIME = "X-Pulsar-event-time"; + private static final String DELIVER_AT_TIME = "X-Pulsar-deliver-at-time"; + private static final String BROKER_ENTRY_TIMESTAMP = "X-Pulsar-Broker-Entry-METADATA-timestamp"; + private static final String BROKER_ENTRY_INDEX = "X-Pulsar-Broker-Entry-METADATA-index"; // CHECKSTYLE.ON: MemberName public TopicsImpl(WebTarget web, Authentication auth, long readTimeoutMs) { @@ -1504,6 +1510,24 @@ private List> getMessagesFromHttpResponse(String topic, Response } String msgId = response.getHeaderString(MESSAGE_ID); + + // build broker entry metadata if exist + String brokerEntryTimestamp = response.getHeaderString(BROKER_ENTRY_TIMESTAMP); + String brokerEntryIndex = response.getHeaderString(BROKER_ENTRY_INDEX); + BrokerEntryMetadata brokerEntryMetadata; + if (brokerEntryTimestamp == null && brokerEntryIndex == null) { + brokerEntryMetadata = null; + } else { + brokerEntryMetadata = new BrokerEntryMetadata(); + if (brokerEntryTimestamp != null) { + brokerEntryMetadata.setBrokerTimestamp(DateFormatter.parse(brokerEntryTimestamp.toString())); + } + + if (brokerEntryIndex != null) { + brokerEntryMetadata.setIndex(Long.parseLong(brokerEntryIndex)); + } + } + MessageMetadata messageMetadata = new MessageMetadata(); try (InputStream stream = (InputStream) response.getEntity()) { byte[] data = new byte[stream.available()]; @@ -1513,7 +1537,17 @@ private List> getMessagesFromHttpResponse(String topic, Response MultivaluedMap headers = response.getHeaders(); Object tmp = headers.getFirst(PUBLISH_TIME); if (tmp != null) { - properties.put("publish-time", (String) tmp); + messageMetadata.setPublishTime(DateFormatter.parse(tmp.toString())); + } + + tmp = headers.getFirst(EVENT_TIME); + if (tmp != null) { + messageMetadata.setEventTime(DateFormatter.parse(tmp.toString())); + } + + tmp = headers.getFirst(DELIVER_AT_TIME); + if (tmp != null) { + messageMetadata.setDeliverAtTime(DateFormatter.parse(tmp.toString())); } tmp = headers.getFirst("X-Pulsar-null-value"); @@ -1546,16 +1580,21 @@ private List> getMessagesFromHttpResponse(String topic, Response } if (!isEncrypted && response.getHeaderString(BATCH_HEADER) != null) { - return getIndividualMsgsFromBatch(topic, msgId, data, properties, messageMetadata); + return getIndividualMsgsFromBatch(topic, msgId, data, properties, messageMetadata, brokerEntryMetadata); } - return Collections.singletonList(new MessageImpl(topic, msgId, properties, - Unpooled.wrappedBuffer(data), Schema.BYTES, messageMetadata)); + MessageImpl message = new MessageImpl(topic, msgId, properties, + Unpooled.wrappedBuffer(data), Schema.BYTES, messageMetadata); + if (brokerEntryMetadata != null) { + message.setBrokerEntryMetadata(brokerEntryMetadata); + } + return Collections.singletonList(message); } } private List> getIndividualMsgsFromBatch(String topic, String msgId, byte[] data, - Map properties, MessageMetadata msgMetadataBuilder) { + Map properties, MessageMetadata msgMetadataBuilder, + BrokerEntryMetadata brokerEntryMetadata) { List> ret = new ArrayList<>(); int batchSize = Integer.parseInt(properties.get(BATCH_HEADER)); ByteBuf buf = Unpooled.wrappedBuffer(data); @@ -1570,8 +1609,12 @@ private List> getIndividualMsgsFromBatch(String topic, String ms properties.put(entry.getKey(), entry.getValue()); } } - ret.add(new MessageImpl<>(topic, batchMsgId, properties, singleMessagePayload, - Schema.BYTES, msgMetadataBuilder)); + MessageImpl message = new MessageImpl<>(topic, batchMsgId, properties, singleMessagePayload, + Schema.BYTES, msgMetadataBuilder); + if (brokerEntryMetadata != null) { + message.setBrokerEntryMetadata(brokerEntryMetadata); + } + ret.add(message); } catch (Exception ex) { log.error("Exception occurred while trying to get BatchMsgId: {}", batchMsgId, ex); } diff --git a/pulsar-client-cpp/CMakeLists.txt b/pulsar-client-cpp/CMakeLists.txt index 48a39643a05342..5cbbea6780b52b 100644 --- a/pulsar-client-cpp/CMakeLists.txt +++ b/pulsar-client-cpp/CMakeLists.txt @@ -27,6 +27,11 @@ if (VCPKG_TRIPLET) message(STATUS "Use CMAKE_PREFIX_PATH: ${CMAKE_PREFIX_PATH}") set(PROTOC_PATH "${CMAKE_PREFIX_PATH}/tools/protobuf/protoc") message(STATUS "Use protoc: ${PROTOC_PATH}") + set(VCPKG_DEBUG_ROOT "${CMAKE_SOURCE_DIR}/vcpkg_installed/${VCPKG_TRIPLET}/debug") + if (CMAKE_BUILD_TYPE STREQUAL "Debug") + set(ZLIB_ROOT ${VCPKG_DEBUG_ROOT}) + set(OPENSSL_ROOT_DIR ${VCPKG_DEBUG_ROOT}) + endif () endif() find_program(CCACHE_PROGRAM ccache) @@ -85,7 +90,6 @@ endif(MSVC) set(CMAKE_POSITION_INDEPENDENT_CODE ON) -set(Protobuf_LITE_LIBRARIES $ENV{PROTOBUF_LIBRARIES}) set(LOG_CATEGORY_NAME $ENV{LOG_CATEGORY_NAME}) if (NOT LOG_CATEGORY_NAME) @@ -123,6 +127,8 @@ if (LINK_STATIC) 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) + message(STATUS "Protobuf_LITE_LIBRARIES: ${Protobuf_LITE_LIBRARIES}") + set(COMMON_LIBS ${Protobuf_LITE_LIBRARIES} ${COMMON_LIBS}) if (USE_LOG4CXX) if (LOG4CXX_USE_DYNAMIC_LIBS) @@ -156,17 +162,46 @@ else() # Link to shared libraries find_package(ZLIB REQUIRED) set(ZLIB_LIBRARIES ${ZLIB_LIBRARIES}) - if (NOT PROTOBUF_LIBRARIES) - find_package(ProtoBuf QUIET) - if (NOT Protobuf_FOUND OR NOT Protobuf_LITE_LIBRARIES) - find_library(Protobuf_LITE_LIBRARIES protobuf-lite libprotobuf-lite) - find_path(Protobuf_INCLUDE_DIRS google/protobuf/stubs/common.h) - endif() - endif (NOT PROTOBUF_LIBRARIES) + # NOTE: The default MODULE mode may not find debug libraries so use CONFIG mode here + find_package(Protobuf QUIET CONFIG) + # NOTE: On Windows x86 platform, Protobuf_FOUND might be set false but Protobuf_INCLUDE_DIRS and + # Protobuf_LITE_LIBRARIES are both found. + if (Protobuf_INCLUDE_DIRS AND Protobuf_LITE_LIBRARIES AND NOT Protobuf_FOUND) + set(Protobuf_FOUND TRUE) + endif () + if (Protobuf_FOUND) + message("Found Protobuf in config mode") + message(STATUS "Protobuf_LITE_LIBRARIES: ${Protobuf_LITE_LIBRARIES}") + message(STATUS "Protobuf_INCLUDE_DIRS: ${Protobuf_INCLUDE_DIRS}") + else () + message("Failed to find Protobuf in config mode, try to find it from system path") + find_library(Protobuf_LITE_LIBRARIES protobuf-lite libprotobuf-lite) + find_path(Protobuf_INCLUDE_DIRS google/protobuf/stubs/common.h) + message(STATUS "Protobuf_LITE_LIBRARIES: ${Protobuf_LITE_LIBRARIES}") + message(STATUS "Protobuf_INCLUDE_DIRS: ${Protobuf_INCLUDE_DIRS}") + endif () + + if (${Protobuf_FOUND} AND (${CMAKE_VERSION} VERSION_GREATER 3.8)) + set(COMMON_LIBS protobuf::libprotobuf-lite ${COMMON_LIBS}) + else () + set(COMMON_LIBS ${Protobuf_LITE_LIBRARIES} ${COMMON_LIBS}) + endif () - find_library(LIB_ZSTD zstd) - find_library(LIB_SNAPPY NAMES snappy libsnappy) - find_library(CURL_LIBRARIES NAMES curl libcurl) + if (MSVC AND (${CMAKE_BUILD_TYPE} STREQUAL Debug)) + find_library(LIB_ZSTD zstdd HINTS "${VCPKG_DEBUG_ROOT}/lib") + else () + find_library(LIB_ZSTD zstd) + endif () + if (MSVC AND (${CMAKE_BUILD_TYPE} STREQUAL Debug)) + find_library(LIB_SNAPPY NAMES snappyd HINTS "${VCPKG_DEBUG_ROOT}/lib") + else () + find_library(LIB_SNAPPY NAMES snappy libsnappy) + endif () + + find_package(CURL REQUIRED) + if (${CMAKE_VERSION} VERSION_GREATER "3.12") + set(COMMON_LIBS ${COMMON_LIBS} CURL::libcurl) + endif () if (USE_LOG4CXX) find_library(LOG4CXX_LIBRARY_PATH log4cxx) @@ -295,7 +330,6 @@ set(COMMON_LIBS ${CURL_LIBRARIES} ${OPENSSL_LIBRARIES} ${ZLIB_LIBRARIES} - ${Protobuf_LITE_LIBRARIES} ${ADDITIONAL_LIBRARIES} ${CMAKE_DL_LIBS} ) diff --git a/pulsar-client-cpp/README.md b/pulsar-client-cpp/README.md index 61f6dff441485a..49c33337678c87 100644 --- a/pulsar-client-cpp/README.md +++ b/pulsar-client-cpp/README.md @@ -214,8 +214,10 @@ cmake --build ./build --config Release Then all artifacts will be built into `build` subdirectory. -> NOTE: For Windows 32-bit, you need to use `-A Win32` and `-DVCPKG_TRIPLET=x86-windows`. +> **NOTE** > +> 1. For Windows 32-bit, you need to use `-A Win32` and `-DVCPKG_TRIPLET=x86-windows`. +> 2. For MSVC Debug mode, you need to replace `Release` with `Debug` for both `CMAKE_BUILD_TYPE` variable and `--config` option. #### Install dependencies manually 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 47e075f0418e72..acbda97d613bfd 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 @@ -517,6 +517,9 @@ public void namespaces() throws Exception { namespaces.run(split("get-subscription-expiration-time myprop/clust/ns1")); verify(mockNamespaces).getSubscriptionExpirationTime("myprop/clust/ns1"); + namespaces.run(split("remove-subscription-expiration-time myprop/clust/ns1")); + verify(mockNamespaces).removeSubscriptionExpirationTime("myprop/clust/ns1"); + namespaces.run(split("set-anti-affinity-group myprop/clust/ns1 -g group")); verify(mockNamespaces).setNamespaceAntiAffinityGroup("myprop/clust/ns1", "group"); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java index bae1ae211f066c..a44022e50d1790 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdNamespaces.java @@ -424,6 +424,18 @@ void run() throws PulsarAdminException { } } + @Parameters(commandDescription = "Remove subscription expiration time for a namespace") + private class RemoveSubscriptionExpirationTime extends CliCommand { + @Parameter(description = "tenant/namespace", required = true) + private java.util.List params; + + @Override + void run() throws PulsarAdminException { + String namespace = validateNamespace(params); + getAdmin().namespaces().removeSubscriptionExpirationTime(namespace); + } + } + @Parameters(commandDescription = "Set Anti-affinity group name for a namespace") private class SetAntiAffinityGroup extends CliCommand { @Parameter(description = "tenant/namespace", required = true) @@ -2330,6 +2342,7 @@ public CmdNamespaces(Supplier admin) { jcommander.addCommand("get-subscription-expiration-time", new GetSubscriptionExpirationTime()); jcommander.addCommand("set-subscription-expiration-time", new SetSubscriptionExpirationTime()); + jcommander.addCommand("remove-subscription-expiration-time", new RemoveSubscriptionExpirationTime()); jcommander.addCommand("get-anti-affinity-group", new GetAntiAffinityGroup()); jcommander.addCommand("set-anti-affinity-group", new SetAntiAffinityGroup()); 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 96454da3cc01b8..fbb94b2001134a 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 @@ -52,6 +52,7 @@ import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BacklogQuota; @@ -837,18 +838,36 @@ void run() throws PulsarAdminException { List> messages = getTopics().peekMessages(persistentTopic, subName, numMessages); int position = 0; for (Message msg : messages) { + MessageImpl message = (MessageImpl) msg; if (++position != 1) { System.out.println("-------------------------------------------------------------------------\n"); } - if (msg.getMessageId() instanceof BatchMessageIdImpl) { - BatchMessageIdImpl msgId = (BatchMessageIdImpl) msg.getMessageId(); + if (message.getMessageId() instanceof BatchMessageIdImpl) { + BatchMessageIdImpl msgId = (BatchMessageIdImpl) message.getMessageId(); System.out.println("Batch Message ID: " + msgId.getLedgerId() + ":" + msgId.getEntryId() + ":" + msgId.getBatchIndex()); } else { MessageIdImpl msgId = (MessageIdImpl) msg.getMessageId(); System.out.println("Message ID: " + msgId.getLedgerId() + ":" + msgId.getEntryId()); } - if (msg.getProperties().size() > 0) { - System.out.println("Tenants:"); + + System.out.println("Publish time: " + message.getPublishTime()); + System.out.println("Event time: " + message.getEventTime()); + + if (message.getDeliverAtTime() != 0) { + System.out.println("Deliver at time: " + message.getDeliverAtTime()); + } + + if (message.getBrokerEntryMetadata() != null) { + if (message.getBrokerEntryMetadata().hasBrokerTimestamp()) { + System.out.println("Broker entry metadata timestamp: " + message.getBrokerEntryMetadata().getBrokerTimestamp()); + } + if (message.getBrokerEntryMetadata().hasIndex()) { + System.out.println("Broker entry metadata index: " + message.getBrokerEntryMetadata().getIndex()); + } + } + + if (message.getProperties().size() > 0) { + System.out.println("Properties:"); print(msg.getProperties()); } ByteBuf data = Unpooled.wrappedBuffer(msg.getData()); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java index 618aa0888ec109..1af69159b48207 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java @@ -335,6 +335,13 @@ public long getEventTime() { return 0; } + public long getDeliverAtTime() { + if (msgMetadata.hasDeliverAtTime()) { + return msgMetadata.getDeliverAtTime(); + } + return 0; + } + public boolean isExpired(int messageTTLInSeconds) { return messageTTLInSeconds != 0 && (brokerEntryMetadata == null || !brokerEntryMetadata.hasBrokerTimestamp() ? (System.currentTimeMillis() > 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 be4b3fc846cea6..cc31b1970a2fc0 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 @@ -121,6 +121,7 @@ public static void addLogicalTypeConversions(ReflectData reflectData, boolean js // Skip if have not provide joda-time dependency. } } + reflectData.addLogicalTypeConversion(new Conversions.UUIDConversion()); } public static class TimestampConversion extends Conversion { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/util/SchemaUtil.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/util/SchemaUtil.java index 70d7fc0bb51a0b..2d0c810d8f0294 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/util/SchemaUtil.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/util/SchemaUtil.java @@ -18,10 +18,12 @@ */ package org.apache.pulsar.client.impl.schema.util; +import org.apache.avro.Conversions; import org.apache.avro.Schema; import org.apache.avro.reflect.ReflectData; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.schema.SchemaDefinition; +import org.apache.pulsar.client.impl.schema.AvroSchema; import org.apache.pulsar.client.impl.schema.SchemaDefinitionBuilderImpl; import org.apache.pulsar.client.impl.schema.SchemaInfoImpl; import org.apache.pulsar.common.schema.SchemaInfo; @@ -89,8 +91,11 @@ public static Schema extractAvroSchema(SchemaDefinition schemaDefinition, Class 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); + ReflectData reflectData = schemaDefinition.getAlwaysAllowNull() + ? new ReflectData.AllowNull() + : new ReflectData(); + AvroSchema.addLogicalTypeConversions(reflectData, schemaDefinition.isJsr310ConversionEnabled()); + return reflectData.getSchema(pojo); } } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/AvroSchemaTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/AvroSchemaTest.java index 66653e30afcbe6..00cbbdd6ad201b 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/AvroSchemaTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/AvroSchemaTest.java @@ -35,6 +35,8 @@ import java.time.LocalTime; import java.time.temporal.ChronoUnit; import java.util.Arrays; +import java.util.UUID; + import lombok.Data; import lombok.extern.slf4j.Slf4j; import org.apache.avro.Schema; @@ -423,4 +425,17 @@ public void testAvroSchemaUserDefinedReadAndWriter() { assertEquals(field1, foo.getField1()); } + static class MyPojo { + public UUID uid; + } + + @Test + public void testAvroUUID() { + org.apache.pulsar.client.api.Schema schema = org.apache.pulsar.client.api.Schema.AVRO(MyPojo.class); + MyPojo pojo1 = new MyPojo(); + pojo1.uid = UUID.randomUUID(); + MyPojo pojo2 = schema.decode(schema.encode(pojo1)); + assertEquals(pojo1.uid, pojo2.uid); + } + } diff --git a/site2/docs/admin-api-brokers.md b/site2/docs/admin-api-brokers.md index 5a2939e8a2b98b..a41616b406f491 100644 --- a/site2/docs/admin-api-brokers.md +++ b/site2/docs/admin-api-brokers.md @@ -8,9 +8,9 @@ sidebar_label: Brokers > > This page only shows **some frequently used operations**. > -> - For the latest and complete information about `pulsar-admin`, including commands, flags, descriptions, and more, see [pulsar-admin doc](https://pulsar.apache.org/tools/pulsar-admin/). +> - For the latest and complete information about `Pulsar admin`, including commands, flags, descriptions, and more, see [Pulsar admin doc](https://pulsar.apache.org/tools/pulsar-admin/) > -> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see [REST API doc]{@inject: rest:REST:/}. +> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see {@inject: rest:REST:/} API doc. > > - For the latest and complete information about `Java admin API`, including classes, methods, descriptions, and more, see [Java admin API doc](https://pulsar.apache.org/api/admin/). diff --git a/site2/docs/admin-api-clusters.md b/site2/docs/admin-api-clusters.md index 45aacb3eb16c5d..5f88af79cff105 100644 --- a/site2/docs/admin-api-clusters.md +++ b/site2/docs/admin-api-clusters.md @@ -8,9 +8,9 @@ sidebar_label: Clusters > > This page only shows **some frequently used operations**. > -> - For the latest and complete information about `pulsar-admin`, including commands, flags, descriptions, and more, see [pulsar-admin doc](https://pulsar.apache.org/tools/pulsar-admin/). +> - For the latest and complete information about `Pulsar admin`, including commands, flags, descriptions, and more, see [Pulsar admin doc](https://pulsar.apache.org/tools/pulsar-admin/) > -> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see [REST API doc]{@inject: rest:REST:/}. +> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see {@inject: rest:REST:/} API doc. > > - For the latest and complete information about `Java admin API`, including classes, methods, descriptions, and more, see [Java admin API doc](https://pulsar.apache.org/api/admin/). diff --git a/site2/docs/admin-api-functions.md b/site2/docs/admin-api-functions.md index b5300516b6af45..50b89ba330d92a 100644 --- a/site2/docs/admin-api-functions.md +++ b/site2/docs/admin-api-functions.md @@ -8,9 +8,9 @@ sidebar_label: Functions > > This page only shows **some frequently used operations**. > -> - For the latest and complete information about `pulsar-admin`, including commands, flags, descriptions, and more, see [pulsar-admin doc](https://pulsar.apache.org/tools/pulsar-admin/). +> - For the latest and complete information about `Pulsar admin`, including commands, flags, descriptions, and more, see [Pulsar admin doc](https://pulsar.apache.org/tools/pulsar-admin/) > -> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see [REST API doc]{@inject: rest:REST:/}. +> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see {@inject: rest:REST:/} API doc. > > - For the latest and complete information about `Java admin API`, including classes, methods, descriptions, and more, see [Java admin API doc](https://pulsar.apache.org/api/admin/). diff --git a/site2/docs/admin-api-namespaces.md b/site2/docs/admin-api-namespaces.md index d1631a741c3a02..3e0e9a5b409bb3 100644 --- a/site2/docs/admin-api-namespaces.md +++ b/site2/docs/admin-api-namespaces.md @@ -8,9 +8,9 @@ sidebar_label: Namespaces > > This page only shows **some frequently used operations**. > -> - For the latest and complete information about `pulsar-admin`, including commands, flags, descriptions, and more, see [pulsar-admin doc](https://pulsar.apache.org/tools/pulsar-admin/). +> - For the latest and complete information about `Pulsar admin`, including commands, flags, descriptions, and more, see [Pulsar admin doc](https://pulsar.apache.org/tools/pulsar-admin/) > -> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see [REST API doc]{@inject: rest:REST:/}. +> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see {@inject: rest:REST:/} API doc. > > - For the latest and complete information about `Java admin API`, including classes, methods, descriptions, and more, see [Java admin API doc](https://pulsar.apache.org/api/admin/). diff --git a/site2/docs/admin-api-overview.md b/site2/docs/admin-api-overview.md index 0ea3105b8c96aa..e610e05939c99a 100644 --- a/site2/docs/admin-api-overview.md +++ b/site2/docs/admin-api-overview.md @@ -16,19 +16,19 @@ You can interact with the admin interface via: > **Important** > - > For the latest and complete information about Java admin API, including commands, flags, descriptions, and more, see [here](https://pulsar.apache.org/tools/pulsar-admin/). + > For the latest and complete information about `Pulsar admin`, including commands, flags, descriptions, and more, see [Pulsar admin doc](https://pulsar.apache.org/tools/pulsar-admin/). - HTTP calls, which are made against the admin {@inject: rest:REST:/} API provided by Pulsar brokers. For some RESTful APIs, they might be redirected to the owner brokers for serving with [`307 Temporary Redirect`](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status/307), hence the HTTP callers should handle `307 Temporary Redirect`. If you use `curl` commands, you should specify `-L` to handle redirections. > **Important** - > - > For the latest and complete information about REST API, including parameters, responses, samples, and more, see [REST API doc]{@inject: rest:REST:/}. + > + > For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see {@inject: rest:REST:/} API doc. - A Java client interface. > **Important** > - > For the latest and complete information about Java admin API, including classes, methods, descriptions, and more, see [Java admin API doc](https://pulsar.apache.org/api/admin/). + > For the latest and complete information about `Java admin API`, including classes, methods, descriptions, and more, see [Java admin API doc](https://pulsar.apache.org/api/admin/). > **The REST API is the admin interface**. Both the `pulsar-admin` CLI tool and the Java client use the REST API. If you implement your own admin interface client, you should use the REST API. diff --git a/site2/docs/admin-api-packages.md b/site2/docs/admin-api-packages.md index 6137dd89f1efe6..03e4d516764c75 100644 --- a/site2/docs/admin-api-packages.md +++ b/site2/docs/admin-api-packages.md @@ -8,9 +8,9 @@ sidebar_label: Packages > > This page only shows **some frequently used operations**. > -> - For the latest and complete information about `pulsar-admin`, including commands, flags, descriptions, and more, see [pulsar-admin doc](https://pulsar.apache.org/tools/pulsar-admin/). +> - For the latest and complete information about `Pulsar admin`, including commands, flags, descriptions, and more, see [Pulsar admin doc](https://pulsar.apache.org/tools/pulsar-admin/) > -> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see [REST API doc]{@inject: rest:REST:/}. +> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see {@inject: rest:REST:/} API doc. > > - For the latest and complete information about `Java admin API`, including classes, methods, descriptions, and more, see [Java admin API doc](https://pulsar.apache.org/api/admin/). diff --git a/site2/docs/admin-api-permissions.md b/site2/docs/admin-api-permissions.md index cedc47170795e1..78adb373d8cd6f 100644 --- a/site2/docs/admin-api-permissions.md +++ b/site2/docs/admin-api-permissions.md @@ -8,9 +8,9 @@ sidebar_label: Permissions > > This page only shows **some frequently used operations**. > -> - For the latest and complete information about `pulsar-admin`, including commands, flags, descriptions, and more, see [pulsar-admin doc](https://pulsar.apache.org/tools/pulsar-admin/). +> - For the latest and complete information about `Pulsar admin`, including commands, flags, descriptions, and more, see [Pulsar admin doc](https://pulsar.apache.org/tools/pulsar-admin/) > -> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see [REST API doc]{@inject: rest:REST:/}. +> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see {@inject: rest:REST:/} API doc. > > - For the latest and complete information about `Java admin API`, including classes, methods, descriptions, and more, see [Java admin API doc](https://pulsar.apache.org/api/admin/). diff --git a/site2/docs/admin-api-tenants.md b/site2/docs/admin-api-tenants.md index 7a6ef0cd838d3c..a2d82e497807ec 100644 --- a/site2/docs/admin-api-tenants.md +++ b/site2/docs/admin-api-tenants.md @@ -8,9 +8,9 @@ sidebar_label: Tenants > > This page only shows **some frequently used operations**. > -> - For the latest and complete information about `pulsar-admin`, including commands, flags, descriptions, and more, see [pulsar-admin doc](https://pulsar.apache.org/tools/pulsar-admin/). +> - For the latest and complete information about `Pulsar admin`, including commands, flags, descriptions, and more, see [Pulsar admin doc](https://pulsar.apache.org/tools/pulsar-admin/) > -> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see [REST API doc]{@inject: rest:REST:/}. +> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see {@inject: rest:REST:/} API doc. > > - For the latest and complete information about `Java admin API`, including classes, methods, descriptions, and more, see [Java admin API doc](https://pulsar.apache.org/api/admin/). diff --git a/site2/docs/admin-api-topics.md b/site2/docs/admin-api-topics.md index 32b001dcd82c39..8904cd19059e5b 100644 --- a/site2/docs/admin-api-topics.md +++ b/site2/docs/admin-api-topics.md @@ -8,9 +8,9 @@ sidebar_label: Topics > > This page only shows **some frequently used operations**. > -> - For the latest and complete information about `pulsar-admin`, including commands, flags, descriptions, and more, see [pulsar-admin doc](https://pulsar.apache.org/tools/pulsar-admin/). +> - For the latest and complete information about `Pulsar admin`, including commands, flags, descriptions, and more, see [Pulsar admin doc](https://pulsar.apache.org/tools/pulsar-admin/) > -> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see [REST API doc]{@inject: rest:REST:/}. +> - For the latest and complete information about `REST API`, including parameters, responses, samples, and more, see {@inject: rest:REST:/} API doc. > > - For the latest and complete information about `Java admin API`, including classes, methods, descriptions, and more, see [Java admin API doc](https://pulsar.apache.org/api/admin/). @@ -197,14 +197,42 @@ You can check the following statistics of a given non-partitioned topic. - **storageSize**: The sum of the ledgers' storage size for this topic. The space used to store the messages for the topic. + - **bytesInCounter**: Total bytes published to the topic. + + - **msgInCounter**: Total messages published to the topic. + + - **bytesOutCounter**: Total bytes delivered to consumers. + + - **msgOutCounter**: Total messages delivered to consumers. + + - **msgChunkPublished**: Topic has chunked message published on it. + + - **backlogSize**: Estimated total unconsumed or backlog size (in bytes). + + - **offloadedStorageSize**: Space used to store the offloaded messages for the topic (in bytes). + + - **waitingPublishers**: The number of publishers waiting in a queue in exclusive access mode. + + - **deduplicationStatus**: The status of message deduplication for the topic. + + - **topicEpoch**: The topic epoch or empty if not set. + + - **nonContiguousDeletedMessagesRanges**: The number of non-contiguous deleted messages ranges. + + - **nonContiguousDeletedMessagesRangesSerializedSize**: The serialized size of non-contiguous deleted messages ranges. + - **publishers**: The list of all local publishers into the topic. The list ranges from zero to thousands. + - **accessMode**: The type of access to the topic that the producer requires. + - **msgRateIn**: The total rate of messages (msg/s) published by this publisher. - **msgThroughputIn**: The total throughput (bytes/s) of the messages published by this publisher. - **averageMsgSize**: The average message size in bytes from this publisher within the last interval. + - **chunkedMessageRate**: Total chunked message count received for this producer on this topic. + - **producerId**: The internal identifier for this producer on this topic. - **producerName**: The internal identifier for this producer, generated by the client library. @@ -213,6 +241,10 @@ You can check the following statistics of a given non-partitioned topic. - **connectedSince**: The timestamp when this producer is created or reconnected last time. + - **clientVersion**: The client library version of this producer. + + - **metadata**: Metadata (key/value strings) associated with this publisher. + - **subscriptions**: The list of all local subscriptions to the topic. - **my-subscription**: The name of this subscription. It is defined by the client. @@ -229,12 +261,46 @@ You can check the following statistics of a given non-partitioned topic. - **lastExpireTimestamp**: The timestamp of the last message expire execution. - - **lastConsumedFlowTimestamp**: The timestamp of the last flow command received. - + - **lastConsumedFlowTimestamp**: The timestamp of the last flow command received. + - **lastConsumedTimestamp**: The latest timestamp of all the consumed timestamp of the consumers. - **lastAckedTimestamp**: The latest timestamp of all the acked timestamp of the consumers. + - **bytesOutCounter**: Total bytes delivered to consumer. + + - **msgOutCounter**: Total messages delivered to consumer. + + - **msgRateRedeliver**: Total rate of messages redelivered on this subscription (msg/s). + + - **chunkedMessageRate**: Chunked message dispatch rate. + + - **backlogSize**: Size of backlog for this subscription (in bytes). + + - **msgBacklogNoDelayed**: Number of messages in the subscription backlog that do not contain the delay messages. + + - **blockedSubscriptionOnUnackedMsgs**: Flag to verify if a subscription is blocked due to reaching threshold of unacked messages. + + - **msgDelayed**: Number of delayed messages currently being tracked. + + - **unackedMessages**: Number of unacknowledged messages for the subscription. + + - **activeConsumerName**: The name of the consumer that is active for single active consumer subscriptions. For example, failover or exclusive. + + - **totalMsgExpired**: Total messages expired on this subscription. + + - **lastMarkDeleteAdvancedTimestamp**: Last MarkDelete position advanced timestamp. + + - **durable**: Whether the subscription is durable or ephemeral (for example, from a reader). + + - **replicated**: Mark that the subscription state is kept in sync across different regions. + + - **consumersAfterMarkDeletePosition**: This is for Key_Shared subscription to get the recentJoinedConsumers in the Key_Shared subscription. + + - **nonContiguousDeletedMessagesRanges**: The number of non-contiguous deleted messages ranges. + + - **nonContiguousDeletedMessagesRangesSerializedSize**: The serialized size of non-contiguous deleted messages ranges. + - **consumers**: The list of connected consumers for this subscription. - **msgRateOut**: The total rate of messages (msg/s) delivered to the consumer. @@ -249,10 +315,32 @@ You can check the following statistics of a given non-partitioned topic. - **blockedConsumerOnUnackedMsgs**: The flag used to verify if the consumer is blocked due to reaching threshold of the unacknowledged messages. - - **lastConsumedTimestamp**: The timestamp when the consumer reads a message the last time. - + - **lastConsumedTimestamp**: The timestamp when the consumer reads a message the last time. + - **lastAckedTimestamp**: The timestamp when the consumer acknowledges a message the last time. + - **address**: The IP address and source port for the connection of this consumer. + + - **connectedSince**: The timestamp when this consumer is created or reconnected last time. + + - **clientVersion**: The client library version of this consumer. + + - **bytesOutCounter**: Total bytes delivered to consumer. + + - **msgOutCounter**: Total messages delivered to consumer. + + - **msgRateRedeliver**: Total rate of messages redelivered by this consumer (msg/s). + + - **chunkedMessageRate**: Total chunked messages dispatched. + + - **avgMessagesPerEntry**: Number of average messages per entry for the consumer consumed. + + - **readPositionWhenJoining**: The read position of the cursor when the consumer joining. + + - **keyHashRanges**: Hash ranges assigned to this consumer if is Key_Shared sub mode. + + - **metadata**: Metadata (key/value strings) associated with this consumer. + - **replication**: This section gives the stats for cross-colo replication of this topic - **msgRateIn**: The total rate (msg/s) of messages received from the remote cluster. @@ -283,34 +371,86 @@ The following is an example of a topic status. ```json { - "msgRateIn": 4641.528542257553, - "msgThroughputIn": 44663039.74947473, - "msgRateOut": 0, - "msgThroughputOut": 0, - "averageMsgSize": 1232439.816728665, - "storageSize": 135532389160, - "publishers": [ - { - "msgRateIn": 57.855383881403576, - "msgThroughputIn": 558994.7078932219, - "averageMsgSize": 613135, - "producerId": 0, - "producerName": null, - "address": null, - "connectedSince": null - } - ], - "subscriptions": { - "my-topic_subscription": { - "msgRateOut": 0, - "msgThroughputOut": 0, - "msgBacklog": 116632, - "type": null, - "msgRateExpired": 36.98245516804671, - "consumers": [] + "msgRateIn" : 0.0, + "msgThroughputIn" : 0.0, + "msgRateOut" : 0.0, + "msgThroughputOut" : 0.0, + "bytesInCounter" : 504, + "msgInCounter" : 9, + "bytesOutCounter" : 2296, + "msgOutCounter" : 41, + "averageMsgSize" : 0.0, + "msgChunkPublished" : false, + "storageSize" : 504, + "backlogSize" : 0, + "offloadedStorageSize" : 0, + "publishers" : [ { + "accessMode" : "Shared", + "msgRateIn" : 0.0, + "msgThroughputIn" : 0.0, + "averageMsgSize" : 0.0, + "chunkedMessageRate" : 0.0, + "producerId" : 0, + "metadata" : { }, + "address" : "/127.0.0.1:65402", + "connectedSince" : "2021-06-09T17:22:55.913+08:00", + "clientVersion" : "2.9.0-SNAPSHOT", + "producerName" : "standalone-1-0" + } ], + "waitingPublishers" : 0, + "subscriptions" : { + "sub-demo" : { + "msgRateOut" : 0.0, + "msgThroughputOut" : 0.0, + "bytesOutCounter" : 2296, + "msgOutCounter" : 41, + "msgRateRedeliver" : 0.0, + "chunkedMessageRate" : 0, + "msgBacklog" : 0, + "backlogSize" : 0, + "msgBacklogNoDelayed" : 0, + "blockedSubscriptionOnUnackedMsgs" : false, + "msgDelayed" : 0, + "unackedMessages" : 0, + "type" : "Exclusive", + "activeConsumerName" : "20b81", + "msgRateExpired" : 0.0, + "totalMsgExpired" : 0, + "lastExpireTimestamp" : 0, + "lastConsumedFlowTimestamp" : 1623230565356, + "lastConsumedTimestamp" : 1623230583946, + "lastAckedTimestamp" : 1623230584033, + "lastMarkDeleteAdvancedTimestamp" : 1623230584033, + "consumers" : [ { + "msgRateOut" : 0.0, + "msgThroughputOut" : 0.0, + "bytesOutCounter" : 2296, + "msgOutCounter" : 41, + "msgRateRedeliver" : 0.0, + "chunkedMessageRate" : 0.0, + "consumerName" : "20b81", + "availablePermits" : 959, + "unackedMessages" : 0, + "avgMessagesPerEntry" : 314, + "blockedConsumerOnUnackedMsgs" : false, + "lastAckedTimestamp" : 1623230584033, + "lastConsumedTimestamp" : 1623230583946, + "metadata" : { }, + "address" : "/127.0.0.1:65172", + "connectedSince" : "2021-06-09T17:22:45.353+08:00", + "clientVersion" : "2.9.0-SNAPSHOT" + } ], + "consumersAfterMarkDeletePosition" : { }, + "nonContiguousDeletedMessagesRanges" : 0, + "nonContiguousDeletedMessagesRangesSerializedSize" : 0, + "durable" : true, + "replicated" : false } }, - "replication": {} + "replication" : { }, + "deduplicationStatus" : "Disabled", + "nonContiguousDeletedMessagesRanges" : 0, + "nonContiguousDeletedMessagesRangesSerializedSize" : 0 } ``` To get the status of a topic, you can use the following ways. diff --git a/site2/docs/developing-binary-protocol.md b/site2/docs/developing-binary-protocol.md index 46585c26222b9a..434ffa74f3d0b2 100644 --- a/site2/docs/developing-binary-protocol.md +++ b/site2/docs/developing-binary-protocol.md @@ -61,7 +61,7 @@ Message metadata is stored alongside the application-specified payload as a seri | `publish_time` | The publish timestamp in Unix time (i.e. as the number of milliseconds since January 1st, 1970 in UTC) | | `properties` | A sequence of key/value pairs (using the [`KeyValue`](https://github.com/apache/pulsar/blob/master/pulsar-common/src/main/proto/PulsarApi.proto#L32) message). These are application-defined keys and values with no special meaning to Pulsar. | | `replicated_from` *(optional)* | Indicates that the message has been replicated and specifies the name of the [cluster](reference-terminology.md#cluster) where the message was originally published | -| `partition_key` *(optional)* | While publishing on a partition topic, if the key is present, the hash of the key is used to determine which partition to choose | +| `partition_key` *(optional)* | While publishing on a partition topic, if the key is present, the hash of the key is used to determine which partition to choose. Partition key is used as the message key. | | `compression` *(optional)* | Signals that payload has been compressed and with which compression library | | `uncompressed_size` *(optional)* | If compression is used, the producer must fill the uncompressed size field with the original payload size | | `num_messages_in_batch` *(optional)* | If this message is really a [batch](#batch-messages) of multiple entries, this field must be set to the number of messages in the batch | diff --git a/site2/docs/helm-deploy.md b/site2/docs/helm-deploy.md index d23eb4b07ea193..33f53541ba43b6 100644 --- a/site2/docs/helm-deploy.md +++ b/site2/docs/helm-deploy.md @@ -346,8 +346,9 @@ In this example, we name our Helm release `pulsar`. ```bash helm repo add apache https://pulsar.apache.org/charts helm repo update -helm upgrade --install pulsar apache/pulsar \ +helm install pulsar apache/pulsar \ --timeout 10m \ + --set initialize=true \ --set [your configuration options] ``` > **Note** diff --git a/site2/docs/reference-configuration.md b/site2/docs/reference-configuration.md index 6d8b13a3fb7c0f..46753292e78a72 100644 --- a/site2/docs/reference-configuration.md +++ b/site2/docs/reference-configuration.md @@ -58,7 +58,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| @@ -708,12 +708,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config | brokerWebServiceURLTLS | The TLS Web service URL pointing to the broker cluster | | | functionWorkerWebServiceURL | The Web service URL pointing to the function worker cluster. It is only configured when you setup function workers in a separate cluster. | | | functionWorkerWebServiceURLTLS | The TLS Web service URL pointing to the function worker cluster. It is only configured when you setup function workers in a separate cluster. | | -|brokerServiceURL|If service discovery is disabled, this url should point to the discovery service provider.|N/A| -|brokerServiceURLTLS|If service discovery is disabled, this url should point to the discovery service provider.|N/A| -|brokerWebServiceURL|This settings are unnecessary if `zookeeperServers` is specified.|N/A| -|brokerWebServiceURLTLS|This settings are unnecessary if `zookeeperServers` is specified.|N/A| -|functionWorkerWebServiceURL|If function workers are setup in a separate cluster, configure the this setting to point to the function workers cluster.|N/A| -|functionWorkerWebServiceURLTLS|If function workers are setup in a separate cluster, configure the this setting to point to the function workers cluster.|N/A| |zookeeperSessionTimeoutMs| ZooKeeper session timeout (in milliseconds) |30000| |zooKeeperCacheExpirySeconds|ZooKeeper cache expiry time in seconds|300| |advertisedAddress|Hostname or IP address the service advertises to the outside world. If not set, the value of `InetAddress.getLocalHost().getHostname()` is used.|N/A| @@ -731,7 +725,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |brokerClientAuthenticationParameters| The authentication parameters used by the Pulsar proxy to authenticate with Pulsar brokers || |brokerClientTrustCertsFilePath| The path to trusted certificates used by the Pulsar proxy to authenticate with Pulsar brokers || |superUserRoles| Role names that are treated as “super-users,” meaning that they will be able to perform all admin || -|forwardAuthorizationCredentials| Whether client authorization credentials are forwared to the broker for re-authorization. Authentication must be enabled via authenticationEnabled=true for this to take effect. |false| |maxConcurrentInboundConnections| Max concurrent inbound connections. The proxy will reject requests beyond that. |10000| |maxConcurrentLookupRequests| Max concurrent outbound connections. The proxy will error out requests beyond that. |50000| |tlsEnabledInProxy| Deprecated - use `servicePortTls` and `webServicePortTls` instead. |false| @@ -752,7 +745,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |tokenAuthClaim| Specify the token claim that will be used as the authentication "principal" or "role". The "subject" field will be used if this is left blank || |tokenAudienceClaim| The token audience "claim" name, e.g. "aud". It is used to get the audience from token. If it is not set, the audience is not verified. || | tokenAudience | The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token need contains this parameter.| | -| proxyLogLevel | Set the Pulsar Proxy log level.

  • If the value is set to 0, no TCP channel information is logged.
  • If the value is set to 1, only the TCP channel information and command information (without message body) are parsed and logged.
  • If the value is set to 2, all TCP channel information, command information, and message body are parsed and logged. | 0 | |haProxyProtocolEnabled | Enable or disable the [HAProxy](http://www.haproxy.org/) protocol. |false| ## ZooKeeper diff --git a/site2/docs/reference-metrics.md b/site2/docs/reference-metrics.md index cbb469334ddddb..f9be3bc3f1537e 100644 --- a/site2/docs/reference-metrics.md +++ b/site2/docs/reference-metrics.md @@ -165,6 +165,10 @@ All the replication metrics are also labelled with `remoteCluster=${pulsar_remot | pulsar_replication_throughput_in | Gauge | The total throughput of the namespace replicating from remote cluster (bytes/second). | | pulsar_replication_throughput_out | Gauge | The total throughput of the namespace replicating to remote cluster (bytes/second). | | pulsar_replication_backlog | Gauge | The total backlog of the namespace replicating to remote cluster (messages). | +| pulsar_replication_rate_expired | Gauge | Total rate of messages expired (messages/second). | +| pulsar_replication_connected_count | Gauge | The count of replication-subscriber up and running to replicate to remote cluster. | +| pulsar_replication_delay_in_seconds | Gauge | Time in seconds from the time a message was produced to the time when it is about to be replicated. | +~~~~ ### Topic metrics diff --git a/site2/docs/reference-pulsar-admin.md b/site2/docs/reference-pulsar-admin.md index 51a6fc2ba63eff..85061895ff8d88 100644 --- a/site2/docs/reference-pulsar-admin.md +++ b/site2/docs/reference-pulsar-admin.md @@ -6,7 +6,7 @@ sidebar_label: Pulsar Admin CLI > **Important** > -> This page is deprecated and not updated anymore. For the latest and complete information about `pulsar-admin`, including commands, flags, descriptions, and more, see [pulsar-admin doc](https://pulsar.apache.org/tools/pulsar-admin/). +> This page is deprecated and not updated anymore. For the latest and complete information about `Pulsar admin`, including commands, flags, descriptions, and more, see [Pulsar admin doc](https://pulsar.apache.org/tools/pulsar-admin/) The `pulsar-admin` tool enables you to manage Pulsar installations, including clusters, brokers, namespaces, tenants, and more. diff --git a/site2/tools/build-site.sh b/site2/tools/build-site.sh index da4ffdac080799..7bd3bae4fa4935 100755 --- a/site2/tools/build-site.sh +++ b/site2/tools/build-site.sh @@ -98,6 +98,7 @@ node ./scripts/split-swagger-by-version.js ${ROOT_DIR}/site2/tools/pulsar-admin-doc-gen.sh ${ROOT_DIR}/site2/tools/pulsar-client-doc-gen.sh ${ROOT_DIR}/site2/tools/pulsar-perf-doc-gen.sh +${ROOT_DIR}/site2/tools/pulsar-doc-gen.sh cd ${ROOT_DIR}/site2/website rm -rf ${ROOT_DIR}/generated-site/content diff --git a/site2/tools/pulsar-doc-gen.sh b/site2/tools/pulsar-doc-gen.sh new file mode 100755 index 00000000000000..d64be66e09c27e --- /dev/null +++ b/site2/tools/pulsar-doc-gen.sh @@ -0,0 +1,55 @@ +#!/usr/bin/env bash +# +# 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. +# + +ROOT_DIR=$(git rev-parse --show-toplevel) +VERSION=`${ROOT_DIR}/src/get-project-version.py` +DEST_DIR=$ROOT_DIR/generated-site + +cd $ROOT_DIR + +mkdir -p $DEST_DIR/tools/pulsar/${VERSION} +mkdir -p $DEST_DIR/tools/pulsar/${VERSION}/node_modules +mkdir -p $ROOT_DIR/site2/website/brodocs/documents + +$ROOT_DIR/bin/pulsar broker -g > $ROOT_DIR/site2/website/brodocs/documents/broker.md +$ROOT_DIR/bin/pulsar broker-tool gen-doc > $ROOT_DIR/site2/website/brodocs/documents/broker-tool.md +$ROOT_DIR/bin/pulsar compact-topic -t tmp -g > $ROOT_DIR/site2/website/brodocs/documents/compact-topic.md +$ROOT_DIR/bin/pulsar tokens gen-doc > $ROOT_DIR/site2/website/brodocs/documents/tokens.md + +cd $ROOT_DIR/site2/website/brodocs +cp pulsar-manifest.json manifest.json +node brodoc.js + +cp index.html $DEST_DIR/tools/pulsar/${VERSION}/ +cp navData.js stylesheet.css $DEST_DIR/tools/pulsar/${VERSION}/ +cp scroll.js tabvisibility.js $DEST_DIR/tools/pulsar/${VERSION}/ +cp favicon.ico $DEST_DIR/tools/pulsar/${VERSION}/ +mkdir -p $DEST_DIR/tools/pulsar/${VERSION}/node_modules/bootstrap/dist/css +cp -r $ROOT_DIR/site2/website/node_modules/bootstrap/dist/css/bootstrap.min.css $DEST_DIR/tools/pulsar/${VERSION}/node_modules/bootstrap/dist/css +mkdir -p $DEST_DIR/tools/pulsar/${VERSION}/node_modules/font-awesome/css +cp -r $ROOT_DIR/site2/website/node_modules/font-awesome/css/font-awesome.min.css $DEST_DIR/tools/pulsar/${VERSION}/node_modules/font-awesome/css +mkdir -p $DEST_DIR/tools/pulsar/${VERSION}/node_modules/highlight.js/styles +cp -r $ROOT_DIR/site2/website/node_modules/highlight.js/styles/default.css $DEST_DIR/tools/pulsar/${VERSION}/node_modules/highlight.js/styles +mkdir -p $DEST_DIR/tools/pulsar/${VERSION}/node_modules/jquery/dist +cp -r $ROOT_DIR/site2/website/node_modules/jquery/dist/jquery.min.js $DEST_DIR/tools/pulsar/${VERSION}/node_modules/jquery/dist/ +mkdir -p $DEST_DIR/tools/pulsar/${VERSION}/node_modules/jquery.scrollto +cp -r $ROOT_DIR/site2/website/node_modules/jquery.scrollto/jquery.scrollTo.min.js $DEST_DIR/tools/pulsar/${VERSION}/node_modules/jquery.scrollto + + diff --git a/site2/website/brodocs/pulsar-manifest.json b/site2/website/brodocs/pulsar-manifest.json new file mode 100644 index 00000000000000..586714e11ce331 --- /dev/null +++ b/site2/website/brodocs/pulsar-manifest.json @@ -0,0 +1,18 @@ +{ + "docs": [ + { + "filename": "broker.md" + }, + { + "filename": "broker-tool.md" + }, + { + "filename": "compact-topic.md" + }, + { + "filename": "tokens.md" + } + ], + "title": "Docs", + "copyright": "Apache Pulsar" +} diff --git a/site2/website/pages/en/pulsar-cli.js b/site2/website/pages/en/pulsar-cli.js new file mode 100644 index 00000000000000..388726e777036d --- /dev/null +++ b/site2/website/pages/en/pulsar-cli.js @@ -0,0 +1,24 @@ +const React = require('react'); +const CompLibrary = require('../../core/CompLibrary.js'); + +const Container = CompLibrary.Container; +const CWD = process.cwd(); +const releases = require(`${CWD}/releases.json`); + +class PulsarCli extends React.Component { + render() { + const latestVersion = releases[0]; + const url = "../js/getCliByVersion.js?latestVersion=" + latestVersion; + return ( +
    + + {latestVersion} + pulsar + + +
    + ); + } +} + +module.exports = PulsarCli; diff --git a/site2/website/pages/en/pulsar-perf-cli.js b/site2/website/pages/en/pulsar-perf-cli.js index b1e5f243b4357f..e6eaaaace0c678 100644 --- a/site2/website/pages/en/pulsar-perf-cli.js +++ b/site2/website/pages/en/pulsar-perf-cli.js @@ -5,7 +5,7 @@ const Container = CompLibrary.Container; const CWD = process.cwd(); const releases = require(`${CWD}/releases.json`); -class PulsarClientCli extends React.Component { +class PulsarPerfCli extends React.Component { render() { const latestVersion = releases[0]; const url = "../js/getCliByVersion.js?latestVersion=" + latestVersion; @@ -21,4 +21,4 @@ class PulsarClientCli extends React.Component { } } -module.exports = PulsarClientCli; +module.exports = PulsarPerfCli; diff --git a/site2/website/static/js/custom.js b/site2/website/static/js/custom.js index 4c8ad53cd9e09d..84d3c7daafe781 100644 --- a/site2/website/static/js/custom.js +++ b/site2/website/static/js/custom.js @@ -116,6 +116,7 @@ window.addEventListener('load', function () { '
  • Pulsar Admin
  • ' + '
  • Pulsar Client
  • ' + '
  • Pulsar Perf
  • ' + + '
  • Pulsar
  • ' + '' + '' + ''; diff --git a/site2/website/static/js/getCliByVersion.js b/site2/website/static/js/getCliByVersion.js index f23c30266ffe01..7f3ab5d4ff49ed 100644 --- a/site2/website/static/js/getCliByVersion.js +++ b/site2/website/static/js/getCliByVersion.js @@ -30,6 +30,9 @@ function getCliByVersion(){ } else if(clientModule === "pulsar-perf") { minMinorVersion = 8 referenceLink = "/reference-cli-tools/#pulsar-perf" + } else if(clientModule === "pulsar") { + minMinorVersion = 8 + referenceLink = "/reference-cli-tools/#pulsar" } if ((majorVersion > 1 && minorVersion <= minMinorVersion) || majorVersion === 1) { if (version === latestVersion) { diff --git a/site2/website/versioned_docs/version-2.1.0-incubating/reference-configuration.md b/site2/website/versioned_docs/version-2.1.0-incubating/reference-configuration.md index 5689bd152edd6a..951a545bd01092 100644 --- a/site2/website/versioned_docs/version-2.1.0-incubating/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.1.0-incubating/reference-configuration.md @@ -51,7 +51,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.1.1-incubating/reference-configuration.md b/site2/website/versioned_docs/version-2.1.1-incubating/reference-configuration.md index 72f630208d59ba..33bed775d85f6a 100644 --- a/site2/website/versioned_docs/version-2.1.1-incubating/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.1.1-incubating/reference-configuration.md @@ -51,7 +51,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.2.0/reference-configuration.md b/site2/website/versioned_docs/version-2.2.0/reference-configuration.md index f8782953f38fb4..1d49cdceaf4bc2 100644 --- a/site2/website/versioned_docs/version-2.2.0/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.2.0/reference-configuration.md @@ -51,7 +51,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.2.1/reference-configuration.md b/site2/website/versioned_docs/version-2.2.1/reference-configuration.md index bf00391f6a4440..9d6910914c3aa2 100644 --- a/site2/website/versioned_docs/version-2.2.1/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.2.1/reference-configuration.md @@ -51,7 +51,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.3.0/reference-configuration.md b/site2/website/versioned_docs/version-2.3.0/reference-configuration.md index 4783bb30f05a18..6654f0ef877671 100644 --- a/site2/website/versioned_docs/version-2.3.0/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.3.0/reference-configuration.md @@ -51,7 +51,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.3.1/reference-configuration.md b/site2/website/versioned_docs/version-2.3.1/reference-configuration.md index 47e4634f826682..75c445c630674f 100644 --- a/site2/website/versioned_docs/version-2.3.1/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.3.1/reference-configuration.md @@ -51,7 +51,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.3.2/reference-configuration.md b/site2/website/versioned_docs/version-2.3.2/reference-configuration.md index 498d5c09c3afef..1d1d11b9ea24c2 100644 --- a/site2/website/versioned_docs/version-2.3.2/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.3.2/reference-configuration.md @@ -51,7 +51,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.4.0/reference-configuration.md b/site2/website/versioned_docs/version-2.4.0/reference-configuration.md index 2a50fd048cc54e..95c8abdce719a1 100644 --- a/site2/website/versioned_docs/version-2.4.0/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.4.0/reference-configuration.md @@ -53,7 +53,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.4.1/reference-configuration.md b/site2/website/versioned_docs/version-2.4.1/reference-configuration.md index 1e9681058f9bb2..dbc2b475b5c65f 100644 --- a/site2/website/versioned_docs/version-2.4.1/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.4.1/reference-configuration.md @@ -53,7 +53,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.4.2/reference-configuration.md b/site2/website/versioned_docs/version-2.4.2/reference-configuration.md index 02caa6fc479330..0dbfc0f7df1048 100644 --- a/site2/website/versioned_docs/version-2.4.2/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.4.2/reference-configuration.md @@ -53,7 +53,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.5.0/reference-configuration.md b/site2/website/versioned_docs/version-2.5.0/reference-configuration.md index c331a0470d3627..f01a98ca6a7070 100644 --- a/site2/website/versioned_docs/version-2.5.0/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.5.0/reference-configuration.md @@ -52,7 +52,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.5.1/reference-configuration.md b/site2/website/versioned_docs/version-2.5.1/reference-configuration.md index fb59fb1ab3e45a..2728c13fa1eb28 100644 --- a/site2/website/versioned_docs/version-2.5.1/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.5.1/reference-configuration.md @@ -52,7 +52,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.5.2/reference-configuration.md b/site2/website/versioned_docs/version-2.5.2/reference-configuration.md index 6562c47bed23ae..77178395be8ade 100644 --- a/site2/website/versioned_docs/version-2.5.2/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.5.2/reference-configuration.md @@ -52,7 +52,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.6.0/helm-deploy.md b/site2/website/versioned_docs/version-2.6.0/helm-deploy.md index b9bd7163f3d007..794ea56dc222c0 100644 --- a/site2/website/versioned_docs/version-2.6.0/helm-deploy.md +++ b/site2/website/versioned_docs/version-2.6.0/helm-deploy.md @@ -347,8 +347,9 @@ In this example, we name our Helm release `pulsar`. ```bash helm repo add apache https://pulsar.apache.org/charts helm repo update -helm upgrade --install pulsar apache/pulsar \ +helm install pulsar apache/pulsar \ --timeout 10m \ + --set initialize=true \ --set [your configuration options] ``` > **Note** diff --git a/site2/website/versioned_docs/version-2.6.0/reference-configuration.md b/site2/website/versioned_docs/version-2.6.0/reference-configuration.md index 416fc527207b83..b3451db4375ffc 100644 --- a/site2/website/versioned_docs/version-2.6.0/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.6.0/reference-configuration.md @@ -52,7 +52,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.6.1/reference-configuration.md b/site2/website/versioned_docs/version-2.6.1/reference-configuration.md index a756379c620fbb..5348f37fbe929f 100644 --- a/site2/website/versioned_docs/version-2.6.1/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.6.1/reference-configuration.md @@ -52,7 +52,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.6.2/reference-configuration.md b/site2/website/versioned_docs/version-2.6.2/reference-configuration.md index ee8d7ba2aa0e81..eec900c57f1d9d 100644 --- a/site2/website/versioned_docs/version-2.6.2/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.6.2/reference-configuration.md @@ -52,7 +52,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.6.3/reference-configuration.md b/site2/website/versioned_docs/version-2.6.3/reference-configuration.md index 49baba79f65a02..dd51867cfc85de 100644 --- a/site2/website/versioned_docs/version-2.6.3/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.6.3/reference-configuration.md @@ -52,7 +52,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.6.4/reference-configuration.md b/site2/website/versioned_docs/version-2.6.4/reference-configuration.md index bfcf7ec1722153..79a5674ab4297a 100644 --- a/site2/website/versioned_docs/version-2.6.4/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.6.4/reference-configuration.md @@ -52,7 +52,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| diff --git a/site2/website/versioned_docs/version-2.7.0/helm-deploy.md b/site2/website/versioned_docs/version-2.7.0/helm-deploy.md index dc5c0d592e422d..ddaa92f5de8cbc 100644 --- a/site2/website/versioned_docs/version-2.7.0/helm-deploy.md +++ b/site2/website/versioned_docs/version-2.7.0/helm-deploy.md @@ -347,8 +347,9 @@ In this example, we name our Helm release `pulsar`. ```bash helm repo add apache https://pulsar.apache.org/charts helm repo update -helm upgrade --install pulsar apache/pulsar \ +helm install pulsar apache/pulsar \ --timeout 10m \ + --set initialize=true \ --set [your configuration options] ``` > **Note** diff --git a/site2/website/versioned_docs/version-2.7.0/reference-configuration.md b/site2/website/versioned_docs/version-2.7.0/reference-configuration.md index 62714c220c4de5..03cb3dc18932ce 100644 --- a/site2/website/versioned_docs/version-2.7.0/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.7.0/reference-configuration.md @@ -59,7 +59,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| @@ -696,12 +696,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config | brokerWebServiceURLTLS | The TLS Web service URL pointing to the broker cluster | | | functionWorkerWebServiceURL | The Web service URL pointing to the function worker cluster. It is only configured when you setup function workers in a separate cluster. | | | functionWorkerWebServiceURLTLS | The TLS Web service URL pointing to the function worker cluster. It is only configured when you setup function workers in a separate cluster. | | -|brokerServiceURL|If service discovery is disabled, this url should point to the discovery service provider.|N/A| -|brokerServiceURLTLS|If service discovery is disabled, this url should point to the discovery service provider.|N/A| -|brokerWebServiceURL|This settings are unnecessary if `zookeeperServers` is specified.|N/A| -|brokerWebServiceURLTLS|This settings are unnecessary if `zookeeperServers` is specified.|N/A| -|functionWorkerWebServiceURL|If function workers are setup in a separate cluster, configure the this setting to point to the function workers cluster.|N/A| -|functionWorkerWebServiceURLTLS|If function workers are setup in a separate cluster, configure the this setting to point to the function workers cluster.|N/A| |zookeeperSessionTimeoutMs| ZooKeeper session timeout (in milliseconds) |30000| |zooKeeperCacheExpirySeconds|ZooKeeper cache expiry time in seconds|300| |advertisedAddress|Hostname or IP address the service advertises to the outside world. If not set, the value of `InetAddress.getLocalHost().getHostname()` is used.|N/A| @@ -720,7 +714,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |brokerClientAuthenticationParameters| The authentication parameters used by the Pulsar proxy to authenticate with Pulsar brokers || |brokerClientTrustCertsFilePath| The path to trusted certificates used by the Pulsar proxy to authenticate with Pulsar brokers || |superUserRoles| Role names that are treated as “super-users,” meaning that they will be able to perform all admin || -|forwardAuthorizationCredentials| Whether client authorization credentials are forwared to the broker for re-authorization. Authentication must be enabled via authenticationEnabled=true for this to take effect. |false| |maxConcurrentInboundConnections| Max concurrent inbound connections. The proxy will reject requests beyond that. |10000| |maxConcurrentLookupRequests| Max concurrent outbound connections. The proxy will error out requests beyond that. |50000| |tlsEnabledInProxy| Deprecated - use `servicePortTls` and `webServicePortTls` instead. |false| @@ -741,7 +734,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |tokenAuthClaim| Specify the token claim that will be used as the authentication "principal" or "role". The "subject" field will be used if this is left blank || |tokenAudienceClaim| The token audience "claim" name, e.g. "aud". It is used to get the audience from token. If it is not set, the audience is not verified. || | tokenAudience | The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token need contains this parameter.| | -| proxyLogLevel | Set the Pulsar Proxy log level.
  • If the value is set to 0, no TCP channel information is logged.
  • If the value is set to 1, only the TCP channel information and command information (without message body) are parsed and logged.
  • If the value is set to 2, all TCP channel information, command information, and message body are parsed and logged. | 0 | |haProxyProtocolEnabled | Enable or disable the [HAProxy](http://www.haproxy.org/) protocol. |false| ## ZooKeeper diff --git a/site2/website/versioned_docs/version-2.7.1/helm-deploy.md b/site2/website/versioned_docs/version-2.7.1/helm-deploy.md index 1f23d0f9b4c2b3..e13e560174fbf7 100644 --- a/site2/website/versioned_docs/version-2.7.1/helm-deploy.md +++ b/site2/website/versioned_docs/version-2.7.1/helm-deploy.md @@ -347,8 +347,9 @@ In this example, we name our Helm release `pulsar`. ```bash helm repo add apache https://pulsar.apache.org/charts helm repo update -helm upgrade --install pulsar apache/pulsar \ +helm install pulsar apache/pulsar \ --timeout 10m \ + --set initialize=true \ --set [your configuration options] ``` > **Note** diff --git a/site2/website/versioned_docs/version-2.7.1/reference-configuration.md b/site2/website/versioned_docs/version-2.7.1/reference-configuration.md index 3dd98b4c0741ac..1114b9ef2201c8 100644 --- a/site2/website/versioned_docs/version-2.7.1/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.7.1/reference-configuration.md @@ -59,7 +59,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| @@ -699,12 +699,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config | brokerWebServiceURLTLS | The TLS Web service URL pointing to the broker cluster | | | functionWorkerWebServiceURL | The Web service URL pointing to the function worker cluster. It is only configured when you setup function workers in a separate cluster. | | | functionWorkerWebServiceURLTLS | The TLS Web service URL pointing to the function worker cluster. It is only configured when you setup function workers in a separate cluster. | | -|brokerServiceURL|If service discovery is disabled, this url should point to the discovery service provider.|N/A| -|brokerServiceURLTLS|If service discovery is disabled, this url should point to the discovery service provider.|N/A| -|brokerWebServiceURL|This settings are unnecessary if `zookeeperServers` is specified.|N/A| -|brokerWebServiceURLTLS|This settings are unnecessary if `zookeeperServers` is specified.|N/A| -|functionWorkerWebServiceURL|If function workers are setup in a separate cluster, configure the this setting to point to the function workers cluster.|N/A| -|functionWorkerWebServiceURLTLS|If function workers are setup in a separate cluster, configure the this setting to point to the function workers cluster.|N/A| |zookeeperSessionTimeoutMs| ZooKeeper session timeout (in milliseconds) |30000| |zooKeeperCacheExpirySeconds|ZooKeeper cache expiry time in seconds|300| |advertisedAddress|Hostname or IP address the service advertises to the outside world. If not set, the value of `InetAddress.getLocalHost().getHostname()` is used.|N/A| @@ -723,7 +717,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |brokerClientAuthenticationParameters| The authentication parameters used by the Pulsar proxy to authenticate with Pulsar brokers || |brokerClientTrustCertsFilePath| The path to trusted certificates used by the Pulsar proxy to authenticate with Pulsar brokers || |superUserRoles| Role names that are treated as “super-users,” meaning that they will be able to perform all admin || -|forwardAuthorizationCredentials| Whether client authorization credentials are forwared to the broker for re-authorization. Authentication must be enabled via authenticationEnabled=true for this to take effect. |false| |maxConcurrentInboundConnections| Max concurrent inbound connections. The proxy will reject requests beyond that. |10000| |maxConcurrentLookupRequests| Max concurrent outbound connections. The proxy will error out requests beyond that. |50000| |tlsEnabledInProxy| Deprecated - use `servicePortTls` and `webServicePortTls` instead. |false| @@ -744,7 +737,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |tokenAuthClaim| Specify the token claim that will be used as the authentication "principal" or "role". The "subject" field will be used if this is left blank || |tokenAudienceClaim| The token audience "claim" name, e.g. "aud". It is used to get the audience from token. If it is not set, the audience is not verified. || | tokenAudience | The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token need contains this parameter.| | -| proxyLogLevel | Set the Pulsar Proxy log level.
  • If the value is set to 0, no TCP channel information is logged.
  • If the value is set to 1, only the TCP channel information and command information (without message body) are parsed and logged.
  • If the value is set to 2, all TCP channel information, command information, and message body are parsed and logged. | 0 | |haProxyProtocolEnabled | Enable or disable the [HAProxy](http://www.haproxy.org/) protocol. |false| ## ZooKeeper diff --git a/site2/website/versioned_docs/version-2.7.2/helm-deploy.md b/site2/website/versioned_docs/version-2.7.2/helm-deploy.md index accae842f6c73b..d693c48c60c33f 100644 --- a/site2/website/versioned_docs/version-2.7.2/helm-deploy.md +++ b/site2/website/versioned_docs/version-2.7.2/helm-deploy.md @@ -347,8 +347,9 @@ In this example, we name our Helm release `pulsar`. ```bash helm repo add apache https://pulsar.apache.org/charts helm repo update -helm upgrade --install pulsar apache/pulsar \ +helm install pulsar apache/pulsar \ --timeout 10m \ + --set initialize=true \ --set [your configuration options] ``` > **Note** diff --git a/site2/website/versioned_docs/version-2.7.2/reference-configuration.md b/site2/website/versioned_docs/version-2.7.2/reference-configuration.md index cb5ae09a90d503..3c982fb1d54a82 100644 --- a/site2/website/versioned_docs/version-2.7.2/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.7.2/reference-configuration.md @@ -59,7 +59,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| @@ -699,12 +699,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config | brokerWebServiceURLTLS | The TLS Web service URL pointing to the broker cluster | | | functionWorkerWebServiceURL | The Web service URL pointing to the function worker cluster. It is only configured when you setup function workers in a separate cluster. | | | functionWorkerWebServiceURLTLS | The TLS Web service URL pointing to the function worker cluster. It is only configured when you setup function workers in a separate cluster. | | -|brokerServiceURL|If service discovery is disabled, this url should point to the discovery service provider.|N/A| -|brokerServiceURLTLS|If service discovery is disabled, this url should point to the discovery service provider.|N/A| -|brokerWebServiceURL|This settings are unnecessary if `zookeeperServers` is specified.|N/A| -|brokerWebServiceURLTLS|This settings are unnecessary if `zookeeperServers` is specified.|N/A| -|functionWorkerWebServiceURL|If function workers are setup in a separate cluster, configure the this setting to point to the function workers cluster.|N/A| -|functionWorkerWebServiceURLTLS|If function workers are setup in a separate cluster, configure the this setting to point to the function workers cluster.|N/A| |zookeeperSessionTimeoutMs| ZooKeeper session timeout (in milliseconds) |30000| |zooKeeperCacheExpirySeconds|ZooKeeper cache expiry time in seconds|300| |advertisedAddress|Hostname or IP address the service advertises to the outside world. If not set, the value of `InetAddress.getLocalHost().getHostname()` is used.|N/A| @@ -723,7 +717,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |brokerClientAuthenticationParameters| The authentication parameters used by the Pulsar proxy to authenticate with Pulsar brokers || |brokerClientTrustCertsFilePath| The path to trusted certificates used by the Pulsar proxy to authenticate with Pulsar brokers || |superUserRoles| Role names that are treated as “super-users,” meaning that they will be able to perform all admin || -|forwardAuthorizationCredentials| Whether client authorization credentials are forwared to the broker for re-authorization. Authentication must be enabled via authenticationEnabled=true for this to take effect. |false| |maxConcurrentInboundConnections| Max concurrent inbound connections. The proxy will reject requests beyond that. |10000| |maxConcurrentLookupRequests| Max concurrent outbound connections. The proxy will error out requests beyond that. |50000| |tlsEnabledInProxy| Deprecated - use `servicePortTls` and `webServicePortTls` instead. |false| @@ -744,7 +737,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |tokenAuthClaim| Specify the token claim that will be used as the authentication "principal" or "role". The "subject" field will be used if this is left blank || |tokenAudienceClaim| The token audience "claim" name, e.g. "aud". It is used to get the audience from token. If it is not set, the audience is not verified. || | tokenAudience | The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token need contains this parameter.| | -| proxyLogLevel | Set the Pulsar Proxy log level.
  • If the value is set to 0, no TCP channel information is logged.
  • If the value is set to 1, only the TCP channel information and command information (without message body) are parsed and logged.
  • If the value is set to 2, all TCP channel information, command information, and message body are parsed and logged. | 0 | |haProxyProtocolEnabled | Enable or disable the [HAProxy](http://www.haproxy.org/) protocol. |false| ## ZooKeeper diff --git a/site2/website/versioned_docs/version-2.7.3/helm-deploy.md b/site2/website/versioned_docs/version-2.7.3/helm-deploy.md index a69b8af994104a..8580122a096343 100644 --- a/site2/website/versioned_docs/version-2.7.3/helm-deploy.md +++ b/site2/website/versioned_docs/version-2.7.3/helm-deploy.md @@ -347,8 +347,9 @@ In this example, we name our Helm release `pulsar`. ```bash helm repo add apache https://pulsar.apache.org/charts helm repo update -helm upgrade --install pulsar apache/pulsar \ +helm install pulsar apache/pulsar \ --timeout 10m \ + --set initialize=true \ --set [your configuration options] ``` > **Note** diff --git a/site2/website/versioned_docs/version-2.7.3/reference-configuration.md b/site2/website/versioned_docs/version-2.7.3/reference-configuration.md index 44fa0a9d6acb42..15edf2edc0329b 100644 --- a/site2/website/versioned_docs/version-2.7.3/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.7.3/reference-configuration.md @@ -59,7 +59,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| @@ -699,12 +699,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config | brokerWebServiceURLTLS | The TLS Web service URL pointing to the broker cluster | | | functionWorkerWebServiceURL | The Web service URL pointing to the function worker cluster. It is only configured when you setup function workers in a separate cluster. | | | functionWorkerWebServiceURLTLS | The TLS Web service URL pointing to the function worker cluster. It is only configured when you setup function workers in a separate cluster. | | -|brokerServiceURL|If service discovery is disabled, this url should point to the discovery service provider.|N/A| -|brokerServiceURLTLS|If service discovery is disabled, this url should point to the discovery service provider.|N/A| -|brokerWebServiceURL|This settings are unnecessary if `zookeeperServers` is specified.|N/A| -|brokerWebServiceURLTLS|This settings are unnecessary if `zookeeperServers` is specified.|N/A| -|functionWorkerWebServiceURL|If function workers are setup in a separate cluster, configure the this setting to point to the function workers cluster.|N/A| -|functionWorkerWebServiceURLTLS|If function workers are setup in a separate cluster, configure the this setting to point to the function workers cluster.|N/A| |zookeeperSessionTimeoutMs| ZooKeeper session timeout (in milliseconds) |30000| |zooKeeperCacheExpirySeconds|ZooKeeper cache expiry time in seconds|300| |advertisedAddress|Hostname or IP address the service advertises to the outside world. If not set, the value of `InetAddress.getLocalHost().getHostname()` is used.|N/A| @@ -723,7 +717,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |brokerClientAuthenticationParameters| The authentication parameters used by the Pulsar proxy to authenticate with Pulsar brokers || |brokerClientTrustCertsFilePath| The path to trusted certificates used by the Pulsar proxy to authenticate with Pulsar brokers || |superUserRoles| Role names that are treated as “super-users,” meaning that they will be able to perform all admin || -|forwardAuthorizationCredentials| Whether client authorization credentials are forwared to the broker for re-authorization. Authentication must be enabled via authenticationEnabled=true for this to take effect. |false| |maxConcurrentInboundConnections| Max concurrent inbound connections. The proxy will reject requests beyond that. |10000| |maxConcurrentLookupRequests| Max concurrent outbound connections. The proxy will error out requests beyond that. |50000| |tlsEnabledInProxy| Deprecated - use `servicePortTls` and `webServicePortTls` instead. |false| @@ -744,7 +737,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |tokenAuthClaim| Specify the token claim that will be used as the authentication "principal" or "role". The "subject" field will be used if this is left blank || |tokenAudienceClaim| The token audience "claim" name, e.g. "aud". It is used to get the audience from token. If it is not set, the audience is not verified. || | tokenAudience | The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token need contains this parameter.| | -| proxyLogLevel | Set the Pulsar Proxy log level.
  • If the value is set to 0, no TCP channel information is logged.
  • If the value is set to 1, only the TCP channel information and command information (without message body) are parsed and logged.
  • If the value is set to 2, all TCP channel information, command information, and message body are parsed and logged. | 0 | |haProxyProtocolEnabled | Enable or disable the [HAProxy](http://www.haproxy.org/) protocol. |false| ## ZooKeeper diff --git a/site2/website/versioned_docs/version-2.8.0/helm-deploy.md b/site2/website/versioned_docs/version-2.8.0/helm-deploy.md index 4c66d53ad38e43..2cb7eb1f6e25f6 100644 --- a/site2/website/versioned_docs/version-2.8.0/helm-deploy.md +++ b/site2/website/versioned_docs/version-2.8.0/helm-deploy.md @@ -347,8 +347,9 @@ In this example, we name our Helm release `pulsar`. ```bash helm repo add apache https://pulsar.apache.org/charts helm repo update -helm upgrade --install pulsar apache/pulsar \ +helm install pulsar apache/pulsar \ --timeout 10m \ + --set initialize=true \ --set [your configuration options] ``` > **Note** diff --git a/site2/website/versioned_docs/version-2.8.0/reference-configuration.md b/site2/website/versioned_docs/version-2.8.0/reference-configuration.md index a131353b419188..7f3f2820042b7d 100644 --- a/site2/website/versioned_docs/version-2.8.0/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.8.0/reference-configuration.md @@ -59,7 +59,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| @@ -709,12 +709,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config | brokerWebServiceURLTLS | The TLS Web service URL pointing to the broker cluster | | | functionWorkerWebServiceURL | The Web service URL pointing to the function worker cluster. It is only configured when you setup function workers in a separate cluster. | | | functionWorkerWebServiceURLTLS | The TLS Web service URL pointing to the function worker cluster. It is only configured when you setup function workers in a separate cluster. | | -|brokerServiceURL|If service discovery is disabled, this url should point to the discovery service provider.|N/A| -|brokerServiceURLTLS|If service discovery is disabled, this url should point to the discovery service provider.|N/A| -|brokerWebServiceURL|This settings are unnecessary if `zookeeperServers` is specified.|N/A| -|brokerWebServiceURLTLS|This settings are unnecessary if `zookeeperServers` is specified.|N/A| -|functionWorkerWebServiceURL|If function workers are setup in a separate cluster, configure the this setting to point to the function workers cluster.|N/A| -|functionWorkerWebServiceURLTLS|If function workers are setup in a separate cluster, configure the this setting to point to the function workers cluster.|N/A| |zookeeperSessionTimeoutMs| ZooKeeper session timeout (in milliseconds) |30000| |zooKeeperCacheExpirySeconds|ZooKeeper cache expiry time in seconds|300| |advertisedAddress|Hostname or IP address the service advertises to the outside world. If not set, the value of `InetAddress.getLocalHost().getHostname()` is used.|N/A| @@ -732,7 +726,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |brokerClientAuthenticationParameters| The authentication parameters used by the Pulsar proxy to authenticate with Pulsar brokers || |brokerClientTrustCertsFilePath| The path to trusted certificates used by the Pulsar proxy to authenticate with Pulsar brokers || |superUserRoles| Role names that are treated as “super-users,” meaning that they will be able to perform all admin || -|forwardAuthorizationCredentials| Whether client authorization credentials are forwared to the broker for re-authorization. Authentication must be enabled via authenticationEnabled=true for this to take effect. |false| |maxConcurrentInboundConnections| Max concurrent inbound connections. The proxy will reject requests beyond that. |10000| |maxConcurrentLookupRequests| Max concurrent outbound connections. The proxy will error out requests beyond that. |50000| |tlsEnabledInProxy| Deprecated - use `servicePortTls` and `webServicePortTls` instead. |false| @@ -753,7 +746,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |tokenAuthClaim| Specify the token claim that will be used as the authentication "principal" or "role". The "subject" field will be used if this is left blank || |tokenAudienceClaim| The token audience "claim" name, e.g. "aud". It is used to get the audience from token. If it is not set, the audience is not verified. || | tokenAudience | The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token need contains this parameter.| | -| proxyLogLevel | Set the Pulsar Proxy log level.
  • If the value is set to 0, no TCP channel information is logged.
  • If the value is set to 1, only the TCP channel information and command information (without message body) are parsed and logged.
  • If the value is set to 2, all TCP channel information, command information, and message body are parsed and logged. | 0 | |haProxyProtocolEnabled | Enable or disable the [HAProxy](http://www.haproxy.org/) protocol. |false| ## ZooKeeper diff --git a/site2/website/versioned_docs/version-2.8.1/helm-deploy.md b/site2/website/versioned_docs/version-2.8.1/helm-deploy.md index f2881c52b810fe..2099f0f903cfab 100644 --- a/site2/website/versioned_docs/version-2.8.1/helm-deploy.md +++ b/site2/website/versioned_docs/version-2.8.1/helm-deploy.md @@ -347,8 +347,9 @@ In this example, we name our Helm release `pulsar`. ```bash helm repo add apache https://pulsar.apache.org/charts helm repo update -helm upgrade --install pulsar apache/pulsar \ +helm install pulsar apache/pulsar \ --timeout 10m \ + --set initialize=true \ --set [your configuration options] ``` > **Note** diff --git a/site2/website/versioned_docs/version-2.8.1/reference-configuration.md b/site2/website/versioned_docs/version-2.8.1/reference-configuration.md index 8448e449762ced..4814c6fc57c116 100644 --- a/site2/website/versioned_docs/version-2.8.1/reference-configuration.md +++ b/site2/website/versioned_docs/version-2.8.1/reference-configuration.md @@ -59,7 +59,7 @@ BookKeeper is a replicated log storage system that Pulsar uses for durable stora |compactionRateByEntries|The rate at which compaction will read entries, in adds per second.|1000| |compactionRateByBytes|Set the rate at which compaction will readd entries. The unit is bytes added per second.|1000000| |journalMaxSizeMB|Max file size of journal file, in megabytes. A new journal file will be created when the old one reaches the file size limitation.|2048| -|journalMaxBackups|The max number of old journal filse to keep. Keeping a number of old journal files would help data recovery in special cases.|5| +|journalMaxBackups|The max number of old journal files to keep. Keeping a number of old journal files would help data recovery in special cases.|5| |journalPreAllocSizeMB|How space to pre-allocate at a time in the journal.|16| |journalWriteBufferSizeKB|The of the write buffers used for the journal.|64| |journalRemoveFromPageCache|Whether pages should be removed from the page cache after force write.|true| @@ -709,12 +709,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config | brokerWebServiceURLTLS | The TLS Web service URL pointing to the broker cluster | | | functionWorkerWebServiceURL | The Web service URL pointing to the function worker cluster. It is only configured when you setup function workers in a separate cluster. | | | functionWorkerWebServiceURLTLS | The TLS Web service URL pointing to the function worker cluster. It is only configured when you setup function workers in a separate cluster. | | -|brokerServiceURL|If service discovery is disabled, this url should point to the discovery service provider.|N/A| -|brokerServiceURLTLS|If service discovery is disabled, this url should point to the discovery service provider.|N/A| -|brokerWebServiceURL|This settings are unnecessary if `zookeeperServers` is specified.|N/A| -|brokerWebServiceURLTLS|This settings are unnecessary if `zookeeperServers` is specified.|N/A| -|functionWorkerWebServiceURL|If function workers are setup in a separate cluster, configure the this setting to point to the function workers cluster.|N/A| -|functionWorkerWebServiceURLTLS|If function workers are setup in a separate cluster, configure the this setting to point to the function workers cluster.|N/A| |zookeeperSessionTimeoutMs| ZooKeeper session timeout (in milliseconds) |30000| |zooKeeperCacheExpirySeconds|ZooKeeper cache expiry time in seconds|300| |advertisedAddress|Hostname or IP address the service advertises to the outside world. If not set, the value of `InetAddress.getLocalHost().getHostname()` is used.|N/A| @@ -732,7 +726,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |brokerClientAuthenticationParameters| The authentication parameters used by the Pulsar proxy to authenticate with Pulsar brokers || |brokerClientTrustCertsFilePath| The path to trusted certificates used by the Pulsar proxy to authenticate with Pulsar brokers || |superUserRoles| Role names that are treated as “super-users,” meaning that they will be able to perform all admin || -|forwardAuthorizationCredentials| Whether client authorization credentials are forwared to the broker for re-authorization. Authentication must be enabled via authenticationEnabled=true for this to take effect. |false| |maxConcurrentInboundConnections| Max concurrent inbound connections. The proxy will reject requests beyond that. |10000| |maxConcurrentLookupRequests| Max concurrent outbound connections. The proxy will error out requests beyond that. |50000| |tlsEnabledInProxy| Deprecated - use `servicePortTls` and `webServicePortTls` instead. |false| @@ -753,7 +746,6 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |tokenAuthClaim| Specify the token claim that will be used as the authentication "principal" or "role". The "subject" field will be used if this is left blank || |tokenAudienceClaim| The token audience "claim" name, e.g. "aud". It is used to get the audience from token. If it is not set, the audience is not verified. || | tokenAudience | The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token need contains this parameter.| | -| proxyLogLevel | Set the Pulsar Proxy log level.
  • If the value is set to 0, no TCP channel information is logged.
  • If the value is set to 1, only the TCP channel information and command information (without message body) are parsed and logged.
  • If the value is set to 2, all TCP channel information, command information, and message body are parsed and logged. | 0 | |haProxyProtocolEnabled | Enable or disable the [HAProxy](http://www.haproxy.org/) protocol. |false| ## ZooKeeper diff --git a/site2/website/versioned_docs/version-2.8.1/reference-metrics.md b/site2/website/versioned_docs/version-2.8.1/reference-metrics.md index a96a97f2aaf6dd..f81378d23d37d3 100644 --- a/site2/website/versioned_docs/version-2.8.1/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.8.1/reference-metrics.md @@ -166,6 +166,9 @@ All the replication metrics are also labelled with `remoteCluster=${pulsar_remot | pulsar_replication_throughput_in | Gauge | The total throughput of the namespace replicating from remote cluster (bytes/second). | | pulsar_replication_throughput_out | Gauge | The total throughput of the namespace replicating to remote cluster (bytes/second). | | pulsar_replication_backlog | Gauge | The total backlog of the namespace replicating to remote cluster (messages). | +| pulsar_replication_rate_expired | Gauge | Total rate of messages expired (messages/second). | +| pulsar_replication_connected_count | Gauge | The count of replication-subscriber up and running to replicate to remote cluster. | +| pulsar_replication_delay_in_seconds | Gauge | Time in seconds from the time a message was produced to the time when it is about to be replicated. | ### Topic metrics